]> granicus.if.org Git - postgresql/blobdiff - src/backend/replication/logical/reorderbuffer.c
Change extension of spilled ReorderBufferChange data to ".tmp".
[postgresql] / src / backend / replication / logical / reorderbuffer.c
index 45207086ac0dafae9a0f9af0e46344e9e44842fd..0d72ac1c791f50a289a06ca432476fda365662a1 100644 (file)
@@ -4,7 +4,7 @@
  *       PostgreSQL logical replay/reorder buffer management
  *
  *
- * Copyright (c) 2012-2016, PostgreSQL Global Development Group
+ * Copyright (c) 2012-2018, PostgreSQL Global Development Group
  *
  *
  * IDENTIFICATION
@@ -15,7 +15,7 @@
  *       they are written to the WAL and is responsible to reassemble them into
  *       toplevel transaction sized pieces. When a transaction is completely
  *       reassembled - signalled by reading the transaction commit record - it
- *       will then call the output plugin (c.f. ReorderBufferCommit()) with the
+ *       will then call the output plugin (cf. ReorderBufferCommit()) with the
  *       individual changes. The output plugins rely on snapshots built by
  *       snapbuild.c which hands them to us.
  *
  *       transaction there will be no other data carrying records between a row's
  *       toast chunks and the row data itself. See ReorderBufferToast* for
  *       details.
+ *
+ *       ReorderBuffer uses two special memory context types - SlabContext for
+ *       allocations of fixed-length structures (changes and transactions), and
+ *       GenerationContext for the variable-length transaction data (allocated
+ *       and freed in groups with similar lifespan).
+ *
  * -------------------------------------------------------------------------
  */
 #include "postgres.h"
 #include "catalog/catalog.h"
 #include "lib/binaryheap.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "replication/logical.h"
 #include "replication/reorderbuffer.h"
 #include "replication/slot.h"
-#include "replication/snapbuild.h"             /* just for SnapBuildSnapDecRefcount */
+#include "replication/snapbuild.h"     /* just for SnapBuildSnapDecRefcount */
 #include "storage/bufmgr.h"
 #include "storage/fd.h"
 #include "storage/sinval.h"
@@ -123,8 +130,8 @@ typedef struct ReorderBufferToastEnt
        Size            num_chunks;             /* number of chunks we've already seen */
        Size            size;                   /* combined size of chunks seen */
        dlist_head      chunks;                 /* linked list of chunks */
-       struct varlena *reconstructed;          /* reconstructed varlena now pointed
-                                                                                * to in main tup */
+       struct varlena *reconstructed;  /* reconstructed varlena now pointed to in
+                                                                        * main tup */
 } ReorderBufferToastEnt;
 
 /* Disk serialization support datastructures */
@@ -149,18 +156,6 @@ typedef struct ReorderBufferDiskChange
  */
 static const Size max_changes_in_memory = 4096;
 
-/*
- * We use a very simple form of a slab allocator for frequently allocated
- * objects, simply keeping a fixed number in a linked list when unused,
- * instead pfree()ing them. Without that in many workloads aset.c becomes a
- * major bottleneck, especially when spilling to disk while decoding batch
- * workloads.
- */
-static const Size max_cached_changes = 4096 * 2;
-static const Size max_cached_tuplebufs = 4096 * 2;             /* ~8MB */
-static const Size max_cached_transactions = 512;
-
-
 /* ---------------------------------------
  * primary reorderbuffer support routines
  * ---------------------------------------
@@ -170,6 +165,8 @@ static void ReorderBufferReturnTXN(ReorderBuffer *rb, ReorderBufferTXN *txn);
 static ReorderBufferTXN *ReorderBufferTXNByXid(ReorderBuffer *rb,
                                          TransactionId xid, bool create, bool *is_new,
                                          XLogRecPtr lsn, bool create_as_top);
+static void ReorderBufferTransferSnapToParent(ReorderBufferTXN *txn,
+                                                                 ReorderBufferTXN *subtxn);
 
 static void AssertTXNLsnOrder(ReorderBuffer *rb);
 
@@ -182,8 +179,7 @@ static void AssertTXNLsnOrder(ReorderBuffer *rb);
  * ---------------------------------------
  */
 static ReorderBufferIterTXNState *ReorderBufferIterTXNInit(ReorderBuffer *rb, ReorderBufferTXN *txn);
-static ReorderBufferChange *
-                       ReorderBufferIterTXNNext(ReorderBuffer *rb, ReorderBufferIterTXNState *state);
+static ReorderBufferChange *ReorderBufferIterTXNNext(ReorderBuffer *rb, ReorderBufferIterTXNState *state);
 static void ReorderBufferIterTXNFinish(ReorderBuffer *rb,
                                                   ReorderBufferIterTXNState *state);
 static void ReorderBufferExecuteInvalidations(ReorderBuffer *rb, ReorderBufferTXN *txn);
@@ -202,6 +198,9 @@ static Size ReorderBufferRestoreChanges(ReorderBuffer *rb, ReorderBufferTXN *txn
 static void ReorderBufferRestoreChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
                                                   char *change);
 static void ReorderBufferRestoreCleanup(ReorderBuffer *rb, ReorderBufferTXN *txn);
+static void ReorderBufferCleanupSerializedTXNs(const char *slotname);
+static void ReorderBufferSerializedPath(char *path, ReplicationSlot *slot,
+                                                       TransactionId xid, XLogSegNo segno);
 
 static void ReorderBufferFreeSnap(ReorderBuffer *rb, Snapshot snap);
 static Snapshot ReorderBufferCopySnap(ReorderBuffer *rb, Snapshot orig_snap,
@@ -220,7 +219,8 @@ static void ReorderBufferToastAppendChunk(ReorderBuffer *rb, ReorderBufferTXN *t
 
 
 /*
- * Allocate a new ReorderBuffer
+ * Allocate a new ReorderBuffer and clean out any old serialized state from
+ * prior ReorderBuffer instances for the same slot.
  */
 ReorderBuffer *
 ReorderBufferAllocate(void)
@@ -229,12 +229,12 @@ ReorderBufferAllocate(void)
        HASHCTL         hash_ctl;
        MemoryContext new_ctx;
 
+       Assert(MyReplicationSlot != NULL);
+
        /* allocate memory in own context, to have better accountability */
        new_ctx = AllocSetContextCreate(CurrentMemoryContext,
                                                                        "ReorderBuffer",
-                                                                       ALLOCSET_DEFAULT_MINSIZE,
-                                                                       ALLOCSET_DEFAULT_INITSIZE,
-                                                                       ALLOCSET_DEFAULT_MAXSIZE);
+                                                                       ALLOCSET_DEFAULT_SIZES);
 
        buffer =
                (ReorderBuffer *) MemoryContextAlloc(new_ctx, sizeof(ReorderBuffer));
@@ -243,6 +243,20 @@ ReorderBufferAllocate(void)
 
        buffer->context = new_ctx;
 
+       buffer->change_context = SlabContextCreate(new_ctx,
+                                                                                          "Change",
+                                                                                          SLAB_DEFAULT_BLOCK_SIZE,
+                                                                                          sizeof(ReorderBufferChange));
+
+       buffer->txn_context = SlabContextCreate(new_ctx,
+                                                                                       "TXN",
+                                                                                       SLAB_DEFAULT_BLOCK_SIZE,
+                                                                                       sizeof(ReorderBufferTXN));
+
+       buffer->tup_context = GenerationContextCreate(new_ctx,
+                                                                                                 "Tuples",
+                                                                                                 SLAB_LARGE_BLOCK_SIZE);
+
        hash_ctl.keysize = sizeof(TransactionId);
        hash_ctl.entrysize = sizeof(ReorderBufferTXNByIdEnt);
        hash_ctl.hcxt = buffer->context;
@@ -253,19 +267,20 @@ ReorderBufferAllocate(void)
        buffer->by_txn_last_xid = InvalidTransactionId;
        buffer->by_txn_last_txn = NULL;
 
-       buffer->nr_cached_transactions = 0;
-       buffer->nr_cached_changes = 0;
-       buffer->nr_cached_tuplebufs = 0;
-
        buffer->outbuf = NULL;
        buffer->outbufsize = 0;
 
        buffer->current_restart_decoding_lsn = InvalidXLogRecPtr;
 
        dlist_init(&buffer->toplevel_by_lsn);
-       dlist_init(&buffer->cached_transactions);
-       dlist_init(&buffer->cached_changes);
-       slist_init(&buffer->cached_tuplebufs);
+       dlist_init(&buffer->txns_by_base_snapshot_lsn);
+
+       /*
+        * Ensure there's no stale data from prior uses of this slot, in case some
+        * prior exit avoided calling ReorderBufferFree. Failure to do this can
+        * produce duplicated txns, and it's very cheap if there's nothing there.
+        */
+       ReorderBufferCleanupSerializedTXNs(NameStr(MyReplicationSlot->data.name));
 
        return buffer;
 }
@@ -283,6 +298,9 @@ ReorderBufferFree(ReorderBuffer *rb)
         * memory context.
         */
        MemoryContextDelete(context);
+
+       /* Free disk space used by unconsumed reorder buffers */
+       ReorderBufferCleanupSerializedTXNs(NameStr(MyReplicationSlot->data.name));
 }
 
 /*
@@ -293,19 +311,8 @@ ReorderBufferGetTXN(ReorderBuffer *rb)
 {
        ReorderBufferTXN *txn;
 
-       /* check the slab cache */
-       if (rb->nr_cached_transactions > 0)
-       {
-               rb->nr_cached_transactions--;
-               txn = (ReorderBufferTXN *)
-                       dlist_container(ReorderBufferTXN, node,
-                                                       dlist_pop_head_node(&rb->cached_transactions));
-       }
-       else
-       {
-               txn = (ReorderBufferTXN *)
-                       MemoryContextAlloc(rb->context, sizeof(ReorderBufferTXN));
-       }
+       txn = (ReorderBufferTXN *)
+               MemoryContextAlloc(rb->txn_context, sizeof(ReorderBufferTXN));
 
        memset(txn, 0, sizeof(ReorderBufferTXN));
 
@@ -318,9 +325,6 @@ ReorderBufferGetTXN(ReorderBuffer *rb)
 
 /*
  * Free a ReorderBufferTXN.
- *
- * Deallocation might be delayed for efficiency purposes, for details check
- * the comments above max_cached_changes's definition.
  */
 static void
 ReorderBufferReturnTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
@@ -346,41 +350,19 @@ ReorderBufferReturnTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
                txn->invalidations = NULL;
        }
 
-       /* check whether to put into the slab cache */
-       if (rb->nr_cached_transactions < max_cached_transactions)
-       {
-               rb->nr_cached_transactions++;
-               dlist_push_head(&rb->cached_transactions, &txn->node);
-               VALGRIND_MAKE_MEM_UNDEFINED(txn, sizeof(ReorderBufferTXN));
-               VALGRIND_MAKE_MEM_DEFINED(&txn->node, sizeof(txn->node));
-       }
-       else
-       {
-               pfree(txn);
-       }
+       pfree(txn);
 }
 
 /*
- * Get an unused, possibly preallocated, ReorderBufferChange.
+ * Get an fresh ReorderBufferChange.
  */
 ReorderBufferChange *
 ReorderBufferGetChange(ReorderBuffer *rb)
 {
        ReorderBufferChange *change;
 
-       /* check the slab cache */
-       if (rb->nr_cached_changes)
-       {
-               rb->nr_cached_changes--;
-               change = (ReorderBufferChange *)
-                       dlist_container(ReorderBufferChange, node,
-                                                       dlist_pop_head_node(&rb->cached_changes));
-       }
-       else
-       {
-               change = (ReorderBufferChange *)
-                       MemoryContextAlloc(rb->context, sizeof(ReorderBufferChange));
-       }
+       change = (ReorderBufferChange *)
+               MemoryContextAlloc(rb->change_context, sizeof(ReorderBufferChange));
 
        memset(change, 0, sizeof(ReorderBufferChange));
        return change;
@@ -388,9 +370,6 @@ ReorderBufferGetChange(ReorderBuffer *rb)
 
 /*
  * Free an ReorderBufferChange.
- *
- * Deallocation might be delayed for efficiency purposes, for details check
- * the comments above max_cached_changes's definition.
  */
 void
 ReorderBufferReturnChange(ReorderBuffer *rb, ReorderBufferChange *change)
@@ -433,27 +412,16 @@ ReorderBufferReturnChange(ReorderBuffer *rb, ReorderBufferChange *change)
                case REORDER_BUFFER_CHANGE_INTERNAL_SPEC_CONFIRM:
                case REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID:
                case REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID:
+               case REORDER_BUFFER_CHANGE_TRUNCATE:
                        break;
        }
 
-       /* check whether to put into the slab cache */
-       if (rb->nr_cached_changes < max_cached_changes)
-       {
-               rb->nr_cached_changes++;
-               dlist_push_head(&rb->cached_changes, &change->node);
-               VALGRIND_MAKE_MEM_UNDEFINED(change, sizeof(ReorderBufferChange));
-               VALGRIND_MAKE_MEM_DEFINED(&change->node, sizeof(change->node));
-       }
-       else
-       {
-               pfree(change);
-       }
+       pfree(change);
 }
 
-
 /*
- * Get an unused, possibly preallocated, ReorderBufferTupleBuf fitting at
- * least a tuple of size tuple_len (excluding header overhead).
+ * Get a fresh ReorderBufferTupleBuf fitting at least a tuple of size
+ * tuple_len (excluding header overhead).
  */
 ReorderBufferTupleBuf *
 ReorderBufferGetTupleBuf(ReorderBuffer *rb, Size tuple_len)
@@ -463,70 +431,23 @@ ReorderBufferGetTupleBuf(ReorderBuffer *rb, Size tuple_len)
 
        alloc_len = tuple_len + SizeofHeapTupleHeader;
 
-       /*
-        * Most tuples are below MaxHeapTupleSize, so we use a slab allocator for
-        * those. Thus always allocate at least MaxHeapTupleSize. Note that tuples
-        * tuples generated for oldtuples can be bigger, as they don't have
-        * out-of-line toast columns.
-        */
-       if (alloc_len < MaxHeapTupleSize)
-               alloc_len = MaxHeapTupleSize;
-
-
-       /* if small enough, check the slab cache */
-       if (alloc_len <= MaxHeapTupleSize && rb->nr_cached_tuplebufs)
-       {
-               rb->nr_cached_tuplebufs--;
-               tuple = slist_container(ReorderBufferTupleBuf, node,
-                                                               slist_pop_head_node(&rb->cached_tuplebufs));
-               Assert(tuple->alloc_tuple_size == MaxHeapTupleSize);
-#ifdef USE_ASSERT_CHECKING
-               memset(&tuple->tuple, 0xa9, sizeof(HeapTupleData));
-               VALGRIND_MAKE_MEM_UNDEFINED(&tuple->tuple, sizeof(HeapTupleData));
-#endif
-               tuple->tuple.t_data = ReorderBufferTupleBufData(tuple);
-#ifdef USE_ASSERT_CHECKING
-               memset(tuple->tuple.t_data, 0xa8, tuple->alloc_tuple_size);
-               VALGRIND_MAKE_MEM_UNDEFINED(tuple->tuple.t_data, tuple->alloc_tuple_size);
-#endif
-       }
-       else
-       {
-               tuple = (ReorderBufferTupleBuf *)
-                       MemoryContextAlloc(rb->context,
-                                                          sizeof(ReorderBufferTupleBuf) +
-                                                          MAXIMUM_ALIGNOF + alloc_len);
-               tuple->alloc_tuple_size = alloc_len;
-               tuple->tuple.t_data = ReorderBufferTupleBufData(tuple);
-       }
+       tuple = (ReorderBufferTupleBuf *)
+               MemoryContextAlloc(rb->tup_context,
+                                                  sizeof(ReorderBufferTupleBuf) +
+                                                  MAXIMUM_ALIGNOF + alloc_len);
+       tuple->alloc_tuple_size = alloc_len;
+       tuple->tuple.t_data = ReorderBufferTupleBufData(tuple);
 
        return tuple;
 }
 
 /*
  * Free an ReorderBufferTupleBuf.
- *
- * Deallocation might be delayed for efficiency purposes, for details check
- * the comments above max_cached_changes's definition.
  */
 void
 ReorderBufferReturnTupleBuf(ReorderBuffer *rb, ReorderBufferTupleBuf *tuple)
 {
-       /* check whether to put into the slab cache, oversized tuples never are */
-       if (tuple->alloc_tuple_size == MaxHeapTupleSize &&
-               rb->nr_cached_tuplebufs < max_cached_tuplebufs)
-       {
-               rb->nr_cached_tuplebufs++;
-               slist_push_head(&rb->cached_tuplebufs, &tuple->node);
-               VALGRIND_MAKE_MEM_UNDEFINED(tuple->tuple.t_data, tuple->alloc_tuple_size);
-               VALGRIND_MAKE_MEM_UNDEFINED(tuple, sizeof(ReorderBufferTupleBuf));
-               VALGRIND_MAKE_MEM_DEFINED(&tuple->node, sizeof(tuple->node));
-               VALGRIND_MAKE_MEM_DEFINED(&tuple->alloc_tuple_size, sizeof(tuple->alloc_tuple_size));
-       }
-       else
-       {
-               pfree(tuple);
-       }
+       pfree(tuple);
 }
 
 /*
@@ -544,7 +465,6 @@ ReorderBufferTXNByXid(ReorderBuffer *rb, TransactionId xid, bool create,
        bool            found;
 
        Assert(TransactionIdIsValid(xid));
-       Assert(!create || lsn != InvalidXLogRecPtr);
 
        /*
         * Check the one-entry lookup cache first
@@ -563,7 +483,7 @@ ReorderBufferTXNByXid(ReorderBuffer *rb, TransactionId xid, bool create,
                }
 
                /*
-                * cached as non-existant, and asked not to create? Then nothing else
+                * cached as non-existent, and asked not to create? Then nothing else
                 * to do.
                 */
                if (!create)
@@ -588,6 +508,7 @@ ReorderBufferTXNByXid(ReorderBuffer *rb, TransactionId xid, bool create,
        {
                /* initialize the new entry, if creation was requested */
                Assert(ent != NULL);
+               Assert(lsn != InvalidXLogRecPtr);
 
                ent->txn = ReorderBufferGetTXN(rb);
                ent->txn->xid = xid;
@@ -666,8 +587,8 @@ ReorderBufferQueueMessage(ReorderBuffer *rb, TransactionId xid,
        }
        else
        {
-               ReorderBufferTXN   *txn = NULL;
-               volatile Snapshot       snapshot_now = snapshot;
+               ReorderBufferTXN *txn = NULL;
+               volatile Snapshot snapshot_now = snapshot;
 
                if (xid != InvalidTransactionId)
                        txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
@@ -689,43 +610,80 @@ ReorderBufferQueueMessage(ReorderBuffer *rb, TransactionId xid,
        }
 }
 
-
+/*
+ * AssertTXNLsnOrder
+ *             Verify LSN ordering of transaction lists in the reorderbuffer
+ *
+ * Other LSN-related invariants are checked too.
+ *
+ * No-op if assertions are not in use.
+ */
 static void
 AssertTXNLsnOrder(ReorderBuffer *rb)
 {
 #ifdef USE_ASSERT_CHECKING
        dlist_iter      iter;
        XLogRecPtr      prev_first_lsn = InvalidXLogRecPtr;
+       XLogRecPtr      prev_base_snap_lsn = InvalidXLogRecPtr;
 
        dlist_foreach(iter, &rb->toplevel_by_lsn)
        {
-               ReorderBufferTXN *cur_txn;
+               ReorderBufferTXN *cur_txn = dlist_container(ReorderBufferTXN, node,
+                                                                                                       iter.cur);
 
-               cur_txn = dlist_container(ReorderBufferTXN, node, iter.cur);
+               /* start LSN must be set */
                Assert(cur_txn->first_lsn != InvalidXLogRecPtr);
 
+               /* If there is an end LSN, it must be higher than start LSN */
                if (cur_txn->end_lsn != InvalidXLogRecPtr)
                        Assert(cur_txn->first_lsn <= cur_txn->end_lsn);
 
+               /* Current initial LSN must be strictly higher than previous */
                if (prev_first_lsn != InvalidXLogRecPtr)
                        Assert(prev_first_lsn < cur_txn->first_lsn);
 
+               /* known-as-subtxn txns must not be listed */
                Assert(!cur_txn->is_known_as_subxact);
+
                prev_first_lsn = cur_txn->first_lsn;
        }
+
+       dlist_foreach(iter, &rb->txns_by_base_snapshot_lsn)
+       {
+               ReorderBufferTXN *cur_txn = dlist_container(ReorderBufferTXN,
+                                                                                                       base_snapshot_node,
+                                                                                                       iter.cur);
+
+               /* base snapshot (and its LSN) must be set */
+               Assert(cur_txn->base_snapshot != NULL);
+               Assert(cur_txn->base_snapshot_lsn != InvalidXLogRecPtr);
+
+               /* current LSN must be strictly higher than previous */
+               if (prev_base_snap_lsn != InvalidXLogRecPtr)
+                       Assert(prev_base_snap_lsn < cur_txn->base_snapshot_lsn);
+
+               /* known-as-subtxn txns must not be listed */
+               Assert(!cur_txn->is_known_as_subxact);
+
+               prev_base_snap_lsn = cur_txn->base_snapshot_lsn;
+       }
 #endif
 }
 
+/*
+ * ReorderBufferGetOldestTXN
+ *             Return oldest transaction in reorderbuffer
+ */
 ReorderBufferTXN *
 ReorderBufferGetOldestTXN(ReorderBuffer *rb)
 {
        ReorderBufferTXN *txn;
 
+       AssertTXNLsnOrder(rb);
+
        if (dlist_is_empty(&rb->toplevel_by_lsn))
                return NULL;
 
-       AssertTXNLsnOrder(rb);
-
        txn = dlist_head_element(ReorderBufferTXN, node, &rb->toplevel_by_lsn);
 
        Assert(!txn->is_known_as_subxact);
@@ -733,12 +691,44 @@ ReorderBufferGetOldestTXN(ReorderBuffer *rb)
        return txn;
 }
 
+/*
+ * ReorderBufferGetOldestXmin
+ *             Return oldest Xmin in reorderbuffer
+ *
+ * Returns oldest possibly running Xid from the point of view of snapshots
+ * used in the transactions kept by reorderbuffer, or InvalidTransactionId if
+ * there are none.
+ *
+ * Since snapshots are assigned monotonically, this equals the Xmin of the
+ * base snapshot with minimal base_snapshot_lsn.
+ */
+TransactionId
+ReorderBufferGetOldestXmin(ReorderBuffer *rb)
+{
+       ReorderBufferTXN *txn;
+
+       AssertTXNLsnOrder(rb);
+
+       if (dlist_is_empty(&rb->txns_by_base_snapshot_lsn))
+               return InvalidTransactionId;
+
+       txn = dlist_head_element(ReorderBufferTXN, base_snapshot_node,
+                                                        &rb->txns_by_base_snapshot_lsn);
+       return txn->base_snapshot->xmin;
+}
+
 void
 ReorderBufferSetRestartPoint(ReorderBuffer *rb, XLogRecPtr ptr)
 {
        rb->current_restart_decoding_lsn = ptr;
 }
 
+/*
+ * ReorderBufferAssignChild
+ *
+ * Make note that we know that subxid is a subtransaction of xid, seen as of
+ * the given lsn.
+ */
 void
 ReorderBufferAssignChild(ReorderBuffer *rb, TransactionId xid,
                                                 TransactionId subxid, XLogRecPtr lsn)
@@ -751,32 +741,107 @@ ReorderBufferAssignChild(ReorderBuffer *rb, TransactionId xid,
        txn = ReorderBufferTXNByXid(rb, xid, true, &new_top, lsn, true);
        subtxn = ReorderBufferTXNByXid(rb, subxid, true, &new_sub, lsn, false);
 
-       if (new_sub)
+       if (new_top && !new_sub)
+               elog(ERROR, "subtransaction logged without previous top-level txn record");
+
+       if (!new_sub)
        {
-               /*
-                * we assign subtransactions to top level transaction even if we don't
-                * have data for it yet, assignment records frequently reference xids
-                * that have not yet produced any records. Knowing those aren't top
-                * level xids allows us to make processing cheaper in some places.
-                */
-               dlist_push_tail(&txn->subtxns, &subtxn->node);
-               txn->nsubtxns++;
+               if (subtxn->is_known_as_subxact)
+               {
+                       /* already associated, nothing to do */
+                       return;
+               }
+               else
+               {
+                       /*
+                        * We already saw this transaction, but initially added it to the
+                        * list of top-level txns.  Now that we know it's not top-level,
+                        * remove it from there.
+                        */
+                       dlist_delete(&subtxn->node);
+               }
        }
-       else if (!subtxn->is_known_as_subxact)
-       {
-               subtxn->is_known_as_subxact = true;
-               Assert(subtxn->nsubtxns == 0);
 
-               /* remove from lsn order list of top-level transactions */
-               dlist_delete(&subtxn->node);
+       subtxn->is_known_as_subxact = true;
+       subtxn->toplevel_xid = xid;
+       Assert(subtxn->nsubtxns == 0);
 
-               /* add to toplevel transaction */
-               dlist_push_tail(&txn->subtxns, &subtxn->node);
-               txn->nsubtxns++;
-       }
-       else if (new_top)
+       /* add to subtransaction list */
+       dlist_push_tail(&txn->subtxns, &subtxn->node);
+       txn->nsubtxns++;
+
+       /* Possibly transfer the subtxn's snapshot to its top-level txn. */
+       ReorderBufferTransferSnapToParent(txn, subtxn);
+
+       /* Verify LSN-ordering invariant */
+       AssertTXNLsnOrder(rb);
+}
+
+/*
+ * ReorderBufferTransferSnapToParent
+ *             Transfer base snapshot from subtxn to top-level txn, if needed
+ *
+ * This is done if the top-level txn doesn't have a base snapshot, or if the
+ * subtxn's base snapshot has an earlier LSN than the top-level txn's base
+ * snapshot's LSN.  This can happen if there are no changes in the toplevel
+ * txn but there are some in the subtxn, or the first change in subtxn has
+ * earlier LSN than first change in the top-level txn and we learned about
+ * their kinship only now.
+ *
+ * The subtransaction's snapshot is cleared regardless of the transfer
+ * happening, since it's not needed anymore in either case.
+ *
+ * We do this as soon as we become aware of their kinship, to avoid queueing
+ * extra snapshots to txns known-as-subtxns -- only top-level txns will
+ * receive further snapshots.
+ */
+static void
+ReorderBufferTransferSnapToParent(ReorderBufferTXN *txn,
+                                                                 ReorderBufferTXN *subtxn)
+{
+       Assert(subtxn->toplevel_xid == txn->xid);
+
+       if (subtxn->base_snapshot != NULL)
        {
-               elog(ERROR, "existing subxact assigned to unknown toplevel xact");
+               if (txn->base_snapshot == NULL ||
+                       subtxn->base_snapshot_lsn < txn->base_snapshot_lsn)
+               {
+                       /*
+                        * If the toplevel transaction already has a base snapshot but
+                        * it's newer than the subxact's, purge it.
+                        */
+                       if (txn->base_snapshot != NULL)
+                       {
+                               SnapBuildSnapDecRefcount(txn->base_snapshot);
+                               dlist_delete(&txn->base_snapshot_node);
+                       }
+
+                       /*
+                        * The snapshot is now the top transaction's; transfer it, and
+                        * adjust the list position of the top transaction in the list by
+                        * moving it to where the subtransaction is.
+                        */
+                       txn->base_snapshot = subtxn->base_snapshot;
+                       txn->base_snapshot_lsn = subtxn->base_snapshot_lsn;
+                       dlist_insert_before(&subtxn->base_snapshot_node,
+                                                               &txn->base_snapshot_node);
+
+                       /*
+                        * The subtransaction doesn't have a snapshot anymore (so it
+                        * mustn't be in the list.)
+                        */
+                       subtxn->base_snapshot = NULL;
+                       subtxn->base_snapshot_lsn = InvalidXLogRecPtr;
+                       dlist_delete(&subtxn->base_snapshot_node);
+               }
+               else
+               {
+                       /* Base snap of toplevel is fine, so subxact's is not needed */
+                       SnapBuildSnapDecRefcount(subtxn->base_snapshot);
+                       dlist_delete(&subtxn->base_snapshot_node);
+                       subtxn->base_snapshot = NULL;
+                       subtxn->base_snapshot_lsn = InvalidXLogRecPtr;
+               }
        }
 }
 
@@ -789,7 +854,6 @@ ReorderBufferCommitChild(ReorderBuffer *rb, TransactionId xid,
                                                 TransactionId subxid, XLogRecPtr commit_lsn,
                                                 XLogRecPtr end_lsn)
 {
-       ReorderBufferTXN *txn;
        ReorderBufferTXN *subtxn;
 
        subtxn = ReorderBufferTXNByXid(rb, subxid, false, NULL,
@@ -801,41 +865,14 @@ ReorderBufferCommitChild(ReorderBuffer *rb, TransactionId xid,
        if (!subtxn)
                return;
 
-       txn = ReorderBufferTXNByXid(rb, xid, false, NULL, commit_lsn, true);
-
-       if (txn == NULL)
-               elog(ERROR, "subxact logged without previous toplevel record");
-
-       /*
-        * Pass the our base snapshot to the parent transaction if it doesn't have
-        * one, or ours is older. That can happen if there are no changes in the
-        * toplevel transaction but in one of the child transactions. This allows
-        * the parent to simply use it's base snapshot initially.
-        */
-       if (txn->base_snapshot == NULL ||
-               txn->base_snapshot_lsn > subtxn->base_snapshot_lsn)
-       {
-               txn->base_snapshot = subtxn->base_snapshot;
-               txn->base_snapshot_lsn = subtxn->base_snapshot_lsn;
-               subtxn->base_snapshot = NULL;
-               subtxn->base_snapshot_lsn = InvalidXLogRecPtr;
-       }
-
        subtxn->final_lsn = commit_lsn;
        subtxn->end_lsn = end_lsn;
 
-       if (!subtxn->is_known_as_subxact)
-       {
-               subtxn->is_known_as_subxact = true;
-               Assert(subtxn->nsubtxns == 0);
-
-               /* remove from lsn order list of top-level transactions */
-               dlist_delete(&subtxn->node);
-
-               /* add to subtransaction list */
-               dlist_push_tail(&txn->subtxns, &subtxn->node);
-               txn->nsubtxns++;
-       }
+       /*
+        * Assign this subxact as a child of the toplevel xact (no-op if already
+        * done.)
+        */
+       ReorderBufferAssignChild(rb, xid, subxid, InvalidXLogRecPtr);
 }
 
 
@@ -935,9 +972,13 @@ ReorderBufferIterTXNInit(ReorderBuffer *rb, ReorderBufferTXN *txn)
        {
                ReorderBufferChange *cur_change;
 
-               if (txn->nentries != txn->nentries_mem)
+               if (txn->serialized)
+               {
+                       /* serialize remaining changes */
+                       ReorderBufferSerializeTXN(rb, txn);
                        ReorderBufferRestoreChanges(rb, txn, &state->entries[off].fd,
                                                                                &state->entries[off].segno);
+               }
 
                cur_change = dlist_head_element(ReorderBufferChange, node,
                                                                                &txn->changes);
@@ -960,11 +1001,14 @@ ReorderBufferIterTXNInit(ReorderBuffer *rb, ReorderBufferTXN *txn)
                {
                        ReorderBufferChange *cur_change;
 
-                       if (txn->nentries != txn->nentries_mem)
+                       if (cur_txn->serialized)
+                       {
+                               /* serialize remaining changes */
+                               ReorderBufferSerializeTXN(rb, cur_txn);
                                ReorderBufferRestoreChanges(rb, cur_txn,
                                                                                        &state->entries[off].fd,
                                                                                        &state->entries[off].segno);
-
+                       }
                        cur_change = dlist_head_element(ReorderBufferChange, node,
                                                                                        &cur_txn->changes);
 
@@ -1152,24 +1196,24 @@ ReorderBufferCleanupTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
                ReorderBufferReturnChange(rb, change);
        }
 
+       /*
+        * Cleanup the base snapshot, if set.
+        */
        if (txn->base_snapshot != NULL)
        {
                SnapBuildSnapDecRefcount(txn->base_snapshot);
-               txn->base_snapshot = NULL;
-               txn->base_snapshot_lsn = InvalidXLogRecPtr;
+               dlist_delete(&txn->base_snapshot_node);
        }
 
-       /* delete from list of known subxacts */
-       if (txn->is_known_as_subxact)
-       {
-               /* NB: nsubxacts count of parent will be too high now */
-               dlist_delete(&txn->node);
-       }
-       /* delete from LSN ordered list of toplevel TXNs */
-       else
-       {
-               dlist_delete(&txn->node);
-       }
+       /*
+        * Remove TXN from its containing list.
+        *
+        * Note: if txn->is_known_as_subxact, we are deleting the TXN from its
+        * parent's list of known subxacts; this leaves the parent's nsubxacts
+        * count too high, but we don't care.  Otherwise, we are deleting the TXN
+        * from the LSN-ordered list of toplevel TXNs.
+        */
+       dlist_delete(&txn->node);
 
        /* now remove reference from buffer */
        hash_search(rb->by_txn,
@@ -1179,7 +1223,7 @@ ReorderBufferCleanupTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
        Assert(found);
 
        /* remove entries spilled to disk */
-       if (txn->nentries != txn->nentries_mem)
+       if (txn->serialized)
                ReorderBufferRestoreCleanup(rb, txn);
 
        /* deallocate */
@@ -1289,7 +1333,7 @@ ReorderBufferCopySnap(ReorderBuffer *rb, Snapshot orig_snap,
 
        /*
         * snap->subxip contains all txids that belong to our transaction which we
-        * need to check via cmin/cmax. Thats why we store the toplevel
+        * need to check via cmin/cmax. That's why we store the toplevel
         * transaction in there as well.
         */
        snap->subxip = snap->xip + snap->xcnt;
@@ -1333,17 +1377,17 @@ ReorderBufferFreeSnap(ReorderBuffer *rb, Snapshot snap)
 }
 
 /*
- * Perform the replay of a transaction and it's non-aborted subtransactions.
+ * Perform the replay of a transaction and its non-aborted subtransactions.
  *
  * Subtransactions previously have to be processed by
  * ReorderBufferCommitChild(), even if previously assigned to the toplevel
  * transaction with ReorderBufferAssignChild.
  *
- * We currently can only decode a transaction's contents in when their commit
- * record is read because that's currently the only place where we know about
- * cache invalidations. Thus, once a toplevel commit is read, we iterate over
- * the top and subtransactions (using a k-way merge) and replay the changes in
- * lsn order.
+ * We currently can only decode a transaction's contents when its commit
+ * record is read because that's the only place where we know about cache
+ * invalidations. Thus, once a toplevel commit is read, we iterate over the top
+ * and subtransactions (using a k-way merge) and replay the changes in lsn
+ * order.
  */
 void
 ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
@@ -1370,15 +1414,11 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
        txn->origin_id = origin_id;
        txn->origin_lsn = origin_lsn;
 
-       /* serialize the last bunch of changes if we need start earlier anyway */
-       if (txn->nentries_mem != txn->nentries)
-               ReorderBufferSerializeTXN(rb, txn);
-
        /*
-        * If this transaction didn't have any real changes in our database, it's
-        * OK not to have a snapshot. Note that ReorderBufferCommitChild will have
-        * transferred its snapshot to this transaction if it had one and the
-        * toplevel tx didn't.
+        * If this transaction has no snapshot, it didn't make any changes to the
+        * database, so there's nothing to decode.  Note that
+        * ReorderBufferCommitChild will have transferred any snapshots from
+        * subtransactions if there were any.
         */
        if (txn->base_snapshot == NULL)
        {
@@ -1434,6 +1474,8 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
                                         * use as a normal record. It'll be cleaned up at the end
                                         * of INSERT processing.
                                         */
+                                       if (specinsert == NULL)
+                                               elog(ERROR, "invalid ordering of speculative insertion changes");
                                        Assert(specinsert->data.tp.oldtuple == NULL);
                                        change = specinsert;
                                        change->action = REORDER_BUFFER_CHANGE_INSERT;
@@ -1445,7 +1487,7 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
                                        Assert(snapshot_now);
 
                                        reloid = RelidByRelfilenode(change->data.tp.relnode.spcNode,
-                                                                                       change->data.tp.relnode.relNode);
+                                                                                               change->data.tp.relnode.relNode);
 
                                        /*
                                         * Catalog tuple without data, emitted while catalog was
@@ -1471,6 +1513,13 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
                                        if (!RelationIsLogicallyLogged(relation))
                                                goto change_done;
 
+                                       /*
+                                        * Ignore temporary heaps created during DDL unless the
+                                        * plugin has asked for them.
+                                        */
+                                       if (relation->rd_rel->relrewrite && !rb->output_rewrites)
+                                               goto change_done;
+
                                        /*
                                         * For now ignore sequence changes entirely. Most of the
                                         * time they don't log changes using records we
@@ -1556,6 +1605,38 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
                                        specinsert = change;
                                        break;
 
+                               case REORDER_BUFFER_CHANGE_TRUNCATE:
+                                       {
+                                               int                     i;
+                                               int                     nrelids = change->data.truncate.nrelids;
+                                               int                     nrelations = 0;
+                                               Relation   *relations;
+
+                                               relations = palloc0(nrelids * sizeof(Relation));
+                                               for (i = 0; i < nrelids; i++)
+                                               {
+                                                       Oid                     relid = change->data.truncate.relids[i];
+                                                       Relation        relation;
+
+                                                       relation = RelationIdGetRelation(relid);
+
+                                                       if (relation == NULL)
+                                                               elog(ERROR, "could not open relation with OID %u", relid);
+
+                                                       if (!RelationIsLogicallyLogged(relation))
+                                                               continue;
+
+                                                       relations[nrelations++] = relation;
+                                               }
+
+                                               rb->apply_truncate(rb, txn, nrelations, relations, change);
+
+                                               for (i = 0; i < nrelations; i++)
+                                                       RelationClose(relations[i]);
+
+                                               break;
+                                       }
+
                                case REORDER_BUFFER_CHANGE_MESSAGE:
                                        rb->message(rb, txn, change->lsn, true,
                                                                change->data.msg.prefix,
@@ -1607,7 +1688,7 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
                                                {
                                                        /* we don't use the global one anymore */
                                                        snapshot_now = ReorderBufferCopySnap(rb, snapshot_now,
-                                                                                                                       txn, command_id);
+                                                                                                                                txn, command_id);
                                                }
 
                                                snapshot_now->curcid = command_id;
@@ -1714,7 +1795,7 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
  *
  * NB: Transactions handled here have to have actively aborted (i.e. have
  * produced an abort record). Implicitly aborted transactions are handled via
- * ReorderBufferAbortOld(); transactions we're just not interesteded in, but
+ * ReorderBufferAbortOld(); transactions we're just not interested in, but
  * which have committed are handled in ReorderBufferForget().
  *
  * This function purges this transaction and its contents from memory and
@@ -1755,8 +1836,8 @@ ReorderBufferAbortOld(ReorderBuffer *rb, TransactionId oldestRunningXid)
         * Iterate through all (potential) toplevel TXNs and abort all that are
         * older than what possibly can be running. Once we've found the first
         * that is alive we stop, there might be some that acquired an xid earlier
-        * but started writing later, but it's unlikely and they will cleaned up
-        * in a later call to ReorderBufferAbortOld().
+        * but started writing later, but it's unlikely and they will be cleaned
+        * up in a later call to this function.
         */
        dlist_foreach_modify(it, &rb->toplevel_by_lsn)
        {
@@ -1766,7 +1847,22 @@ ReorderBufferAbortOld(ReorderBuffer *rb, TransactionId oldestRunningXid)
 
                if (TransactionIdPrecedes(txn->xid, oldestRunningXid))
                {
-                       elog(DEBUG1, "aborting old transaction %u", txn->xid);
+                       /*
+                        * We set final_lsn on a transaction when we decode its commit or
+                        * abort record, but we never see those records for crashed
+                        * transactions.  To ensure cleanup of these transactions, set
+                        * final_lsn to that of their last change; this causes
+                        * ReorderBufferRestoreCleanup to do the right thing.
+                        */
+                       if (txn->serialized && txn->final_lsn == 0)
+                       {
+                               ReorderBufferChange *last =
+                               dlist_tail_element(ReorderBufferChange, node, &txn->changes);
+
+                               txn->final_lsn = last->lsn;
+                       }
+
+                       elog(DEBUG2, "aborting old transaction %u", txn->xid);
 
                        /* remove potential on-disk data, and deallocate this tx */
                        ReorderBufferCleanupTXN(rb, txn);
@@ -1782,7 +1878,7 @@ ReorderBufferAbortOld(ReorderBuffer *rb, TransactionId oldestRunningXid)
  * toplevel xid.
  *
  * This is significantly different to ReorderBufferAbort() because
- * transactions that have committed need to be treated differenly from aborted
+ * transactions that have committed need to be treated differently from aborted
  * ones since they may have modified the catalog.
  *
  * Note that this is only allowed to be called in the moment a transaction
@@ -1810,26 +1906,8 @@ ReorderBufferForget(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn)
         * catalog and we need to update the caches according to that.
         */
        if (txn->base_snapshot != NULL && txn->ninvalidations > 0)
-       {
-               bool            use_subtxn = IsTransactionOrTransactionBlock();
-
-               if (use_subtxn)
-                       BeginInternalSubTransaction("replay");
-
-               /*
-                * Force invalidations to happen outside of a valid transaction - that
-                * way entries will just be marked as invalid without accessing the
-                * catalog. That's advantageous because we don't need to setup the
-                * full state necessary for catalog access.
-                */
-               if (use_subtxn)
-                       AbortCurrentTransaction();
-
-               ReorderBufferExecuteInvalidations(rb, txn);
-
-               if (use_subtxn)
-                       RollbackAndReleaseCurrentSubTransaction();
-       }
+               ReorderBufferImmediateInvalidation(rb, txn->ninvalidations,
+                                                                                  txn->invalidations);
        else
                Assert(txn->ninvalidations == 0);
 
@@ -1837,6 +1915,37 @@ ReorderBufferForget(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn)
        ReorderBufferCleanupTXN(rb, txn);
 }
 
+/*
+ * Execute invalidations happening outside the context of a decoded
+ * transaction. That currently happens either for xid-less commits
+ * (cf. RecordTransactionCommit()) or for invalidations in uninteresting
+ * transactions (via ReorderBufferForget()).
+ */
+void
+ReorderBufferImmediateInvalidation(ReorderBuffer *rb, uint32 ninvalidations,
+                                                                  SharedInvalidationMessage *invalidations)
+{
+       bool            use_subtxn = IsTransactionOrTransactionBlock();
+       int                     i;
+
+       if (use_subtxn)
+               BeginInternalSubTransaction("replay");
+
+       /*
+        * Force invalidations to happen outside of a valid transaction - that way
+        * entries will just be marked as invalid without accessing the catalog.
+        * That's advantageous because we don't need to setup the full state
+        * necessary for catalog access.
+        */
+       if (use_subtxn)
+               AbortCurrentTransaction();
+
+       for (i = 0; i < ninvalidations; i++)
+               LocalExecuteInvalidationMessage(&invalidations[i]);
+
+       if (use_subtxn)
+               RollbackAndReleaseCurrentSubTransaction();
+}
 
 /*
  * Tell reorderbuffer about an xid seen in the WAL stream. Has to be called at
@@ -1874,12 +1983,10 @@ ReorderBufferAddSnapshot(ReorderBuffer *rb, TransactionId xid,
 }
 
 /*
- * Setup the base snapshot of a transaction. The base snapshot is the snapshot
- * that is used to decode all changes until either this transaction modifies
- * the catalog or another catalog modifying transaction commits.
+ * Set up the transaction's base snapshot.
  *
- * Needs to be called before any changes are added with
- * ReorderBufferQueueChange().
+ * If we know that xid is a subtransaction, set the base snapshot on the
+ * top-level transaction instead.
  */
 void
 ReorderBufferSetBaseSnapshot(ReorderBuffer *rb, TransactionId xid,
@@ -1888,12 +1995,23 @@ ReorderBufferSetBaseSnapshot(ReorderBuffer *rb, TransactionId xid,
        ReorderBufferTXN *txn;
        bool            is_new;
 
+       AssertArg(snap != NULL);
+
+       /*
+        * Fetch the transaction to operate on.  If we know it's a subtransaction,
+        * operate on its top-level transaction instead.
+        */
        txn = ReorderBufferTXNByXid(rb, xid, true, &is_new, lsn, true);
+       if (txn->is_known_as_subxact)
+               txn = ReorderBufferTXNByXid(rb, txn->toplevel_xid, false,
+                                                                       NULL, InvalidXLogRecPtr, false);
        Assert(txn->base_snapshot == NULL);
-       Assert(snap != NULL);
 
        txn->base_snapshot = snap;
        txn->base_snapshot_lsn = lsn;
+       dlist_push_tail(&rb->txns_by_base_snapshot_lsn, &txn->base_snapshot_node);
+
+       AssertTXNLsnOrder(rb);
 }
 
 /*
@@ -2012,25 +2130,26 @@ ReorderBufferXidHasCatalogChanges(ReorderBuffer *rb, TransactionId xid)
 }
 
 /*
- * Have we already added the first snapshot?
+ * ReorderBufferXidHasBaseSnapshot
+ *             Have we already set the base snapshot for the given txn/subtxn?
  */
 bool
 ReorderBufferXidHasBaseSnapshot(ReorderBuffer *rb, TransactionId xid)
 {
        ReorderBufferTXN *txn;
 
-       txn = ReorderBufferTXNByXid(rb, xid, false, NULL, InvalidXLogRecPtr,
-                                                               false);
+       txn = ReorderBufferTXNByXid(rb, xid, false,
+                                                               NULL, InvalidXLogRecPtr, false);
 
        /* transaction isn't known yet, ergo no snapshot */
        if (txn == NULL)
                return false;
 
-       /*
-        * TODO: It would be a nice improvement if we would check the toplevel
-        * transaction in subtransactions, but we'd need to keep track of a bit
-        * more state.
-        */
+       /* a known subtxn? operate on top-level txn instead */
+       if (txn->is_known_as_subxact)
+               txn = ReorderBufferTXNByXid(rb, txn->toplevel_xid, false,
+                                                                       NULL, InvalidXLogRecPtr, false);
+
        return txn->base_snapshot != NULL;
 }
 
@@ -2087,7 +2206,6 @@ ReorderBufferSerializeTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
        int                     fd = -1;
        XLogSegNo       curOpenSegNo = 0;
        Size            spilled = 0;
-       char            path[MAXPGPATH];
 
        elog(DEBUG2, "spill %u changes in XID %u to disk",
                 (uint32) txn->nentries_mem, txn->xid);
@@ -2112,34 +2230,31 @@ ReorderBufferSerializeTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
                 * store in segment in which it belongs by start lsn, don't split over
                 * multiple segments tho
                 */
-               if (fd == -1 || !XLByteInSeg(change->lsn, curOpenSegNo))
+               if (fd == -1 ||
+                       !XLByteInSeg(change->lsn, curOpenSegNo, wal_segment_size))
                {
-                       XLogRecPtr      recptr;
+                       char            path[MAXPGPATH];
 
                        if (fd != -1)
                                CloseTransientFile(fd);
 
-                       XLByteToSeg(change->lsn, curOpenSegNo);
-                       XLogSegNoOffsetToRecPtr(curOpenSegNo, 0, recptr);
+                       XLByteToSeg(change->lsn, curOpenSegNo, wal_segment_size);
 
                        /*
                         * No need to care about TLIs here, only used during a single run,
                         * so each LSN only maps to a specific WAL record.
                         */
-                       sprintf(path, "pg_replslot/%s/xid-%u-lsn-%X-%X.snap",
-                                       NameStr(MyReplicationSlot->data.name), txn->xid,
-                                       (uint32) (recptr >> 32), (uint32) recptr);
+                       ReorderBufferSerializedPath(path, MyReplicationSlot, txn->xid,
+                                                                               curOpenSegNo);
 
                        /* open segment, create it if necessary */
                        fd = OpenTransientFile(path,
-                                                                  O_CREAT | O_WRONLY | O_APPEND | PG_BINARY,
-                                                                  S_IRUSR | S_IWUSR);
+                                                                  O_CREAT | O_WRONLY | O_APPEND | PG_BINARY);
 
                        if (fd < 0)
                                ereport(ERROR,
                                                (errcode_for_file_access(),
-                                                errmsg("could not open file \"%s\": %m",
-                                                               path)));
+                                                errmsg("could not open file \"%s\": %m", path)));
                }
 
                ReorderBufferSerializeChange(rb, txn, fd, change);
@@ -2152,6 +2267,7 @@ ReorderBufferSerializeTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
        Assert(spilled == txn->nentries_mem);
        Assert(dlist_is_empty(&txn->changes));
        txn->nentries_mem = 0;
+       txn->serialized = true;
 
        if (fd != -1)
                CloseTransientFile(fd);
@@ -2240,6 +2356,9 @@ ReorderBufferSerializeChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
 
                                data = ((char *) rb->outbuf) + sizeof(ReorderBufferDiskChange);
 
+                               /* might have been reallocated above */
+                               ondisk = (ReorderBufferDiskChange *) rb->outbuf;
+
                                /* write the prefix including the size */
                                memcpy(data, &prefix_size, sizeof(Size));
                                data += sizeof(Size);
@@ -2292,6 +2411,7 @@ ReorderBufferSerializeChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
                                }
                                break;
                        }
+               case REORDER_BUFFER_CHANGE_TRUNCATE:
                case REORDER_BUFFER_CHANGE_INTERNAL_SPEC_CONFIRM:
                case REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID:
                case REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID:
@@ -2301,14 +2421,22 @@ ReorderBufferSerializeChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
 
        ondisk->size = sz;
 
+       errno = 0;
+       pgstat_report_wait_start(WAIT_EVENT_REORDER_BUFFER_WRITE);
        if (write(fd, rb->outbuf, ondisk->size) != ondisk->size)
        {
+               int                     save_errno = errno;
+
                CloseTransientFile(fd);
+
+               /* if write didn't set errno, assume problem is no disk space */
+               errno = save_errno ? save_errno : ENOSPC;
                ereport(ERROR,
                                (errcode_for_file_access(),
                                 errmsg("could not write to data file for XID %u: %m",
                                                txn->xid)));
        }
+       pgstat_report_wait_end();
 
        Assert(ondisk->change.action == change->action);
 }
@@ -2339,7 +2467,7 @@ ReorderBufferRestoreChanges(ReorderBuffer *rb, ReorderBufferTXN *txn,
        txn->nentries_mem = 0;
        Assert(dlist_is_empty(&txn->changes));
 
-       XLByteToSeg(txn->final_lsn, last_segno);
+       XLByteToSeg(txn->final_lsn, last_segno, wal_segment_size);
 
        while (restored < max_changes_in_memory && *segno <= last_segno)
        {
@@ -2348,27 +2476,22 @@ ReorderBufferRestoreChanges(ReorderBuffer *rb, ReorderBufferTXN *txn,
 
                if (*fd == -1)
                {
-                       XLogRecPtr      recptr;
                        char            path[MAXPGPATH];
 
                        /* first time in */
                        if (*segno == 0)
-                       {
-                               XLByteToSeg(txn->first_lsn, *segno);
-                       }
+                               XLByteToSeg(txn->first_lsn, *segno, wal_segment_size);
 
                        Assert(*segno != 0 || dlist_is_empty(&txn->changes));
-                       XLogSegNoOffsetToRecPtr(*segno, 0, recptr);
 
                        /*
                         * No need to care about TLIs here, only used during a single run,
                         * so each LSN only maps to a specific WAL record.
                         */
-                       sprintf(path, "pg_replslot/%s/xid-%u-lsn-%X-%X.snap",
-                                       NameStr(MyReplicationSlot->data.name), txn->xid,
-                                       (uint32) (recptr >> 32), (uint32) recptr);
+                       ReorderBufferSerializedPath(path, MyReplicationSlot, txn->xid,
+                                                                               *segno);
 
-                       *fd = OpenTransientFile(path, O_RDONLY | PG_BINARY, 0);
+                       *fd = OpenTransientFile(path, O_RDONLY | PG_BINARY);
                        if (*fd < 0 && errno == ENOENT)
                        {
                                *fd = -1;
@@ -2380,7 +2503,6 @@ ReorderBufferRestoreChanges(ReorderBuffer *rb, ReorderBufferTXN *txn,
                                                (errcode_for_file_access(),
                                                 errmsg("could not open file \"%s\": %m",
                                                                path)));
-
                }
 
                /*
@@ -2389,7 +2511,9 @@ ReorderBufferRestoreChanges(ReorderBuffer *rb, ReorderBufferTXN *txn,
                 * end of this file.
                 */
                ReorderBufferSerializeReserve(rb, sizeof(ReorderBufferDiskChange));
+               pgstat_report_wait_start(WAIT_EVENT_REORDER_BUFFER_READ);
                readBytes = read(*fd, rb->outbuf, sizeof(ReorderBufferDiskChange));
+               pgstat_report_wait_end();
 
                /* eof */
                if (readBytes == 0)
@@ -2402,7 +2526,7 @@ ReorderBufferRestoreChanges(ReorderBuffer *rb, ReorderBufferTXN *txn,
                else if (readBytes < 0)
                        ereport(ERROR,
                                        (errcode_for_file_access(),
-                               errmsg("could not read from reorderbuffer spill file: %m")));
+                                        errmsg("could not read from reorderbuffer spill file: %m")));
                else if (readBytes != sizeof(ReorderBufferDiskChange))
                        ereport(ERROR,
                                        (errcode_for_file_access(),
@@ -2413,22 +2537,24 @@ ReorderBufferRestoreChanges(ReorderBuffer *rb, ReorderBufferTXN *txn,
                ondisk = (ReorderBufferDiskChange *) rb->outbuf;
 
                ReorderBufferSerializeReserve(rb,
-                                                        sizeof(ReorderBufferDiskChange) + ondisk->size);
+                                                                         sizeof(ReorderBufferDiskChange) + ondisk->size);
                ondisk = (ReorderBufferDiskChange *) rb->outbuf;
 
+               pgstat_report_wait_start(WAIT_EVENT_REORDER_BUFFER_READ);
                readBytes = read(*fd, rb->outbuf + sizeof(ReorderBufferDiskChange),
                                                 ondisk->size - sizeof(ReorderBufferDiskChange));
+               pgstat_report_wait_end();
 
                if (readBytes < 0)
                        ereport(ERROR,
                                        (errcode_for_file_access(),
-                               errmsg("could not read from reorderbuffer spill file: %m")));
+                                        errmsg("could not read from reorderbuffer spill file: %m")));
                else if (readBytes != ondisk->size - sizeof(ReorderBufferDiskChange))
                        ereport(ERROR,
                                        (errcode_for_file_access(),
                                         errmsg("could not read from reorderbuffer spill file: read %d instead of %u bytes",
                                                        readBytes,
-                               (uint32) (ondisk->size - sizeof(ReorderBufferDiskChange)))));
+                                                       (uint32) (ondisk->size - sizeof(ReorderBufferDiskChange)))));
 
                /*
                 * ok, read a full change from disk, now restore it into proper
@@ -2530,14 +2656,14 @@ ReorderBufferRestoreChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
                                change->data.msg.prefix = MemoryContextAlloc(rb->context,
                                                                                                                         prefix_size);
                                memcpy(change->data.msg.prefix, data, prefix_size);
-                               Assert(change->data.msg.prefix[prefix_size-1] == '\0');
+                               Assert(change->data.msg.prefix[prefix_size - 1] == '\0');
                                data += prefix_size;
 
-                               /* read the messsage */
+                               /* read the message */
                                memcpy(&change->data.msg.message_size, data, sizeof(Size));
                                data += sizeof(Size);
                                change->data.msg.message = MemoryContextAlloc(rb->context,
-                                                                                               change->data.msg.message_size);
+                                                                                                                         change->data.msg.message_size);
                                memcpy(change->data.msg.message, data,
                                           change->data.msg.message_size);
                                data += change->data.msg.message_size;
@@ -2568,6 +2694,7 @@ ReorderBufferRestoreChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
                                break;
                        }
                        /* the base struct contains all the data, easy peasy */
+               case REORDER_BUFFER_CHANGE_TRUNCATE:
                case REORDER_BUFFER_CHANGE_INTERNAL_SPEC_CONFIRM:
                case REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID:
                case REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID:
@@ -2591,20 +2718,15 @@ ReorderBufferRestoreCleanup(ReorderBuffer *rb, ReorderBufferTXN *txn)
        Assert(txn->first_lsn != InvalidXLogRecPtr);
        Assert(txn->final_lsn != InvalidXLogRecPtr);
 
-       XLByteToSeg(txn->first_lsn, first);
-       XLByteToSeg(txn->final_lsn, last);
+       XLByteToSeg(txn->first_lsn, first, wal_segment_size);
+       XLByteToSeg(txn->final_lsn, last, wal_segment_size);
 
        /* iterate over all possible filenames, and delete them */
        for (cur = first; cur <= last; cur++)
        {
                char            path[MAXPGPATH];
-               XLogRecPtr      recptr;
 
-               XLogSegNoOffsetToRecPtr(cur, 0, recptr);
-
-               sprintf(path, "pg_replslot/%s/xid-%u-lsn-%X-%X.snap",
-                               NameStr(MyReplicationSlot->data.name), txn->xid,
-                               (uint32) (recptr >> 32), (uint32) recptr);
+               ReorderBufferSerializedPath(path, MyReplicationSlot, txn->xid, cur);
                if (unlink(path) != 0 && errno != ENOENT)
                        ereport(ERROR,
                                        (errcode_for_file_access(),
@@ -2612,6 +2734,63 @@ ReorderBufferRestoreCleanup(ReorderBuffer *rb, ReorderBufferTXN *txn)
        }
 }
 
+/*
+ * Remove any leftover serialized reorder buffers from a slot directory after a
+ * prior crash or decoding session exit.
+ */
+static void
+ReorderBufferCleanupSerializedTXNs(const char *slotname)
+{
+       DIR                *spill_dir;
+       struct dirent *spill_de;
+       struct stat statbuf;
+       char            path[MAXPGPATH * 2 + 12];
+
+       sprintf(path, "pg_replslot/%s", slotname);
+
+       /* we're only handling directories here, skip if it's not ours */
+       if (lstat(path, &statbuf) == 0 && !S_ISDIR(statbuf.st_mode))
+               return;
+
+       spill_dir = AllocateDir(path);
+       while ((spill_de = ReadDirExtended(spill_dir, path, INFO)) != NULL)
+       {
+               /* only look at names that can be ours */
+               if (strncmp(spill_de->d_name, "xid", 3) == 0)
+               {
+                       snprintf(path, sizeof(path),
+                                        "pg_replslot/%s/%s", slotname,
+                                        spill_de->d_name);
+
+                       if (unlink(path) != 0)
+                               ereport(ERROR,
+                                               (errcode_for_file_access(),
+                                                errmsg("could not remove file \"%s\" during removal of pg_replslot/%s/xid*: %m",
+                                                               path, slotname)));
+               }
+       }
+       FreeDir(spill_dir);
+}
+
+/*
+ * Given a replication slot, transaction ID and segment number, fill in the
+ * corresponding spill file into 'path', which is a caller-owned buffer of size
+ * at least MAXPGPATH.
+ */
+static void
+ReorderBufferSerializedPath(char *path, ReplicationSlot *slot, TransactionId xid,
+                                                       XLogSegNo segno)
+{
+       XLogRecPtr      recptr;
+
+       XLogSegNoOffsetToRecPtr(segno, 0, wal_segment_size, recptr);
+
+       snprintf(path, MAXPGPATH, "pg_replslot/%s/xid-%u-lsn-%X-%X.tmp",
+                        NameStr(MyReplicationSlot->data.name),
+                        xid,
+                        (uint32) (recptr >> 32), (uint32) recptr);
+}
+
 /*
  * Delete all data spilled to disk after we've restarted/crashed. It will be
  * recreated when the respective slots are reused.
@@ -2622,15 +2801,9 @@ StartupReorderBuffer(void)
        DIR                *logical_dir;
        struct dirent *logical_de;
 
-       DIR                *spill_dir;
-       struct dirent *spill_de;
-
        logical_dir = AllocateDir("pg_replslot");
        while ((logical_de = ReadDir(logical_dir, "pg_replslot")) != NULL)
        {
-               struct stat statbuf;
-               char            path[MAXPGPATH];
-
                if (strcmp(logical_de->d_name, ".") == 0 ||
                        strcmp(logical_de->d_name, "..") == 0)
                        continue;
@@ -2641,35 +2814,9 @@ StartupReorderBuffer(void)
 
                /*
                 * ok, has to be a surviving logical slot, iterate and delete
-                * everythign starting with xid-*
+                * everything starting with xid-*
                 */
-               sprintf(path, "pg_replslot/%s", logical_de->d_name);
-
-               /* we're only creating directories here, skip if it's not our's */
-               if (lstat(path, &statbuf) == 0 && !S_ISDIR(statbuf.st_mode))
-                       continue;
-
-               spill_dir = AllocateDir(path);
-               while ((spill_de = ReadDir(spill_dir, path)) != NULL)
-               {
-                       if (strcmp(spill_de->d_name, ".") == 0 ||
-                               strcmp(spill_de->d_name, "..") == 0)
-                               continue;
-
-                       /* only look at names that can be ours */
-                       if (strncmp(spill_de->d_name, "xid", 3) == 0)
-                       {
-                               sprintf(path, "pg_replslot/%s/%s", logical_de->d_name,
-                                               spill_de->d_name);
-
-                               if (unlink(path) != 0)
-                                       ereport(PANIC,
-                                                       (errcode_for_file_access(),
-                                                        errmsg("could not remove file \"%s\": %m",
-                                                                       path)));
-                       }
-               }
-               FreeDir(spill_dir);
+               ReorderBufferCleanupSerializedTXNs(logical_de->d_name);
        }
        FreeDir(logical_dir);
 }
@@ -2816,7 +2963,7 @@ ReorderBufferToastReplace(ReorderBuffer *rb, ReorderBufferTXN *txn,
 
        for (natt = 0; natt < desc->natts; natt++)
        {
-               Form_pg_attribute attr = desc->attrs[natt];
+               Form_pg_attribute attr = TupleDescAttr(desc, natt);
                ReorderBufferToastEnt *ent;
                struct varlena *varlena;
 
@@ -2883,7 +3030,7 @@ ReorderBufferToastReplace(ReorderBuffer *rb, ReorderBufferTXN *txn,
                        cchange = dlist_container(ReorderBufferChange, node, it.cur);
                        ctup = cchange->data.tp.newtuple;
                        chunk = DatumGetPointer(
-                                                 fastgetattr(&ctup->tuple, 3, toast_desc, &isnull));
+                                                                       fastgetattr(&ctup->tuple, 3, toast_desc, &isnull));
 
                        Assert(!isnull);
                        Assert(!VARATT_IS_EXTERNAL(chunk));
@@ -3029,8 +3176,8 @@ DisplayMapping(HTAB *tuplecid_data)
                         ent->key.relnode.dbNode,
                         ent->key.relnode.spcNode,
                         ent->key.relnode.relNode,
-                        BlockIdGetBlockNumber(&ent->key.tid.ip_blkid),
-                        ent->key.tid.ip_posid,
+                        ItemPointerGetBlockNumber(&ent->key.tid),
+                        ItemPointerGetOffsetNumber(&ent->key.tid),
                         ent->cmin,
                         ent->cmax
                        );
@@ -3053,10 +3200,11 @@ ApplyLogicalMappingFile(HTAB *tuplecid_data, Oid relid, const char *fname)
        LogicalRewriteMappingData map;
 
        sprintf(path, "pg_logical/mappings/%s", fname);
-       fd = OpenTransientFile(path, O_RDONLY | PG_BINARY, 0);
+       fd = OpenTransientFile(path, O_RDONLY | PG_BINARY);
        if (fd < 0)
                ereport(ERROR,
-                               (errmsg("could not open file \"%s\": %m", path)));
+                               (errcode_for_file_access(),
+                                errmsg("could not open file \"%s\": %m", path)));
 
        while (true)
        {
@@ -3069,7 +3217,9 @@ ApplyLogicalMappingFile(HTAB *tuplecid_data, Oid relid, const char *fname)
                memset(&key, 0, sizeof(ReorderBufferTupleCidKey));
 
                /* read all mappings till the end of the file */
+               pgstat_report_wait_start(WAIT_EVENT_REORDER_LOGICAL_MAPPING_READ);
                readBytes = read(fd, &map, sizeof(LogicalRewriteMappingData));
+               pgstat_report_wait_end();
 
                if (readBytes < 0)
                        ereport(ERROR,
@@ -3128,11 +3278,13 @@ ApplyLogicalMappingFile(HTAB *tuplecid_data, Oid relid, const char *fname)
                        new_ent->combocid = ent->combocid;
                }
        }
+
+       CloseTransientFile(fd);
 }
 
 
 /*
- * Check whether the TransactionOId 'xid' is in the pre-sorted array 'xip'.
+ * Check whether the TransactionOid 'xid' is in the pre-sorted array 'xip'.
  */
 static bool
 TransactionIdInArray(TransactionId xid, TransactionId *xip, Size num)