Re: [HACKERS] WAL logging problem in 9.4.3?

Поиск
Список
Период
Сортировка
От Kyotaro HORIGUCHI
Тема Re: [HACKERS] WAL logging problem in 9.4.3?
Дата
Msg-id 20190326.163507.239339952.horiguchi.kyotaro@lab.ntt.co.jp
обсуждение исходный текст
Ответ на Re: [HACKERS] WAL logging problem in 9.4.3?  (Kyotaro HORIGUCHI <horiguchi.kyotaro@lab.ntt.co.jp>)
Ответы Re: [HACKERS] WAL logging problem in 9.4.3?  (Noah Misch <noah@leadboat.com>)
Список pgsql-hackers
Hello. I revised the patch I think addressing all your comments.

Differences from v7 patch are:

v9-0001:

 - Renamed the script from 016_ to 017_.

 - Added some additional tests.


v9-0002:
 - Fixed _bt_blwritepage().
   It is re-modified by v9-0007.


v9-0003: New patch.
 - Refactors out xlog sutff from heap_insert/delete.
   (log_heap_insert(), log_heap_udpate())


v9-0004: (v7-0003, v8-0004)
 - Renamed some struct names and member names.
   (PendingRelSync -> RelWalRequirement
     .sync_above -> skip_wal_min_blk, .truncated_to -> wal_log_min_blk)

 - Rename the addtional members in RelationData to rd_*.

 - Explicitly initialize the additional members only in
   load_relcache_init_file().

 - Added new interface functions that accept block number and
   SMgrRelation.
   (BlockNeedsWAL(), RecordPendingSync())

 - Support subtransaction, (or invalidation).
   (RelWalRequirement.create_sxid, invalidate_sxid,
    RelationInvalidateWALRequirements(), smgrDoPendingSyncs())

 - Support forks.
   (RelWalRequirement.forks, smgrDoPendingSyncs(), RecordPendingSync())

 - Removd elog(LOG)s and a leftover comment.

v9-0005: (v7-0004, v8-0005)

 - Fixed heap_update().
   (heap_update())

v9-0006: New patch.

 - Modifies CLUSTER to skip WAL logging.

v9-0007: New patch.

 - Modifies ALTER TABLE SET TABLESPACE to skip WAL logging.

v9-0008: New patch.

 - Modifies btbuild to skip WAL logging.

 - Modifies btinsertonpg to skip WAL logging after truncation.

 - Overrites on v9-0002's change.


ALL:

 - Rebased.

 - Fixed typos and mistakes in comments.
   

> At Wed, 20 Mar 2019 17:17:54 +0900 (Tokyo Standard Time), Kyotaro HORIGUCHI <horiguchi.kyotaro@lab.ntt.co.jp> wrote
in<20190320.171754.171896368.horiguchi.kyotaro@lab.ntt.co.jp>
 
> > > We still use heap_sync() in CLUSTER.  Can we migrate CLUSTER to the newer
> > > heap_register_sync()?  Patch v7 makes some commands use the new way (COPY,
> > > CREATE TABLE AS, REFRESH MATERIALIZED VIEW, ALTER TABLE) and leaves other
> > > commands using the old way (CREATE INDEX USING btree, ALTER TABLE SET
> > > TABLESPACE, CLUSTER).  It would make the system simpler to understand if we
> > > eliminated the old way.  If that creates more problems than it solves, please
> > > at least write down a coding rule to explain why certain commands shouldn't
> > > use the old way.
> > 
> > Perhaps doable for TABLESPACE and CLUSTER. I'm not sure about
> > CREATE INDEX. I'll consider them.
> 
> I added the CLUSTER case in the new patchset.  For the SET
> TABLESPACE case, it works on SMGR layer and manipulates fork
> files explicitly but this stuff is Relation based and doesn't
> distinguish forks. We can modify this stuff to work on smgr and
> make it fork-aware but I don't think it is worth doing.
> 
> CREATE INDEX is not changed in this version. I continue to
> consider it.

I managed to simplify the change. Please look at v9-0008.

regards.

-- 
Kyotaro Horiguchi
NTT Open Source Software Center
From 13fe16c4527273426d93429986700ac66810945d 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/8] TAP test for copy-truncation optimization.

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

diff --git a/src/test/recovery/t/017_wal_optimize.pl b/src/test/recovery/t/017_wal_optimize.pl
new file mode 100644
index 0000000000..5d67548b54
--- /dev/null
+++ b/src/test/recovery/t/017_wal_optimize.pl
@@ -0,0 +1,254 @@
+# 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 => 20;
+
+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
+));
+    $node->start;
+
+    # Setup
+    my $tablespace_dir = $node->basedir . '/tablespace_other';
+    mkdir ($tablespace_dir);
+    $tablespace_dir = TestLib::real_dir($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");
+
+    # 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,10000));
+        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,10000));
+        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");
+
+    # 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 01691f5cf36e3bc75952b630088788c0da36b594 Mon Sep 17 00:00:00 2001
From: Kyotaro Horiguchi <horiguchi.kyotaro@lab.ntt.co.jp>
Date: Thu, 11 Oct 2018 10:03:08 +0900
Subject: [PATCH 2/8] Write WAL for empty nbtree index build

After relation truncation indexes are also rebuild. It doesn't emit
WAL in minimal mode and if truncation happened within its creation
transaction, crash recovery leaves an empty index heap, which is
considered broken. This patch forces to emit WAL when an index_build
turns into empty nbtree index.
---
 src/backend/access/nbtree/nbtsort.c | 11 +++++++++--
 1 file changed, 9 insertions(+), 2 deletions(-)

diff --git a/src/backend/access/nbtree/nbtsort.c b/src/backend/access/nbtree/nbtsort.c
index 46e0831834..e65d4aab0f 100644
--- a/src/backend/access/nbtree/nbtsort.c
+++ b/src/backend/access/nbtree/nbtsort.c
@@ -622,8 +622,15 @@ _bt_blwritepage(BTWriteState *wstate, Page page, BlockNumber blkno)
     /* Ensure rd_smgr is open (could have been closed by relcache flush!) */
     RelationOpenSmgr(wstate->index);
 
-    /* XLOG stuff */
-    if (wstate->btws_use_wal)
+    /* XLOG stuff
+     *
+     * Even if minimal mode, WAL is required here if truncation happened after
+     * being created in the same transaction. It is not needed otherwise but
+     * we don't bother identifying the case precisely.
+     */
+    if (wstate->btws_use_wal ||
+        (RelationNeedsWAL(wstate->index) &&
+         (blkno == BTREE_METAPAGE && BTPageGetMeta(page)->btm_root == 0)))
     {
         /* We use the heap NEWPAGE record type for this */
         log_newpage(&wstate->index->rd_node, MAIN_FORKNUM, blkno, page, true);
-- 
2.16.3

From 09ecd87dee4187d1266799c8cc68e2ea9f700c9b Mon Sep 17 00:00:00 2001
From: Kyotaro Horiguchi <horiguchi.kyotaro@lab.ntt.co.jp>
Date: Mon, 25 Mar 2019 13:29:50 +0900
Subject: [PATCH 3/8] Move XLOG stuff from heap_insert and heap_delete

Succeeding commit makes heap_update emit insert and delete WAL
records. Move out XLOG stuff for insert and delete so that heap_update
can use the stuff.
---
 src/backend/access/heap/heapam.c | 277 ++++++++++++++++++++++-----------------
 1 file changed, 157 insertions(+), 120 deletions(-)

diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 137cc9257d..c6e71dba6b 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -71,6 +71,11 @@
 
 static HeapTuple heap_prepare_insert(Relation relation, HeapTuple tup,
                     TransactionId xid, CommandId cid, int options);
+static XLogRecPtr log_heap_insert(Relation relation, Buffer buffer,
+                HeapTuple heaptup, int options, bool all_visible_cleared);
+static XLogRecPtr log_heap_delete(Relation relation, Buffer buffer,
+                HeapTuple tp, HeapTuple old_key_tuple, TransactionId new_xmax,
+                bool changingPart, bool all_visible_cleared);
 static XLogRecPtr log_heap_update(Relation reln, Buffer oldbuf,
                 Buffer newbuf, HeapTuple oldtup,
                 HeapTuple newtup, HeapTuple old_key_tup,
@@ -1860,6 +1865,7 @@ heap_insert(Relation relation, HeapTuple tup, CommandId cid,
     TransactionId xid = GetCurrentTransactionId();
     HeapTuple    heaptup;
     Buffer        buffer;
+    Page        page;
     Buffer        vmbuffer = InvalidBuffer;
     bool        all_visible_cleared = false;
 
@@ -1896,16 +1902,18 @@ heap_insert(Relation relation, HeapTuple tup, CommandId cid,
      */
     CheckForSerializableConflictIn(relation, NULL, InvalidBuffer);
 
+    page = BufferGetPage(buffer);
+
     /* NO EREPORT(ERROR) from here till changes are logged */
     START_CRIT_SECTION();
 
     RelationPutHeapTuple(relation, buffer, heaptup,
                          (options & HEAP_INSERT_SPECULATIVE) != 0);
 
-    if (PageIsAllVisible(BufferGetPage(buffer)))
+    if (PageIsAllVisible(page))
     {
         all_visible_cleared = true;
-        PageClearAllVisible(BufferGetPage(buffer));
+        PageClearAllVisible(page);
         visibilitymap_clear(relation,
                             ItemPointerGetBlockNumber(&(heaptup->t_self)),
                             vmbuffer, VISIBILITYMAP_VALID_BITS);
@@ -1927,76 +1935,11 @@ heap_insert(Relation relation, HeapTuple tup, CommandId cid,
     /* XLOG stuff */
     if (!(options & HEAP_INSERT_SKIP_WAL) && RelationNeedsWAL(relation))
     {
-        xl_heap_insert xlrec;
-        xl_heap_header xlhdr;
         XLogRecPtr    recptr;
-        Page        page = BufferGetPage(buffer);
-        uint8        info = XLOG_HEAP_INSERT;
-        int            bufflags = 0;
-
-        /*
-         * If this is a catalog, we need to transmit combocids to properly
-         * decode, so log that as well.
-         */
-        if (RelationIsAccessibleInLogicalDecoding(relation))
-            log_heap_new_cid(relation, heaptup);
-
-        /*
-         * If this is the single and first tuple on page, we can reinit the
-         * page instead of restoring the whole thing.  Set flag, and hide
-         * buffer references from XLogInsert.
-         */
-        if (ItemPointerGetOffsetNumber(&(heaptup->t_self)) == FirstOffsetNumber &&
-            PageGetMaxOffsetNumber(page) == FirstOffsetNumber)
-        {
-            info |= XLOG_HEAP_INIT_PAGE;
-            bufflags |= REGBUF_WILL_INIT;
-        }
-
-        xlrec.offnum = ItemPointerGetOffsetNumber(&heaptup->t_self);
-        xlrec.flags = 0;
-        if (all_visible_cleared)
-            xlrec.flags |= XLH_INSERT_ALL_VISIBLE_CLEARED;
-        if (options & HEAP_INSERT_SPECULATIVE)
-            xlrec.flags |= XLH_INSERT_IS_SPECULATIVE;
-        Assert(ItemPointerGetBlockNumber(&heaptup->t_self) == BufferGetBlockNumber(buffer));
-
-        /*
-         * For logical decoding, we need the tuple even if we're doing a full
-         * page write, so make sure it's included even if we take a full-page
-         * image. (XXX We could alternatively store a pointer into the FPW).
-         */
-        if (RelationIsLogicallyLogged(relation) &&
-            !(options & HEAP_INSERT_NO_LOGICAL))
-        {
-            xlrec.flags |= XLH_INSERT_CONTAINS_NEW_TUPLE;
-            bufflags |= REGBUF_KEEP_DATA;
-        }
-
-        XLogBeginInsert();
-        XLogRegisterData((char *) &xlrec, SizeOfHeapInsert);
-
-        xlhdr.t_infomask2 = heaptup->t_data->t_infomask2;
-        xlhdr.t_infomask = heaptup->t_data->t_infomask;
-        xlhdr.t_hoff = heaptup->t_data->t_hoff;
-
-        /*
-         * note we mark xlhdr as belonging to buffer; if XLogInsert decides to
-         * write the whole page to the xlog, we don't need to store
-         * xl_heap_header in the xlog.
-         */
-        XLogRegisterBuffer(0, buffer, REGBUF_STANDARD | bufflags);
-        XLogRegisterBufData(0, (char *) &xlhdr, SizeOfHeapHeader);
-        /* PG73FORMAT: write bitmap [+ padding] [+ oid] + data */
-        XLogRegisterBufData(0,
-                            (char *) heaptup->t_data + SizeofHeapTupleHeader,
-                            heaptup->t_len - SizeofHeapTupleHeader);
-
-        /* filtering by origin on a row level is much more efficient */
-        XLogSetRecordFlags(XLOG_INCLUDE_ORIGIN);
-
-        recptr = XLogInsert(RM_HEAP_ID, info);
 
+        recptr = log_heap_insert(relation, buffer, heaptup,
+                                 options, all_visible_cleared);
+            
         PageSetLSN(page, recptr);
     }
 
@@ -2715,58 +2658,10 @@ l1:
      */
     if (RelationNeedsWAL(relation))
     {
-        xl_heap_delete xlrec;
-        xl_heap_header xlhdr;
         XLogRecPtr    recptr;
 
-        /* For logical decode we need combocids to properly decode the catalog */
-        if (RelationIsAccessibleInLogicalDecoding(relation))
-            log_heap_new_cid(relation, &tp);
-
-        xlrec.flags = 0;
-        if (all_visible_cleared)
-            xlrec.flags |= XLH_DELETE_ALL_VISIBLE_CLEARED;
-        if (changingPart)
-            xlrec.flags |= XLH_DELETE_IS_PARTITION_MOVE;
-        xlrec.infobits_set = compute_infobits(tp.t_data->t_infomask,
-                                              tp.t_data->t_infomask2);
-        xlrec.offnum = ItemPointerGetOffsetNumber(&tp.t_self);
-        xlrec.xmax = new_xmax;
-
-        if (old_key_tuple != NULL)
-        {
-            if (relation->rd_rel->relreplident == REPLICA_IDENTITY_FULL)
-                xlrec.flags |= XLH_DELETE_CONTAINS_OLD_TUPLE;
-            else
-                xlrec.flags |= XLH_DELETE_CONTAINS_OLD_KEY;
-        }
-
-        XLogBeginInsert();
-        XLogRegisterData((char *) &xlrec, SizeOfHeapDelete);
-
-        XLogRegisterBuffer(0, buffer, REGBUF_STANDARD);
-
-        /*
-         * Log replica identity of the deleted tuple if there is one
-         */
-        if (old_key_tuple != NULL)
-        {
-            xlhdr.t_infomask2 = old_key_tuple->t_data->t_infomask2;
-            xlhdr.t_infomask = old_key_tuple->t_data->t_infomask;
-            xlhdr.t_hoff = old_key_tuple->t_data->t_hoff;
-
-            XLogRegisterData((char *) &xlhdr, SizeOfHeapHeader);
-            XLogRegisterData((char *) old_key_tuple->t_data
-                             + SizeofHeapTupleHeader,
-                             old_key_tuple->t_len
-                             - SizeofHeapTupleHeader);
-        }
-
-        /* filtering by origin on a row level is much more efficient */
-        XLogSetRecordFlags(XLOG_INCLUDE_ORIGIN);
-
-        recptr = XLogInsert(RM_HEAP_ID, XLOG_HEAP_DELETE);
-
+        recptr = log_heap_delete(relation, buffer, &tp, old_key_tuple, new_xmax,
+                                 changingPart, all_visible_cleared);
         PageSetLSN(page, recptr);
     }
 
@@ -7016,6 +6911,148 @@ log_heap_visible(RelFileNode rnode, Buffer heap_buffer, Buffer vm_buffer,
     return recptr;
 }
 
+/*
+ * Perform XLogInsert for a heap-insert operation.  Caller must already
+ * have modified the buffer and marked it dirty.
+ */
+XLogRecPtr
+log_heap_insert(Relation relation, Buffer buffer,
+                HeapTuple heaptup, int options, bool all_visible_cleared)
+{
+    xl_heap_insert xlrec;
+    xl_heap_header xlhdr;
+    uint8        info = XLOG_HEAP_INSERT;
+    int            bufflags = 0;
+    Page        page = BufferGetPage(buffer);
+
+    /*
+     * If this is a catalog, we need to transmit combocids to properly
+     * decode, so log that as well.
+     */
+    if (RelationIsAccessibleInLogicalDecoding(relation))
+        log_heap_new_cid(relation, heaptup);
+
+    /*
+     * If this is the single and first tuple on page, we can reinit the
+     * page instead of restoring the whole thing.  Set flag, and hide
+     * buffer references from XLogInsert.
+     */
+    if (ItemPointerGetOffsetNumber(&(heaptup->t_self)) == FirstOffsetNumber &&
+        PageGetMaxOffsetNumber(page) == FirstOffsetNumber)
+    {
+        info |= XLOG_HEAP_INIT_PAGE;
+        bufflags |= REGBUF_WILL_INIT;
+    }
+
+    xlrec.offnum = ItemPointerGetOffsetNumber(&heaptup->t_self);
+    xlrec.flags = 0;
+    if (all_visible_cleared)
+        xlrec.flags |= XLH_INSERT_ALL_VISIBLE_CLEARED;
+    if (options & HEAP_INSERT_SPECULATIVE)
+        xlrec.flags |= XLH_INSERT_IS_SPECULATIVE;
+    Assert(ItemPointerGetBlockNumber(&heaptup->t_self) == BufferGetBlockNumber(buffer));
+
+    /*
+     * For logical decoding, we need the tuple even if we're doing a full
+     * page write, so make sure it's included even if we take a full-page
+     * image. (XXX We could alternatively store a pointer into the FPW).
+     */
+    if (RelationIsLogicallyLogged(relation) &&
+        !(options & HEAP_INSERT_NO_LOGICAL))
+    {
+        xlrec.flags |= XLH_INSERT_CONTAINS_NEW_TUPLE;
+        bufflags |= REGBUF_KEEP_DATA;
+    }
+
+    XLogBeginInsert();
+    XLogRegisterData((char *) &xlrec, SizeOfHeapInsert);
+
+    xlhdr.t_infomask2 = heaptup->t_data->t_infomask2;
+    xlhdr.t_infomask = heaptup->t_data->t_infomask;
+    xlhdr.t_hoff = heaptup->t_data->t_hoff;
+
+    /*
+     * note we mark xlhdr as belonging to buffer; if XLogInsert decides to
+     * write the whole page to the xlog, we don't need to store
+     * xl_heap_header in the xlog.
+     */
+    XLogRegisterBuffer(0, buffer, REGBUF_STANDARD | bufflags);
+    XLogRegisterBufData(0, (char *) &xlhdr, SizeOfHeapHeader);
+    /* PG73FORMAT: write bitmap [+ padding] [+ oid] + data */
+    XLogRegisterBufData(0,
+                        (char *) heaptup->t_data + SizeofHeapTupleHeader,
+                        heaptup->t_len - SizeofHeapTupleHeader);
+
+    /* filtering by origin on a row level is much more efficient */
+    XLogSetRecordFlags(XLOG_INCLUDE_ORIGIN);
+
+    return XLogInsert(RM_HEAP_ID, info);
+}
+
+/*
+ * Perform XLogInsert for a heap-insert operation.  Caller must already
+ * have modified the buffer and marked it dirty.
+ *
+ * NB: heap_abort_speculative() uses the same xlog record and replay
+ * routines.
+ */
+static XLogRecPtr
+log_heap_delete(Relation relation, Buffer buffer,
+                HeapTuple tp, HeapTuple old_key_tuple, TransactionId new_xmax,
+                bool changingPart, bool all_visible_cleared)
+{
+    xl_heap_delete xlrec;
+    xl_heap_header xlhdr;
+
+    /* For logical decode we need combocids to properly decode the catalog */
+    if (RelationIsAccessibleInLogicalDecoding(relation))
+        log_heap_new_cid(relation, tp);
+
+    xlrec.flags = 0;
+    if (all_visible_cleared)
+        xlrec.flags |= XLH_DELETE_ALL_VISIBLE_CLEARED;
+    if (changingPart)
+        xlrec.flags |= XLH_DELETE_IS_PARTITION_MOVE;
+    xlrec.infobits_set = compute_infobits(tp->t_data->t_infomask,
+                                          tp->t_data->t_infomask2);
+    xlrec.offnum = ItemPointerGetOffsetNumber(&tp->t_self);
+    xlrec.xmax = new_xmax;
+
+    if (old_key_tuple != NULL)
+    {
+        if (relation->rd_rel->relreplident == REPLICA_IDENTITY_FULL)
+            xlrec.flags |= XLH_DELETE_CONTAINS_OLD_TUPLE;
+        else
+            xlrec.flags |= XLH_DELETE_CONTAINS_OLD_KEY;
+    }
+
+    XLogBeginInsert();
+    XLogRegisterData((char *) &xlrec, SizeOfHeapDelete);
+
+    XLogRegisterBuffer(0, buffer, REGBUF_STANDARD);
+
+    /*
+     * Log replica identity of the deleted tuple if there is one
+     */
+    if (old_key_tuple != NULL)
+    {
+        xlhdr.t_infomask2 = old_key_tuple->t_data->t_infomask2;
+        xlhdr.t_infomask = old_key_tuple->t_data->t_infomask;
+        xlhdr.t_hoff = old_key_tuple->t_data->t_hoff;
+
+        XLogRegisterData((char *) &xlhdr, SizeOfHeapHeader);
+        XLogRegisterData((char *) old_key_tuple->t_data
+                         + SizeofHeapTupleHeader,
+                         old_key_tuple->t_len
+                         - SizeofHeapTupleHeader);
+    }
+
+    /* filtering by origin on a row level is much more efficient */
+    XLogSetRecordFlags(XLOG_INCLUDE_ORIGIN);
+
+    return XLogInsert(RM_HEAP_ID, XLOG_HEAP_DELETE);
+}
+
 /*
  * Perform XLogInsert for a heap-update operation.  Caller must already
  * have modified the buffer(s) and marked them dirty.
-- 
2.16.3

From ed3a737a571b268503804372ebac3a31247493be Mon Sep 17 00:00:00 2001
From: Kyotaro Horiguchi <horiguchi.kyotaro@lab.ntt.co.jp>
Date: Tue, 26 Mar 2019 15:34:48 +0900
Subject: [PATCH 4/8] Add infrastructure to WAL-logging skip feature

We used to optimize WAL-logging for truncation of in-transaction
crated tables in minimal mode by just signaling by
HEAP_INSERT_SKIP_WAL option on heap operations. This mechanism can
emit WAL records that results in corrupt state for certain series of
in-transaction operations. This patch provides infrastructure to track
pending at-commit fsyncs for a relation and in-transaction
truncations. heap_register_sync() should be used to start tracking
before batch operations like COPY and CLUSTER, and use
BufferNeedsWAL() instead of RelationNeedsWAL() at the places related
to WAL-logging about heap-modifying operations.
---
 src/backend/access/heap/heapam.c    |  31 +++
 src/backend/access/transam/xact.c   |  11 +
 src/backend/catalog/storage.c       | 418 ++++++++++++++++++++++++++++++++++--
 src/backend/commands/tablecmds.c    |   4 +-
 src/backend/storage/buffer/bufmgr.c |  39 +++-
 src/backend/utils/cache/relcache.c  |   3 +
 src/include/access/heapam.h         |   1 +
 src/include/catalog/storage.h       |   8 +
 src/include/storage/bufmgr.h        |   2 +
 src/include/utils/rel.h             |   8 +
 10 files changed, 493 insertions(+), 32 deletions(-)

diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index c6e71dba6b..5a8627507f 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -51,6 +51,7 @@
 #include "access/xloginsert.h"
 #include "access/xlogutils.h"
 #include "catalog/catalog.h"
+#include "catalog/storage.h"
 #include "miscadmin.h"
 #include "pgstat.h"
 #include "port/atomics.h"
@@ -8800,3 +8801,33 @@ heap_mask(char *pagedata, BlockNumber blkno)
         }
     }
 }
+
+/*
+ *    heap_register_sync    - register a heap to be synced to disk at commit
+ *
+ * This can be used to skip WAL-logging changes on a relation file that has
+ * been created in the same transaction. This makes note of the current size of
+ * the relation, and ensures that when the relation is extended, any changes
+ * to the new blocks in the heap, in the same transaction, will not be
+ * WAL-logged. Instead, the heap contents are flushed to disk at commit,
+ * like heap_sync() does.
+ *
+ * This does the same for the TOAST heap, if any. Indexes are not affected.
+ */
+void
+heap_register_sync(Relation rel)
+{
+    /* non-WAL-logged tables never need fsync */
+    if (!RelationNeedsWAL(rel))
+        return;
+
+    RecordWALSkipping(rel);
+    if (OidIsValid(rel->rd_rel->reltoastrelid))
+    {
+        Relation    toastrel;
+
+        toastrel = heap_open(rel->rd_rel->reltoastrelid, AccessShareLock);
+        RecordWALSkipping(toastrel);
+        heap_close(toastrel, AccessShareLock);
+    }
+}
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index c3214d4f4d..ad7cb3bcb9 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -2022,6 +2022,9 @@ CommitTransaction(void)
     /* close large objects before lower-level cleanup */
     AtEOXact_LargeObject(true);
 
+    /* Flush updates to relations that we didn't WAL-logged */
+    smgrDoPendingSyncs(true);
+
     /*
      * Mark serializable transaction as complete for predicate locking
      * purposes.  This should be done as late as we can put it and still allow
@@ -2254,6 +2257,9 @@ PrepareTransaction(void)
     /* close large objects before lower-level cleanup */
     AtEOXact_LargeObject(true);
 
+    /* Flush updates to relations that we didn't WAL-logged */
+    smgrDoPendingSyncs(true);
+
     /*
      * Mark serializable transaction as complete for predicate locking
      * purposes.  This should be done as late as we can put it and still allow
@@ -2579,6 +2585,7 @@ AbortTransaction(void)
     AtAbort_Notify();
     AtEOXact_RelationMap(false, is_parallel_worker);
     AtAbort_Twophase();
+    smgrDoPendingSyncs(false);    /* abandon pending syncs */
 
     /*
      * Advertise the fact that we aborted in pg_xact (assuming that we got as
@@ -4097,6 +4104,8 @@ ReleaseSavepoint(const char *name)
                 (errcode(ERRCODE_S_E_INVALID_SPECIFICATION),
                  errmsg("savepoint \"%s\" does not exist within current savepoint level", name)));
 
+    smgrProcessWALRequirementInval(s->subTransactionId, true);
+
     /*
      * Mark "commit pending" all subtransactions up to the target
      * subtransaction.  The actual commits will happen when control gets to
@@ -4206,6 +4215,8 @@ RollbackToSavepoint(const char *name)
                 (errcode(ERRCODE_S_E_INVALID_SPECIFICATION),
                  errmsg("savepoint \"%s\" does not exist within current savepoint level", name)));
 
+    smgrProcessWALRequirementInval(s->subTransactionId, false);
+
     /*
      * Mark "abort pending" all subtransactions up to the target
      * subtransaction.  The actual aborts will happen when control gets to
diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index 0302507e6f..be37174ef2 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -27,7 +27,7 @@
 #include "catalog/storage.h"
 #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"
 
@@ -62,6 +62,58 @@ typedef struct PendingRelDelete
 
 static PendingRelDelete *pendingDeletes = NULL; /* head of linked list */
 
+/*
+ * We also track relation files (RelFileNode values) that have been created
+ * in the same transaction, and that have been modified without WAL-logging
+ * the action (an optimization possible with wal_level=minimal). When we are
+ * about to skip WAL-logging, a RelWalRequirement entry is created, and
+ * 'skip_wal_min_blk' is set to the current size of the relation. Any operations
+ * on blocks < skip_wal_min_blk need to be WAL-logged as usual, but for
+ * operations on higher blocks, WAL-logging is skipped.
+
+ *
+ * NB: after WAL-logging has been skipped for a block, we must not WAL-log
+ * any subsequent actions on the same block either. Replaying the WAL record
+ * of the subsequent action might fail otherwise, as the "before" state of
+ * the block might not match, as the earlier actions were not WAL-logged.
+ * Likewise, after we have WAL-logged an operation for a block, we must
+ * WAL-log any subsequent operations on the same page as well. Replaying
+ * a possible full-page-image from the earlier WAL record would otherwise
+ * revert the page to the old state, even if we sync the relation at end
+ * of transaction.
+ *
+ * If a relation is truncated (without creating a new relfilenode), and we
+ * emit a WAL record of the truncation, we can't skip WAL-logging for any
+ * of the truncated blocks anymore, as replaying the truncation record will
+ * destroy all the data inserted after that. But if we have already decided
+ * to skip WAL-logging changes to a relation, and the relation is truncated,
+ * we don't need to WAL-log the truncation either.
+ *
+ * This mechanism is currently only used by heaps. Indexes are always
+ * WAL-logged. Also, this only applies for wal_level=minimal; with higher
+ * WAL levels we need the WAL for PITR/replication anyway.
+ */
+typedef struct RelWalRequirement
+{
+    RelFileNode relnode;            /* relation created in same xact */
+    bool        forks[MAX_FORKNUM + 1];    /* target forknums */
+    BlockNumber skip_wal_min_blk;    /* WAL-logging skipped for blocks >=
+                                     * skip_wal_min_blk */
+    BlockNumber wal_log_min_blk;     /* The minimum blk number that requires
+                                     * WAL-logging even if skipped by the
+                                     * above*/
+    SubTransactionId create_sxid;    /* subxid where this entry is created */
+    SubTransactionId invalidate_sxid; /* subxid where this entry is
+                                       * invalidated */
+}    RelWalRequirement;
+
+/* Relations that need to be fsync'd at commit */
+static HTAB *walRequirements = NULL;
+
+static RelWalRequirement *getWalRequirementEntry(Relation rel, bool create);
+static RelWalRequirement *getWalRequirementEntryRNode(RelFileNode *node,
+                                                      bool create);
+
 /*
  * RelationCreateStorage
  *        Create physical storage for a relation.
@@ -259,37 +311,290 @@ RelationTruncate(Relation rel, BlockNumber nblocks)
      */
     if (RelationNeedsWAL(rel))
     {
-        /*
-         * Make an XLOG entry reporting the file truncation.
-         */
-        XLogRecPtr    lsn;
-        xl_smgr_truncate xlrec;
+        RelWalRequirement *walreq;
 
-        xlrec.blkno = nblocks;
-        xlrec.rnode = rel->rd_node;
-        xlrec.flags = SMGR_TRUNCATE_ALL;
+        /* get pending sync entry, create if not yet */
+        walreq = getWalRequirementEntry(rel, true);
 
-        XLogBeginInsert();
-        XLogRegisterData((char *) &xlrec, sizeof(xlrec));
+        if (walreq->skip_wal_min_blk == InvalidBlockNumber ||
+            walreq->skip_wal_min_blk < nblocks)
+        {
+            /*
+             * This is the first time truncation of this relation in this
+             * transaction or truncation that leaves pages that need at-commit
+             * fsync.  Make an XLOG entry reporting the file truncation.
+             */
+            XLogRecPtr        lsn;
+            xl_smgr_truncate xlrec;
 
-        lsn = XLogInsert(RM_SMGR_ID,
-                         XLOG_SMGR_TRUNCATE | XLR_SPECIAL_REL_UPDATE);
+            xlrec.blkno = nblocks;
+            xlrec.rnode = rel->rd_node;
+            xlrec.flags = SMGR_TRUNCATE_ALL;
 
-        /*
-         * Flush, because otherwise the truncation of the main relation might
-         * hit the disk before the WAL record, and the truncation of the FSM
-         * or visibility map. If we crashed during that window, we'd be left
-         * with a truncated heap, but the FSM or visibility map would still
-         * contain entries for the non-existent heap pages.
-         */
-        if (fsm || vm)
-            XLogFlush(lsn);
+            XLogBeginInsert();
+            XLogRegisterData((char *) &xlrec, sizeof(xlrec));
+
+            lsn = XLogInsert(RM_SMGR_ID,
+                             XLOG_SMGR_TRUNCATE | XLR_SPECIAL_REL_UPDATE);
+
+            /*
+             * Flush, because otherwise the truncation of the main relation
+             * might hit the disk before the WAL record, and the truncation of
+             * the FSM or visibility map. If we crashed during that window,
+             * we'd be left with a truncated heap, but the FSM or visibility
+             * map would still contain entries for the non-existent heap
+             * pages.
+             */
+            if (fsm || vm)
+                XLogFlush(lsn);
+
+            /* no longer skip WAL-logging for the blocks */
+            rel->rd_walrequirement->wal_log_min_blk = nblocks;
+        }
     }
 
     /* Do the real work */
     smgrtruncate(rel->rd_smgr, MAIN_FORKNUM, nblocks);
 }
 
+/*
+ * Do changes to given heap page need to be WAL-logged?
+ *
+ * This takes into account any previous RecordPendingSync() requests.
+ *
+ * Note that it is required to check this before creating any WAL records for
+ * heap pages - it is not merely an optimization! WAL-logging a record, when
+ * we have already skipped a previous WAL record for the same page could lead
+ * to failure at WAL replay, as the "before" state expected by the record
+ * might not match what's on disk. Also, if the heap was truncated earlier, we
+ * must WAL-log any changes to the once-truncated blocks, because replaying
+ * the truncation record will destroy them.
+ */
+bool
+BufferNeedsWAL(Relation rel, Buffer buf)
+{
+    BlockNumber        blkno = InvalidBlockNumber;
+    RelWalRequirement *walreq;
+
+    if (!RelationNeedsWAL(rel))
+        return false;
+
+    /* fetch existing pending sync entry */
+    walreq = getWalRequirementEntry(rel, false);
+
+    /*
+     * no point in doing further work if we know that we don't have special
+     * WAL requirement
+     */
+    if (!walreq)
+        return true;
+
+    Assert(BufferIsValid(buf));
+
+    blkno = BufferGetBlockNumber(buf);
+
+    /*
+     * We don't skip WAL-logging for pages that once done.
+     */
+    if (walreq->skip_wal_min_blk == InvalidBlockNumber ||
+        walreq->skip_wal_min_blk > blkno)
+        return true;
+
+    /*
+     * we don't skip WAL-logging for blocks that have got WAL-logged
+     * truncation
+     */
+    if (walreq->wal_log_min_blk != InvalidBlockNumber &&
+        walreq->wal_log_min_blk <= blkno)
+        return true;
+
+    return false;
+}
+
+bool
+BlockNeedsWAL(Relation rel, BlockNumber blkno)
+{
+    RelWalRequirement *walreq;
+
+    if (!RelationNeedsWAL(rel))
+        return false;
+
+    /* fetch exising pending sync entry */
+    walreq = getWalRequirementEntry(rel, false);
+
+    /*
+     * no point in doing further work if we know that we don't have special
+     * WAL requirement
+     */
+    if (!walreq)
+        return true;
+
+    /*
+     * We don't skip WAL-logging for pages that once done.
+     */
+    if (walreq->skip_wal_min_blk == InvalidBlockNumber ||
+        walreq->skip_wal_min_blk > blkno)
+        return true;
+
+    /*
+     * we don't skip WAL-logging for blocks that have got WAL-logged
+     * truncation
+     */
+    if (walreq->wal_log_min_blk != InvalidBlockNumber &&
+        walreq->wal_log_min_blk <= blkno)
+        return true;
+
+    return false;
+}
+
+/*
+ * Remember that the given relation doesn't need WAL-logging for the blocks
+ * after the current block size and fo the blocks that are going to be synced
+ * at commit.
+ */
+void
+RecordWALSkipping(Relation rel)
+{
+    RelWalRequirement *walreq;
+
+    Assert(RelationNeedsWAL(rel));
+
+    /* get pending sync entry, create if not yet  */
+    walreq = getWalRequirementEntry(rel, true);
+
+    /*
+     *  Record only the first registration.
+     */
+    if (walreq->skip_wal_min_blk != InvalidBlockNumber)
+        return;
+
+    walreq->skip_wal_min_blk = RelationGetNumberOfBlocks(rel);
+}
+
+/*
+ * Record commit-time file sync. This shouldn't be used mixing with
+ * RecordWALSkipping.
+ */
+void
+RecordPendingSync(SMgrRelation rel, ForkNumber forknum)
+{
+    RelWalRequirement *walreq;
+
+    walreq = getWalRequirementEntryRNode(&rel->smgr_rnode.node, true);
+    walreq->forks[forknum] = true;
+    walreq->skip_wal_min_blk = 0;
+}
+
+/*
+ * RelationInvalidateWALRequirements() -- invalidate wal requirement entry
+ */
+void
+RelationInvalidateWALRequirements(Relation rel)
+{
+    RelWalRequirement *walreq;
+
+    /* we know we don't have one */
+    if (rel->rd_nowalrequirement)
+        return;
+
+    walreq = getWalRequirementEntry(rel, false);
+    
+    if (!walreq)
+        return;
+
+    /*
+     * The state is reset at subtransaction commit/abort. No invalidation
+     * request must not come for the same relation in the same subtransaction.
+     */
+    Assert(walreq->invalidate_sxid == InvalidSubTransactionId);
+
+    walreq->invalidate_sxid = GetCurrentSubTransactionId();
+}
+
+/*
+ * getWalRequirementEntry: get WAL requirement entry.
+ *
+ * Returns WAL requirement entry for the relation. The entry tracks
+ * WAL-skipping blocks for the relation.  The WAL-skipped blocks need fsync at
+ * commit time.  Creates one if needed when create is true.
+ */
+static RelWalRequirement *
+getWalRequirementEntry(Relation rel, bool create)
+{
+    RelWalRequirement *walreq_entry = NULL;
+
+    if (rel->rd_walrequirement)
+        return rel->rd_walrequirement;
+
+    /* we know we don't have pending sync entry */
+    if (!create && rel->rd_nowalrequirement)
+        return NULL;
+
+    walreq_entry = getWalRequirementEntryRNode(&rel->rd_node, create);
+
+    if (!walreq_entry)
+    {
+        /* prevent further hash lookup */
+        rel->rd_nowalrequirement = true;
+        return NULL;
+    }
+
+    walreq_entry->forks[MAIN_FORKNUM] = true;
+
+    /* hold shortcut in Relation */
+    rel->rd_nowalrequirement = false;
+    rel->rd_walrequirement = walreq_entry;
+
+    return walreq_entry;
+}
+
+/*
+ * getWalRequirementEntryRNode: get WAL requirement entry by rnode
+ *
+ * Returns WAL requirement entry for the RelFileNode.
+ */
+static RelWalRequirement *
+getWalRequirementEntryRNode(RelFileNode *rnode, bool create)
+{
+    RelWalRequirement *walreq_entry = NULL;
+    bool            found;
+
+    if (!walRequirements)
+    {
+        /* First time through: initialize the hash table */
+        HASHCTL        ctl;
+
+        if (!create)
+            return NULL;
+
+        MemSet(&ctl, 0, sizeof(ctl));
+        ctl.keysize = sizeof(RelFileNode);
+        ctl.entrysize = sizeof(RelWalRequirement);
+        ctl.hash = tag_hash;
+        walRequirements = hash_create("pending relation sync table", 5,
+                                   &ctl, HASH_ELEM | HASH_FUNCTION);
+    }
+
+    walreq_entry = (RelWalRequirement *)
+        hash_search(walRequirements, (void *) rnode,
+                    create ? HASH_ENTER: HASH_FIND,    &found);
+
+    if (!walreq_entry)
+        return NULL;
+
+    /* new entry created */
+    if (!found)
+    {
+        memset(&walreq_entry->forks, 0, sizeof(sizeof(walreq_entry->forks)));
+        walreq_entry->wal_log_min_blk = InvalidBlockNumber;
+        walreq_entry->skip_wal_min_blk = InvalidBlockNumber;
+        walreq_entry->create_sxid = GetCurrentSubTransactionId();
+        walreq_entry->invalidate_sxid = InvalidSubTransactionId;
+    }
+
+    return walreq_entry;
+}
+
 /*
  *    smgrDoPendingDeletes() -- Take care of relation deletes at end of xact.
  *
@@ -418,6 +723,75 @@ smgrGetPendingDeletes(bool forCommit, RelFileNode **ptr)
     return nrels;
 }
 
+/*
+ * Sync to disk any relations that we have skipped WAL-logging earlier.
+ */
+void
+smgrDoPendingSyncs(bool isCommit)
+{
+    if (!walRequirements)
+        return;
+
+    if (isCommit)
+    {
+        HASH_SEQ_STATUS status;
+        RelWalRequirement *walreq;
+
+        hash_seq_init(&status, walRequirements);
+
+        while ((walreq = hash_seq_search(&status)) != NULL)
+        {
+            if (walreq->skip_wal_min_blk != InvalidBlockNumber &&
+                walreq->invalidate_sxid == InvalidSubTransactionId)
+            {
+                int f;
+
+                FlushRelationBuffersWithoutRelCache(walreq->relnode, false);
+
+                /* flush all requested forks  */
+                for (f = MAIN_FORKNUM ; f <= MAX_FORKNUM ; f++)
+                {
+                    if (walreq->forks[f])
+                        smgrimmedsync(smgropen(walreq->relnode,
+                                               InvalidBackendId), f);
+                }
+            }
+        }
+    }
+
+    hash_destroy(walRequirements);
+    walRequirements = NULL;
+}
+
+/*
+ * Process pending invalidation of WAL requirements happened in the
+ * subtransaction
+ */
+void
+smgrProcessWALRequirementInval(SubTransactionId sxid, bool isCommit)
+{
+    HASH_SEQ_STATUS status;
+    RelWalRequirement *walreq;
+
+    if (!walRequirements)
+        return;
+
+    /* We expect that we don't have walRequirements in almost all cases */
+    hash_seq_init(&status, walRequirements);
+
+    while ((walreq = hash_seq_search(&status)) != NULL)
+    {
+        /* remove useless entry */
+        if (isCommit ?
+            walreq->invalidate_sxid == sxid :
+            walreq->create_sxid == sxid)
+            hash_search(walRequirements, &walreq->relnode, HASH_REMOVE, NULL);
+        /* or cancel invalidation  */
+        else if (!isCommit && walreq->invalidate_sxid == sxid)
+            walreq->invalidate_sxid = InvalidSubTransactionId;
+    }
+}
+
 /*
  *    PostPrepare_smgr -- Clean up after a successful PREPARE
  *
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 3183b2aaa1..c9a0e02168 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -11587,11 +11587,13 @@ ATExecSetTableSpace(Oid tableOid, Oid newTableSpace, LOCKMODE lockmode)
 
     /*
      * Create and copy all forks of the relation, and schedule unlinking of
-     * old physical files.
+     * old physical files. WAL requirements for the old node is no longer
+     * needed.
      *
      * NOTE: any conflict in relfilenode value will be caught in
      * RelationCreateStorage().
      */
+    RelationInvalidateWALRequirements(rel);
     RelationCreateStorage(newrnode, rel->rd_rel->relpersistence);
 
     /* copy main fork */
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 273e2f385f..f00826712a 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -451,6 +451,7 @@ static BufferDesc *BufferAlloc(SMgrRelation smgr,
             BufferAccessStrategy strategy,
             bool *foundPtr);
 static void FlushBuffer(BufferDesc *buf, SMgrRelation reln);
+static void FlushRelationBuffers_common(SMgrRelation smgr, bool islocal);
 static void AtProcExit_Buffers(int code, Datum arg);
 static void CheckForBufferLeaks(void);
 static int    rnode_comparator(const void *p1, const void *p2);
@@ -3153,20 +3154,40 @@ 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));
+}
+
+/*
+ * Like FlushRelationBuffers(), but the relation is specified by RelFileNode
+ */
+void
+FlushRelationBuffersWithoutRelCache(RelFileNode rnode, bool islocal)
+{
+    FlushRelationBuffers_common(smgropen(rnode, InvalidBackendId), islocal);
+}
+
+/*
+ * Code shared between functions FlushRelationBuffers() and
+ * FlushRelationBuffersWithoutRelCache().
+ */
+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))
             {
@@ -3183,7 +3204,7 @@ FlushRelationBuffers(Relation rel)
 
                 PageSetChecksumInplace(localpage, bufHdr->tag.blockNum);
 
-                smgrwrite(rel->rd_smgr,
+                smgrwrite(smgr,
                           bufHdr->tag.forkNum,
                           bufHdr->tag.blockNum,
                           localpage,
@@ -3213,18 +3234,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/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index 84609e0725..95e834d45e 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -75,6 +75,7 @@
 #include "partitioning/partdesc.h"
 #include "rewrite/rewriteDefine.h"
 #include "rewrite/rowsecurity.h"
+#include "storage/bufmgr.h"
 #include "storage/lmgr.h"
 #include "storage/smgr.h"
 #include "utils/array.h"
@@ -5625,6 +5626,8 @@ load_relcache_init_file(bool shared)
         rel->rd_newRelfilenodeSubid = InvalidSubTransactionId;
         rel->rd_amcache = NULL;
         MemSet(&rel->pgstat_info, 0, sizeof(rel->pgstat_info));
+        rel->rd_nowalrequirement = false;
+        rel->rd_walrequirement = NULL;
 
         /*
          * Recompute lock and physical addressing info.  This is needed in
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index 3773a4df85..3d4fb7f3c3 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -172,6 +172,7 @@ extern void simple_heap_delete(Relation relation, ItemPointer tid);
 extern void simple_heap_update(Relation relation, ItemPointer otid,
                    HeapTuple tup);
 
+extern void heap_register_sync(Relation relation);
 extern void heap_sync(Relation relation);
 
 /* in heap/pruneheap.c */
diff --git a/src/include/catalog/storage.h b/src/include/catalog/storage.h
index 9f638be924..9034465001 100644
--- a/src/include/catalog/storage.h
+++ b/src/include/catalog/storage.h
@@ -16,12 +16,18 @@
 
 #include "storage/block.h"
 #include "storage/relfilenode.h"
+#include "storage/smgr.h"
 #include "utils/relcache.h"
 
 extern void RelationCreateStorage(RelFileNode rnode, char relpersistence);
 extern void RelationDropStorage(Relation rel);
 extern void RelationPreserveStorage(RelFileNode rnode, bool atCommit);
 extern void RelationTruncate(Relation rel, BlockNumber nblocks);
+extern bool BufferNeedsWAL(Relation rel, Buffer buf);
+extern bool BlockNeedsWAL(Relation rel, BlockNumber blkno);
+extern void RecordWALSkipping(Relation rel);
+extern void RecordPendingSync(SMgrRelation rel, ForkNumber forknum);
+extern void RelationInvalidateWALRequirements(Relation rel);
 
 /*
  * These functions used to be in storage/smgr/smgr.c, which explains the
@@ -29,6 +35,8 @@ extern void RelationTruncate(Relation rel, BlockNumber nblocks);
  */
 extern void smgrDoPendingDeletes(bool isCommit);
 extern int    smgrGetPendingDeletes(bool forCommit, RelFileNode **ptr);
+extern void smgrDoPendingSyncs(bool isCommit);
+extern void smgrProcessWALRequirementInval(SubTransactionId sxid, bool isCommit);
 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 c5826f691d..8a9ea041dd 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -189,6 +189,8 @@ 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/utils/rel.h b/src/include/utils/rel.h
index 54028515a7..30f0d5bd83 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -198,6 +198,14 @@ typedef struct RelationData
 
     /* use "struct" here to avoid needing to include pgstat.h: */
     struct PgStat_TableStatus *pgstat_info; /* statistics collection area */
+
+    /*
+     * rd_nowalrequirement is true if this relation is known not to have
+     * special WAL requirements.  Otherwise we need to ask smgr for an entry
+     * if rd_walrequirement is NULL.
+     */
+    bool                        rd_nowalrequirement;
+    struct RelWalRequirement   *rd_walrequirement;
 } RelationData;
 
 
-- 
2.16.3

From 24e392a8423c1bed350b58e5cda56a140d2730ce Mon Sep 17 00:00:00 2001
From: Kyotaro Horiguchi <horiguchi.kyotaro@lab.ntt.co.jp>
Date: Thu, 11 Oct 2018 16:18:04 +0900
Subject: [PATCH 5/8] Fix WAL skipping feature.

This patch replaces WAL-skipping means from HEAP_INSERT_SKIP_WAL to
pending-sync tracking infrastructure.
---
 src/backend/access/heap/heapam.c        | 109 ++++++++++++++++++++++++--------
 src/backend/access/heap/pruneheap.c     |   3 +-
 src/backend/access/heap/rewriteheap.c   |   3 -
 src/backend/access/heap/vacuumlazy.c    |   6 +-
 src/backend/access/heap/visibilitymap.c |   3 +-
 src/backend/commands/copy.c             |  13 ++--
 src/backend/commands/createas.c         |   9 ++-
 src/backend/commands/matview.c          |   6 +-
 src/backend/commands/tablecmds.c        |   6 +-
 src/include/access/heapam.h             |   3 +-
 src/include/access/tableam.h            |  11 +---
 11 files changed, 106 insertions(+), 66 deletions(-)

diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 5a8627507f..00416c4a99 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -28,6 +28,27 @@
  *      the POSTGRES heap access method used for all POSTGRES
  *      relations.
  *
+ * WAL CONSIDERATIONS
+ *      All heap operations are normally WAL-logged. but there are a few
+ *      exceptions. Temporary and unlogged relations never need to be
+ *      WAL-logged, but we can also skip WAL-logging for a table that was
+ *      created in the same transaction, if we don't need WAL for PITR or WAL
+ *      archival purposes (i.e. if wal_level=minimal), and we fsync() the file
+ *      to disk at COMMIT instead.
+ *
+ *      The same-relation optimization is not employed automatically on all
+ *      updates to a table that was created in the same transaction, because for
+ *      a small number of changes, it's cheaper to just create the WAL records
+ *      than fsync()ing the whole relation at COMMIT. It is only worthwhile for
+ *      (presumably) large operations like COPY, CLUSTER, or VACUUM FULL. Use
+ *      heap_register_sync() to initiate such an operation; it will cause any
+ *      subsequent updates to the table to skip WAL-logging, if possible, and
+ *      cause the heap to be synced to disk at COMMIT.
+ *
+ *      To make that work, all modifications to heap must use
+ *      BufferNeedsWAL() to check if WAL-logging is needed in this transaction
+ *      for the given block.
+ *
  *-------------------------------------------------------------------------
  */
 #include "postgres.h"
@@ -1934,7 +1955,7 @@ heap_insert(Relation relation, HeapTuple tup, CommandId cid,
     MarkBufferDirty(buffer);
 
     /* XLOG stuff */
-    if (!(options & HEAP_INSERT_SKIP_WAL) && RelationNeedsWAL(relation))
+    if (BufferNeedsWAL(relation, buffer))
     {
         XLogRecPtr    recptr;
 
@@ -2044,7 +2065,6 @@ heap_multi_insert(Relation relation, HeapTuple *tuples, int ntuples,
     int            ndone;
     PGAlignedBlock scratch;
     Page        page;
-    bool        needwal;
     Size        saveFreeSpace;
     bool        need_tuple_data = RelationIsLogicallyLogged(relation);
     bool        need_cids = RelationIsAccessibleInLogicalDecoding(relation);
@@ -2052,7 +2072,6 @@ heap_multi_insert(Relation relation, HeapTuple *tuples, 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);
     saveFreeSpace = RelationGetTargetPageFreeSpace(relation,
                                                    HEAP_DEFAULT_FILLFACTOR);
 
@@ -2094,6 +2113,7 @@ heap_multi_insert(Relation relation, HeapTuple *tuples, int ntuples,
         Buffer        vmbuffer = InvalidBuffer;
         bool        all_visible_cleared = false;
         int            nthispage;
+        bool        needwal;
 
         CHECK_FOR_INTERRUPTS();
 
@@ -2105,6 +2125,7 @@ heap_multi_insert(Relation relation, HeapTuple *tuples, int ntuples,
                                            InvalidBuffer, options, bistate,
                                            &vmbuffer, NULL);
         page = BufferGetPage(buffer);
+        needwal = BufferNeedsWAL(relation, buffer);
 
         /* NO EREPORT(ERROR) from here till changes are logged */
         START_CRIT_SECTION();
@@ -2657,7 +2678,7 @@ l1:
      * NB: heap_abort_speculative() uses the same xlog record and replay
      * routines.
      */
-    if (RelationNeedsWAL(relation))
+    if (BufferNeedsWAL(relation, buffer))
     {
         XLogRecPtr    recptr;
 
@@ -2791,6 +2812,8 @@ heap_update(Relation relation, ItemPointer otid, HeapTuple newtup,
                 vmbuffer = InvalidBuffer,
                 vmbuffer_new = InvalidBuffer;
     bool        need_toast;
+    bool        oldbuf_needs_wal,
+                newbuf_needs_wal;
     Size        newtupsize,
                 pagefree;
     bool        have_tuple_lock = false;
@@ -3342,7 +3365,7 @@ l2:
 
         MarkBufferDirty(buffer);
 
-        if (RelationNeedsWAL(relation))
+        if (BufferNeedsWAL(relation, buffer))
         {
             xl_heap_lock xlrec;
             XLogRecPtr    recptr;
@@ -3556,8 +3579,20 @@ l2:
         MarkBufferDirty(newbuf);
     MarkBufferDirty(buffer);
 
-    /* XLOG stuff */
-    if (RelationNeedsWAL(relation))
+    /*
+     *  XLOG stuff
+     *
+     * Emit heap-update log. When wal_level = minimal, we may emit insert or
+     * delete record according to wal-optimization.
+     */
+    oldbuf_needs_wal = BufferNeedsWAL(relation, buffer);
+
+    if (newbuf == buffer)
+        newbuf_needs_wal = oldbuf_needs_wal;
+    else
+        newbuf_needs_wal = BufferNeedsWAL(relation, newbuf);
+
+    if (oldbuf_needs_wal || newbuf_needs_wal)
     {
         XLogRecPtr    recptr;
 
@@ -3567,15 +3602,26 @@ l2:
          */
         if (RelationIsAccessibleInLogicalDecoding(relation))
         {
-            log_heap_new_cid(relation, &oldtup);
-            log_heap_new_cid(relation, heaptup);
+            if (oldbuf_needs_wal)
+                log_heap_new_cid(relation, &oldtup);
+            if (newbuf_needs_wal)
+                log_heap_new_cid(relation, heaptup);
         }
 
-        recptr = log_heap_update(relation, buffer,
-                                 newbuf, &oldtup, heaptup,
-                                 old_key_tuple,
-                                 all_visible_cleared,
-                                 all_visible_cleared_new);
+        if (oldbuf_needs_wal && newbuf_needs_wal)
+            recptr = log_heap_update(relation, buffer, newbuf,
+                                     &oldtup, heaptup,
+                                     old_key_tuple,
+                                     all_visible_cleared,
+                                     all_visible_cleared_new);
+        else if (oldbuf_needs_wal)
+            recptr = log_heap_delete(relation, buffer, &oldtup, old_key_tuple,
+                                     xmax_old_tuple, false,
+                                     all_visible_cleared);
+        else
+            recptr = log_heap_insert(relation, buffer, newtup,
+                                     0, all_visible_cleared_new);
+
         if (newbuf != buffer)
         {
             PageSetLSN(BufferGetPage(newbuf), recptr);
@@ -4453,7 +4499,7 @@ failed:
      * (Also, in a PITR log-shipping or 2PC environment, we have to have XLOG
      * entries for everything anyway.)
      */
-    if (RelationNeedsWAL(relation))
+    if (BufferNeedsWAL(relation, *buffer))
     {
         xl_heap_lock xlrec;
         XLogRecPtr    recptr;
@@ -5205,7 +5251,7 @@ l4:
         MarkBufferDirty(buf);
 
         /* XLOG stuff */
-        if (RelationNeedsWAL(rel))
+        if (BufferNeedsWAL(rel, buf))
         {
             xl_heap_lock_updated xlrec;
             XLogRecPtr    recptr;
@@ -5365,7 +5411,7 @@ heap_finish_speculative(Relation relation, ItemPointer tid)
     htup->t_ctid = *tid;
 
     /* XLOG stuff */
-    if (RelationNeedsWAL(relation))
+    if (BufferNeedsWAL(relation, buffer))
     {
         xl_heap_confirm xlrec;
         XLogRecPtr    recptr;
@@ -5497,7 +5543,7 @@ heap_abort_speculative(Relation relation, ItemPointer tid)
      * The WAL records generated here match heap_delete().  The same recovery
      * routines are used.
      */
-    if (RelationNeedsWAL(relation))
+    if (BufferNeedsWAL(relation, buffer))
     {
         xl_heap_delete xlrec;
         XLogRecPtr    recptr;
@@ -5606,7 +5652,7 @@ heap_inplace_update(Relation relation, HeapTuple tuple)
     MarkBufferDirty(buffer);
 
     /* XLOG stuff */
-    if (RelationNeedsWAL(relation))
+    if (BufferNeedsWAL(relation, buffer))
     {
         xl_heap_inplace xlrec;
         XLogRecPtr    recptr;
@@ -6802,8 +6848,8 @@ log_heap_clean(Relation reln, Buffer buffer,
     xl_heap_clean xlrec;
     XLogRecPtr    recptr;
 
-    /* Caller should not call me on a non-WAL-logged relation */
-    Assert(RelationNeedsWAL(reln));
+    /* Caller should not call me on non-WAL-logged buffers */
+    Assert(BufferNeedsWAL(reln, buffer));
 
     xlrec.latestRemovedXid = latestRemovedXid;
     xlrec.nredirected = nredirected;
@@ -6850,8 +6896,8 @@ log_heap_freeze(Relation reln, Buffer buffer, TransactionId cutoff_xid,
     xl_heap_freeze_page xlrec;
     XLogRecPtr    recptr;
 
-    /* Caller should not call me on a non-WAL-logged relation */
-    Assert(RelationNeedsWAL(reln));
+    /* Caller should not call me on non-WAL-logged buffers */
+    Assert(BufferNeedsWAL(reln, buffer));
     /* nor when there are no tuples to freeze */
     Assert(ntuples > 0);
 
@@ -7077,8 +7123,8 @@ log_heap_update(Relation reln, Buffer oldbuf,
     bool        init;
     int            bufflags;
 
-    /* Caller should not call me on a non-WAL-logged relation */
-    Assert(RelationNeedsWAL(reln));
+    /* Caller should not call me when no buffer needs WAL-logging */
+    Assert(BufferNeedsWAL(reln, newbuf) || BufferNeedsWAL(reln, oldbuf));
 
     XLogBeginInsert();
 
@@ -8682,9 +8728,16 @@ 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.
+ * If we did any changes to the heap bypassing the buffer manager, we must
+ * force the relation down to disk before it's safe to commit the
+ * transaction, because the direct modifications will not be flushed by
+ * the next checkpoint.
+ *
+ * We used to also use this after batch operations like COPY and CLUSTER,
+ * if we skipped using WAL and WAL is otherwise needed, but there were
+ * corner-cases involving other WAL-logged operations to the same
+ * relation, where that was not enough. heap_register_sync() should be
+ * used for that purpose instead.
  *
  * Indexes are not touched.  (Currently, index operations associated with
  * the commands that use this are WAL-logged and so do not need fsync.
diff --git a/src/backend/access/heap/pruneheap.c b/src/backend/access/heap/pruneheap.c
index a3e51922d8..a05659b168 100644
--- a/src/backend/access/heap/pruneheap.c
+++ b/src/backend/access/heap/pruneheap.c
@@ -20,6 +20,7 @@
 #include "access/htup_details.h"
 #include "access/xlog.h"
 #include "catalog/catalog.h"
+#include "catalog/storage.h"
 #include "miscadmin.h"
 #include "pgstat.h"
 #include "storage/bufmgr.h"
@@ -258,7 +259,7 @@ heap_page_prune(Relation relation, Buffer buffer, TransactionId OldestXmin,
         /*
          * Emit a WAL HEAP_CLEAN record showing what we did
          */
-        if (RelationNeedsWAL(relation))
+        if (BufferNeedsWAL(relation, buffer))
         {
             XLogRecPtr    recptr;
 
diff --git a/src/backend/access/heap/rewriteheap.c b/src/backend/access/heap/rewriteheap.c
index bce4274362..1ac77f7c14 100644
--- a/src/backend/access/heap/rewriteheap.c
+++ b/src/backend/access/heap/rewriteheap.c
@@ -654,9 +654,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
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 5c554f9465..3f5df63df8 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -929,7 +929,7 @@ lazy_scan_heap(Relation onerel, int options, LVRelStats *vacrelstats,
                  * page has been previously WAL-logged, and if not, do that
                  * now.
                  */
-                if (RelationNeedsWAL(onerel) &&
+                if (BufferNeedsWAL(onerel, buf) &&
                     PageGetLSN(page) == InvalidXLogRecPtr)
                     log_newpage_buffer(buf, true);
 
@@ -1193,7 +1193,7 @@ lazy_scan_heap(Relation onerel, int options, LVRelStats *vacrelstats,
             }
 
             /* Now WAL-log freezing if necessary */
-            if (RelationNeedsWAL(onerel))
+            if (BufferNeedsWAL(onerel, buf))
             {
                 XLogRecPtr    recptr;
 
@@ -1575,7 +1575,7 @@ lazy_vacuum_page(Relation onerel, BlockNumber blkno, Buffer buffer,
     MarkBufferDirty(buffer);
 
     /* XLOG stuff */
-    if (RelationNeedsWAL(onerel))
+    if (BufferNeedsWAL(onerel, buffer))
     {
         XLogRecPtr    recptr;
 
diff --git a/src/backend/access/heap/visibilitymap.c b/src/backend/access/heap/visibilitymap.c
index 64dfe06b26..1f5f7d92dd 100644
--- a/src/backend/access/heap/visibilitymap.c
+++ b/src/backend/access/heap/visibilitymap.c
@@ -88,6 +88,7 @@
 #include "access/heapam_xlog.h"
 #include "access/visibilitymap.h"
 #include "access/xlog.h"
+#include "catalog/storage.h"
 #include "miscadmin.h"
 #include "port/pg_bitutils.h"
 #include "storage/bufmgr.h"
@@ -276,7 +277,7 @@ visibilitymap_set(Relation rel, BlockNumber heapBlk, Buffer heapBuf,
         map[mapByte] |= (flags << mapOffset);
         MarkBufferDirty(vmBuf);
 
-        if (RelationNeedsWAL(rel))
+        if (BufferNeedsWAL(rel, heapBuf))
         {
             if (XLogRecPtrIsInvalid(recptr))
             {
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 705df8900b..1074320a5a 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -2391,8 +2391,7 @@ CopyFrom(CopyState cstate)
      *    - 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 heap_sync at the bottom of this
-     * routine first.
+     * If it does commit, commit will do heap_sync().
      *
      * As mentioned in comments in utils/rel.h, the in-same-transaction test
      * is not always set correctly, since in rare cases rd_newRelfilenodeSubid
@@ -2438,7 +2437,7 @@ CopyFrom(CopyState cstate)
     {
         hi_options |= HEAP_INSERT_SKIP_FSM;
         if (!XLogIsNeeded())
-            hi_options |= HEAP_INSERT_SKIP_WAL;
+            heap_register_sync(cstate->rel);
     }
 
     /*
@@ -3091,11 +3090,11 @@ CopyFrom(CopyState cstate)
     FreeExecutorState(estate);
 
     /*
-     * If we skipped writing WAL, then we need to sync the heap (but not
-     * indexes since those use WAL anyway)
+     * If we skipped writing WAL, then we will sync the heap at the end of
+     * the transaction. (We used to do it here, but it was later found out
+     * that to be safe, we must also avoid WAL-logging any subsequent
+     * actions on the pages we skipped WAL for). Indexes always use WAL.
      */
-    if (hi_options & HEAP_INSERT_SKIP_WAL)
-        heap_sync(cstate->rel);
 
     return processed;
 }
diff --git a/src/backend/commands/createas.c b/src/backend/commands/createas.c
index 3bdb67c697..b4431f2af3 100644
--- a/src/backend/commands/createas.c
+++ b/src/backend/commands/createas.c
@@ -558,8 +558,9 @@ 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->hi_options = HEAP_INSERT_SKIP_FSM |
-        (XLogIsNeeded() ? 0 : HEAP_INSERT_SKIP_WAL);
+    if (!XLogIsNeeded())
+        heap_register_sync(intoRelationDesc);
+    myState->hi_options = HEAP_INSERT_SKIP_FSM;
     myState->bistate = GetBulkInsertState();
 
     /* Not using WAL requires smgr_targblock be initially invalid */
@@ -604,9 +605,7 @@ intorel_shutdown(DestReceiver *self)
 
     FreeBulkInsertState(myState->bistate);
 
-    /* If we skipped using WAL, must heap_sync before commit */
-    if (myState->hi_options & HEAP_INSERT_SKIP_WAL)
-        heap_sync(myState->rel);
+    /* If we skipped using WAL, we will sync the relation at commit */
 
     /* close rel, but keep lock until commit */
     table_close(myState->rel, NoLock);
diff --git a/src/backend/commands/matview.c b/src/backend/commands/matview.c
index 5b2cbc7c89..45e693129d 100644
--- a/src/backend/commands/matview.c
+++ b/src/backend/commands/matview.c
@@ -463,7 +463,7 @@ transientrel_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
      */
     myState->hi_options = HEAP_INSERT_SKIP_FSM | HEAP_INSERT_FROZEN;
     if (!XLogIsNeeded())
-        myState->hi_options |= HEAP_INSERT_SKIP_WAL;
+        heap_register_sync(transientrel);
     myState->bistate = GetBulkInsertState();
 
     /* Not using WAL requires smgr_targblock be initially invalid */
@@ -508,9 +508,7 @@ transientrel_shutdown(DestReceiver *self)
 
     FreeBulkInsertState(myState->bistate);
 
-    /* If we skipped using WAL, must heap_sync before commit */
-    if (myState->hi_options & HEAP_INSERT_SKIP_WAL)
-        heap_sync(myState->transientrel);
+    /* If we skipped using WAL, we will sync the relation at commit */
 
     /* close transientrel, but keep lock until commit */
     table_close(myState->transientrel, NoLock);
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index c9a0e02168..54ce52eaae 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -4664,10 +4664,10 @@ ATRewriteTable(AlteredTableInfo *tab, Oid OIDNewHeap, LOCKMODE lockmode)
     {
         mycid = GetCurrentCommandId(true);
         bistate = GetBulkInsertState();
-
         hi_options = HEAP_INSERT_SKIP_FSM;
+
         if (!XLogIsNeeded())
-            hi_options |= HEAP_INSERT_SKIP_WAL;
+            heap_register_sync(newrel);
     }
     else
     {
@@ -4958,8 +4958,6 @@ ATRewriteTable(AlteredTableInfo *tab, Oid OIDNewHeap, LOCKMODE lockmode)
         FreeBulkInsertState(bistate);
 
         /* If we skipped writing WAL, then we need to sync the heap. */
-        if (hi_options & HEAP_INSERT_SKIP_WAL)
-            heap_sync(newrel);
 
         table_close(newrel, NoLock);
     }
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index 3d4fb7f3c3..97114aed3e 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -29,11 +29,10 @@
 
 
 /* "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
-#define HEAP_INSERT_SPECULATIVE 0x0010
+#define HEAP_INSERT_SPECULATIVE 0x0008
 
 typedef struct BulkInsertStateData *BulkInsertState;
 
diff --git a/src/include/access/tableam.h b/src/include/access/tableam.h
index 4699335cdf..cf7f8e7da0 100644
--- a/src/include/access/tableam.h
+++ b/src/include/access/tableam.h
@@ -94,10 +94,9 @@ typedef struct TM_FailureData
 } TM_FailureData;
 
 /* "options" flag bits for table_insert */
-#define TABLE_INSERT_SKIP_WAL        0x0001
-#define TABLE_INSERT_SKIP_FSM        0x0002
-#define TABLE_INSERT_FROZEN            0x0004
-#define TABLE_INSERT_NO_LOGICAL        0x0008
+#define TABLE_INSERT_SKIP_FSM        0x0001
+#define TABLE_INSERT_FROZEN            0x0002
+#define TABLE_INSERT_NO_LOGICAL        0x0004
 
 /* flag bits fortable_lock_tuple */
 /* Follow tuples whose update is in progress if lock modes don't conflict  */
@@ -702,10 +701,6 @@ table_tuple_satisfies_snapshot(Relation rel, TupleTableSlot *slot, Snapshot snap
  * behaviour of the AM. Several options might be ignored by AMs not supporting
  * them.
  *
- * If the TABLE_INSERT_SKIP_WAL option is specified, the new tuple will not
- * necessarily 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.  It's
-- 
2.16.3

From 5b047a9514613c42c9ef1fb395ca401b55d7e2de Mon Sep 17 00:00:00 2001
From: Kyotaro Horiguchi <horiguchi.kyotaro@lab.ntt.co.jp>
Date: Tue, 26 Mar 2019 15:32:28 +0900
Subject: [PATCH 6/8] Change cluster to use the new pending sync infrastructure

Apply the pending-sync infrastructure to CLUSTER command. It gets
benefits from moving file sync from command end to transaction end
when wal_level is minimal.
---
 src/backend/access/heap/rewriteheap.c | 25 +++++--------------------
 src/backend/commands/cluster.c        | 13 +++++--------
 src/include/access/rewriteheap.h      |  2 +-
 3 files changed, 11 insertions(+), 29 deletions(-)

diff --git a/src/backend/access/heap/rewriteheap.c b/src/backend/access/heap/rewriteheap.c
index 1ac77f7c14..494f7fcd41 100644
--- a/src/backend/access/heap/rewriteheap.c
+++ b/src/backend/access/heap/rewriteheap.c
@@ -116,6 +116,7 @@
 #include "access/xloginsert.h"
 
 #include "catalog/catalog.h"
+#include "catalog/storage.h"
 
 #include "lib/ilist.h"
 
@@ -144,7 +145,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 +238,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
  * min_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 +269,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 +327,7 @@ end_heap_rewrite(RewriteState state)
     /* Write the last page, if any */
     if (state->rs_buffer_valid)
     {
-        if (state->rs_use_wal)
+        if (BlockNeedsWAL(state->rs_new_rel, state->rs_blockno))
             log_newpage(&state->rs_new_rel->rd_node,
                         MAIN_FORKNUM,
                         state->rs_blockno,
@@ -344,19 +341,7 @@ end_heap_rewrite(RewriteState state)
                    (char *) state->rs_buffer, true);
     }
 
-    /*
-     * If the rel is WAL-logged, must fsync before commit.  We use heap_sync
-     * to ensure that the toast table gets fsync'd too.
-     *
-     * 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
-     * reason is the same as in tablecmds.c's copy_relation_data(): 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);
+    /* If we skipped using WAL, we will sync the relation at commit */
 
     logical_end_heap_rewrite(state);
 
@@ -692,7 +677,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 (BlockNeedsWAL(state->rs_new_rel, state->rs_blockno))
                 log_newpage(&state->rs_new_rel->rd_node,
                             MAIN_FORKNUM,
                             state->rs_blockno,
diff --git a/src/backend/commands/cluster.c b/src/backend/commands/cluster.c
index 205070b83d..34c1a5e96c 100644
--- a/src/backend/commands/cluster.c
+++ b/src/backend/commands/cluster.c
@@ -788,7 +788,6 @@ copy_heap_data(Oid OIDNewHeap, Oid OIDOldHeap, Oid OIDOldIndex, bool verbose,
     IndexScanDesc indexScan;
     TableScanDesc tableScan;
     HeapScanDesc heapScan;
-    bool        use_wal;
     bool        is_system_catalog;
     TransactionId OldestXmin;
     TransactionId FreezeXid;
@@ -847,13 +846,11 @@ copy_heap_data(Oid OIDNewHeap, Oid OIDOldHeap, Oid OIDOldIndex, bool verbose,
         LockRelationOid(OldHeap->rd_rel->reltoastrelid, AccessExclusiveLock);
 
     /*
-     * We need to log the copied data in WAL iff WAL archiving/streaming is
-     * enabled AND it's a WAL-logged rel.
+     * If wal_level is minimal, we skip WAL-logging even for WAL-logging
+     * relations. The heap will be synced at commit.
      */
-    use_wal = XLogIsNeeded() && RelationNeedsWAL(NewHeap);
-
-    /* use_wal off requires smgr_targblock be initially invalid */
-    Assert(RelationGetTargetBlock(NewHeap) == InvalidBlockNumber);
+    if (!XLogIsNeeded())
+        heap_register_sync(NewHeap);
 
     /*
      * If both tables have TOAST tables, perform toast swap by content.  It is
@@ -920,7 +917,7 @@ copy_heap_data(Oid OIDNewHeap, Oid OIDOldHeap, Oid OIDOldIndex, bool verbose,
 
     /* Initialize the rewrite operation */
     rwstate = begin_heap_rewrite(OldHeap, NewHeap, OldestXmin, FreezeXid,
-                                 MultiXactCutoff, use_wal);
+                                 MultiXactCutoff);
 
     /*
      * Decide whether to use an indexscan or seqscan-and-optional-sort to scan
diff --git a/src/include/access/rewriteheap.h b/src/include/access/rewriteheap.h
index 6006249d96..64efecf48b 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);
-- 
2.16.3

From 50740e21bcb34b89334f7e5756d757b469a087c9 Mon Sep 17 00:00:00 2001
From: Kyotaro Horiguchi <horiguchi.kyotaro@lab.ntt.co.jp>
Date: Mon, 25 Mar 2019 20:39:21 +0900
Subject: [PATCH 7/8] Change ALTER TABLESPACE to use the pending-sync
 infrastructure

Apply heap_register_sync() to ATLER TABLESPACE stuff.
---
 src/backend/commands/tablecmds.c | 54 +++++++++++++++++++++-------------------
 1 file changed, 28 insertions(+), 26 deletions(-)

diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 54ce52eaae..aabb3806f6 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -84,7 +84,6 @@
 #include "storage/lmgr.h"
 #include "storage/lock.h"
 #include "storage/predicate.h"
-#include "storage/smgr.h"
 #include "utils/acl.h"
 #include "utils/builtins.h"
 #include "utils/fmgroids.h"
@@ -11891,7 +11890,7 @@ copy_relation_data(SMgrRelation src, SMgrRelation dst,
 {
     PGAlignedBlock buf;
     Page        page;
-    bool        use_wal;
+    bool        use_wal = false;
     bool        copying_initfork;
     BlockNumber nblocks;
     BlockNumber blkno;
@@ -11906,12 +11905,33 @@ copy_relation_data(SMgrRelation src, SMgrRelation dst,
     copying_initfork = relpersistence == RELPERSISTENCE_UNLOGGED &&
         forkNum == INIT_FORKNUM;
 
-    /*
-     * We need to log the copied data in WAL iff WAL archiving/streaming is
-     * enabled AND it's a permanent relation.
-     */
-    use_wal = XLogIsNeeded() &&
-        (relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork);
+    if (relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork)
+    {
+        /*
+         * We need to log the copied data in WAL iff WAL archiving/streaming
+         * is enabled AND it's a permanent relation.
+         */
+        if (XLogIsNeeded())
+            use_wal = true;
+
+        /*
+         * If the rel is WAL-logged, must fsync at commit.  We do the same 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.
+         */
+        RecordPendingSync(dst, forkNum);
+    }
 
     nblocks = smgrnblocks(src, forkNum);
 
@@ -11948,24 +11968,6 @@ copy_relation_data(SMgrRelation src, SMgrRelation dst,
          */
         smgrextend(dst, forkNum, blkno, buf.data, true);
     }
-
-    /*
-     * If the rel is WAL-logged, must fsync before commit.  We use heap_sync
-     * to ensure that the toast table gets fsync'd too.  (For a temp or
-     * unlogged rel we don't care since the data will be gone after a crash
-     * anyway.)
-     *
-     * It's obvious that we must do this when not WAL-logging the copy. It's
-     * less obvious that we have to do it even if we did WAL-log the copied
-     * pages. The reason is that since we're copying outside shared buffers, a
-     * CHECKPOINT occurring during the copy has no way to flush the previously
-     * written data to disk (indeed it won't know the new rel even exists).  A
-     * crash later on would replay WAL from the checkpoint, therefore it
-     * wouldn't replay our earlier WAL entries. If we do not fsync those pages
-     * here, they might still not be on disk when the crash occurs.
-     */
-    if (relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork)
-        smgrimmedsync(dst, forkNum);
 }
 
 /*
-- 
2.16.3

From 2928ccd4197d237294215e4b9f0c9a6e8aa42eae Mon Sep 17 00:00:00 2001
From: Kyotaro Horiguchi <horiguchi.kyotaro@lab.ntt.co.jp>
Date: Tue, 26 Mar 2019 14:48:26 +0900
Subject: [PATCH 8/8] Optimize WAL-logging on btree bulk insertion

Likewise the heap case, bulk insertion into btree can be optimized to
omit WAL-logging on certain conditions.
---
 src/backend/access/heap/heapam.c      | 13 +++++++++++++
 src/backend/access/nbtree/nbtinsert.c |  5 ++++-
 src/backend/access/nbtree/nbtsort.c   | 23 +++++++----------------
 3 files changed, 24 insertions(+), 17 deletions(-)

diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 00416c4a99..c28b479141 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -8870,6 +8870,8 @@ heap_mask(char *pagedata, BlockNumber blkno)
 void
 heap_register_sync(Relation rel)
 {
+    ListCell   *indlist;
+
     /* non-WAL-logged tables never need fsync */
     if (!RelationNeedsWAL(rel))
         return;
@@ -8883,4 +8885,15 @@ heap_register_sync(Relation rel)
         RecordWALSkipping(toastrel);
         heap_close(toastrel, AccessShareLock);
     }
+
+    /* Do the same to all index relations */
+    foreach(indlist, RelationGetIndexList(rel))
+    {
+        Oid            indexId = lfirst_oid(indlist);
+        Relation    indexRel;
+
+        indexRel = index_open(indexId, AccessShareLock);
+        RecordWALSkipping(indexRel);
+        index_close(indexRel, NoLock);
+    }
 }
diff --git a/src/backend/access/nbtree/nbtinsert.c b/src/backend/access/nbtree/nbtinsert.c
index 96b7593fc1..fadcc09cb1 100644
--- a/src/backend/access/nbtree/nbtinsert.c
+++ b/src/backend/access/nbtree/nbtinsert.c
@@ -20,6 +20,7 @@
 #include "access/tableam.h"
 #include "access/transam.h"
 #include "access/xloginsert.h"
+#include "catalog/storage.h"
 #include "miscadmin.h"
 #include "storage/lmgr.h"
 #include "storage/predicate.h"
@@ -1096,7 +1097,9 @@ _bt_insertonpg(Relation rel,
             cachedBlock = BufferGetBlockNumber(buf);
 
         /* XLOG stuff */
-        if (RelationNeedsWAL(rel))
+        if (BufferNeedsWAL(rel, buf) ||
+            (!P_ISLEAF(lpageop) && BufferNeedsWAL(rel, cbuf)) ||
+            (BufferIsValid(metabuf) && BufferNeedsWAL(rel, metabuf)))
         {
             xl_btree_insert xlrec;
             xl_btree_metadata xlmeta;
diff --git a/src/backend/access/nbtree/nbtsort.c b/src/backend/access/nbtree/nbtsort.c
index e65d4aab0f..90a5d6ae13 100644
--- a/src/backend/access/nbtree/nbtsort.c
+++ b/src/backend/access/nbtree/nbtsort.c
@@ -66,6 +66,7 @@
 #include "access/xlog.h"
 #include "access/xloginsert.h"
 #include "catalog/index.h"
+#include "catalog/storage.h"
 #include "miscadmin.h"
 #include "pgstat.h"
 #include "storage/smgr.h"
@@ -264,7 +265,6 @@ typedef struct BTWriteState
     Relation    heap;
     Relation    index;
     BTScanInsert inskey;        /* generic insertion scankey */
-    bool        btws_use_wal;    /* dump pages to WAL? */
     BlockNumber btws_pages_alloced; /* # pages allocated */
     BlockNumber btws_pages_written; /* # pages written out */
     Page        btws_zeropage;    /* workspace for filling zeroes */
@@ -334,6 +334,10 @@ btbuild(Relation heap, Relation index, IndexInfo *indexInfo)
 
     reltuples = _bt_spools_heapscan(heap, index, &buildstate, indexInfo);
 
+    /* Skip WAL-logging if wal_level = minimal */
+    if (!XLogIsNeeded())
+        RecordWALSkipping(index);
+
     /*
      * Finish the build by (1) completing the sort of the spool file, (2)
      * inserting the sorted tuples into btree pages and (3) building the upper
@@ -543,12 +547,6 @@ _bt_leafbuild(BTSpool *btspool, BTSpool *btspool2)
     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);
-
     /* reserve the metapage */
     wstate.btws_pages_alloced = BTREE_METAPAGE + 1;
     wstate.btws_pages_written = 0;
@@ -622,15 +620,8 @@ _bt_blwritepage(BTWriteState *wstate, Page page, BlockNumber blkno)
     /* Ensure rd_smgr is open (could have been closed by relcache flush!) */
     RelationOpenSmgr(wstate->index);
 
-    /* XLOG stuff
-     *
-     * Even if minimal mode, WAL is required here if truncation happened after
-     * being created in the same transaction. It is not needed otherwise but
-     * we don't bother identifying the case precisely.
-     */
-    if (wstate->btws_use_wal ||
-        (RelationNeedsWAL(wstate->index) &&
-         (blkno == BTREE_METAPAGE && BTPageGetMeta(page)->btm_root == 0)))
+    /* XLOG stuff */
+    if (BlockNeedsWAL(wstate->index, blkno))
     {
         /* We use the heap NEWPAGE record type for this */
         log_newpage(&wstate->index->rd_node, MAIN_FORKNUM, blkno, page, true);
-- 
2.16.3


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

Предыдущее
От: Amit Langote
Дата:
Сообщение: Re: speeding up planning with partitions
Следующее
От: Surafel Temesgen
Дата:
Сообщение: Re: Re: FETCH FIRST clause WITH TIES option