]> 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 cd132c16adf7872ed1803ab38e9483ad4159fd0e..0d72ac1c791f50a289a06ca432476fda365662a1 100644 (file)
@@ -4,7 +4,7 @@
  *       PostgreSQL logical replay/reorder buffer management
  *
  *
- * Copyright (c) 2012-2014, 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"
@@ -69,7 +76,7 @@
 #include "utils/combocid.h"
 #include "utils/memdebug.h"
 #include "utils/memutils.h"
-#include "utils/relcache.h"
+#include "utils/rel.h"
 #include "utils/relfilenodemap.h"
 #include "utils/tqual.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 */
@@ -143,24 +150,12 @@ typedef struct ReorderBufferDiskChange
  * without hitting disk in OLTP workloads, while starting to spool to disk in
  * other workloads reasonably fast.
  *
- * At some point in the future it probaly makes sense to have a more elaborate
+ * At some point in the future it probably makes sense to have a more elaborate
  * resource management here, but it's not entirely clear what that would look
  * like.
  */
 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,30 +243,44 @@ 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.hash = tag_hash;
        hash_ctl.hcxt = buffer->context;
 
        buffer->by_txn = hash_create("ReorderBufferByXid", 1000, &hash_ctl,
-                                                                HASH_ELEM | HASH_FUNCTION | HASH_CONTEXT);
+                                                                HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
 
        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;
 }
@@ -284,29 +298,21 @@ ReorderBufferFree(ReorderBuffer *rb)
         * memory context.
         */
        MemoryContextDelete(context);
+
+       /* Free disk space used by unconsumed reorder buffers */
+       ReorderBufferCleanupSerializedTXNs(NameStr(MyReplicationSlot->data.name));
 }
 
 /*
- * Get a unused, possibly preallocated, ReorderBufferTXN.
+ * Get an unused, possibly preallocated, ReorderBufferTXN.
  */
 static ReorderBufferTXN *
 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));
 
@@ -319,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)
@@ -347,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 a 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;
@@ -389,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)
@@ -402,6 +380,7 @@ ReorderBufferReturnChange(ReorderBuffer *rb, ReorderBufferChange *change)
                case REORDER_BUFFER_CHANGE_INSERT:
                case REORDER_BUFFER_CHANGE_UPDATE:
                case REORDER_BUFFER_CHANGE_DELETE:
+               case REORDER_BUFFER_CHANGE_INTERNAL_SPEC_INSERT:
                        if (change->data.tp.newtuple)
                        {
                                ReorderBufferReturnTupleBuf(rb, change->data.tp.newtuple);
@@ -414,6 +393,14 @@ ReorderBufferReturnChange(ReorderBuffer *rb, ReorderBufferChange *change)
                                change->data.tp.oldtuple = NULL;
                        }
                        break;
+               case REORDER_BUFFER_CHANGE_MESSAGE:
+                       if (change->data.msg.prefix != NULL)
+                               pfree(change->data.msg.prefix);
+                       change->data.msg.prefix = NULL;
+                       if (change->data.msg.message != NULL)
+                               pfree(change->data.msg.message);
+                       change->data.msg.message = NULL;
+                       break;
                case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
                        if (change->data.snapshot)
                        {
@@ -421,75 +408,46 @@ ReorderBufferReturnChange(ReorderBuffer *rb, ReorderBufferChange *change)
                                change->data.snapshot = NULL;
                        }
                        break;
+                       /* no data in addition to the struct itself */
+               case REORDER_BUFFER_CHANGE_INTERNAL_SPEC_CONFIRM:
                case REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID:
-                       break;
                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 a unused, possibly preallocated, ReorderBufferTupleBuf
+ * Get a fresh ReorderBufferTupleBuf fitting at least a tuple of size
+ * tuple_len (excluding header overhead).
  */
 ReorderBufferTupleBuf *
-ReorderBufferGetTupleBuf(ReorderBuffer *rb)
+ReorderBufferGetTupleBuf(ReorderBuffer *rb, Size tuple_len)
 {
        ReorderBufferTupleBuf *tuple;
+       Size            alloc_len;
 
-       /* check the slab cache */
-       if (rb->nr_cached_tuplebufs)
-       {
-               rb->nr_cached_tuplebufs--;
-               tuple = slist_container(ReorderBufferTupleBuf, node,
-                                                               slist_pop_head_node(&rb->cached_tuplebufs));
-#ifdef USE_ASSERT_CHECKING
-               memset(tuple, 0xa9, sizeof(ReorderBufferTupleBuf));
-#endif
-       }
-       else
-       {
-               tuple = (ReorderBufferTupleBuf *)
-                       MemoryContextAlloc(rb->context, sizeof(ReorderBufferTupleBuf));
-       }
+       alloc_len = tuple_len + SizeofHeapTupleHeader;
+
+       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 */
-       if (rb->nr_cached_tuplebufs < max_cached_tuplebufs)
-       {
-               rb->nr_cached_tuplebufs++;
-               slist_push_head(&rb->cached_tuplebufs, &tuple->node);
-               VALGRIND_MAKE_MEM_UNDEFINED(tuple, sizeof(ReorderBufferTupleBuf));
-               VALGRIND_MAKE_MEM_DEFINED(&tuple->node, sizeof(tuple->node));
-       }
-       else
-       {
-               pfree(tuple);
-       }
+       pfree(tuple);
 }
 
 /*
@@ -507,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
@@ -526,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)
@@ -551,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;
@@ -574,7 +532,7 @@ ReorderBufferTXNByXid(ReorderBuffer *rb, TransactionId xid, bool create,
        if (is_new)
                *is_new = !found;
 
-       Assert(!create || !!txn);
+       Assert(!create || txn != NULL);
        return txn;
 }
 
@@ -598,42 +556,134 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
        ReorderBufferCheckSerializeTXN(rb, txn);
 }
 
+/*
+ * Queue message into a transaction so it can be processed upon commit.
+ */
+void
+ReorderBufferQueueMessage(ReorderBuffer *rb, TransactionId xid,
+                                                 Snapshot snapshot, XLogRecPtr lsn,
+                                                 bool transactional, const char *prefix,
+                                                 Size message_size, const char *message)
+{
+       if (transactional)
+       {
+               MemoryContext oldcontext;
+               ReorderBufferChange *change;
+
+               Assert(xid != InvalidTransactionId);
+
+               oldcontext = MemoryContextSwitchTo(rb->context);
+
+               change = ReorderBufferGetChange(rb);
+               change->action = REORDER_BUFFER_CHANGE_MESSAGE;
+               change->data.msg.prefix = pstrdup(prefix);
+               change->data.msg.message_size = message_size;
+               change->data.msg.message = palloc(message_size);
+               memcpy(change->data.msg.message, message, message_size);
+
+               ReorderBufferQueueChange(rb, xid, lsn, change);
+
+               MemoryContextSwitchTo(oldcontext);
+       }
+       else
+       {
+               ReorderBufferTXN *txn = NULL;
+               volatile Snapshot snapshot_now = snapshot;
+
+               if (xid != InvalidTransactionId)
+                       txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
+
+               /* setup snapshot to allow catalog access */
+               SetupHistoricSnapshot(snapshot_now, NULL);
+               PG_TRY();
+               {
+                       rb->message(rb, txn, lsn, false, prefix, message_size, message);
+
+                       TeardownHistoricSnapshot(false);
+               }
+               PG_CATCH();
+               {
+                       TeardownHistoricSnapshot(true);
+                       PG_RE_THROW();
+               }
+               PG_END_TRY();
+       }
+}
+
+/*
+ * 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);
@@ -641,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)
@@ -659,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;
+               }
        }
 }
 
@@ -697,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,
@@ -709,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);
 }
 
 
@@ -843,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);
@@ -868,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);
 
@@ -1060,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,
@@ -1087,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 */
@@ -1111,7 +1247,6 @@ ReorderBufferBuildTupleCidHash(ReorderBuffer *rb, ReorderBufferTXN *txn)
 
        hash_ctl.keysize = sizeof(ReorderBufferTupleCidKey);
        hash_ctl.entrysize = sizeof(ReorderBufferTupleCidEnt);
-       hash_ctl.hash = tag_hash;
        hash_ctl.hcxt = rb->context;
 
        /*
@@ -1120,7 +1255,7 @@ ReorderBufferBuildTupleCidHash(ReorderBuffer *rb, ReorderBufferTXN *txn)
         */
        txn->tuplecid_hash =
                hash_create("ReorderBufferTupleCid", txn->ntuplecids, &hash_ctl,
-                                       HASH_ELEM | HASH_FUNCTION | HASH_CONTEXT);
+                                       HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
 
        dlist_foreach(iter, &txn->tuplecids)
        {
@@ -1190,15 +1325,15 @@ ReorderBufferCopySnap(ReorderBuffer *rb, Snapshot orig_snap,
        memcpy(snap, orig_snap, sizeof(SnapshotData));
 
        snap->copied = true;
-       snap->active_count = 0;
-       snap->regd_count = 1;
+       snap->active_count = 1;         /* mark as active so nobody frees it */
+       snap->regd_count = 0;
        snap->xip = (TransactionId *) (snap + 1);
 
        memcpy(snap->xip, orig_snap->xip, sizeof(TransactionId) * snap->xcnt);
 
        /*
         * 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;
@@ -1242,30 +1377,29 @@ 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,
                                        XLogRecPtr commit_lsn, XLogRecPtr end_lsn,
-                                       TimestampTz commit_time)
+                                       TimestampTz commit_time,
+                                       RepOriginId origin_id, XLogRecPtr origin_lsn)
 {
        ReorderBufferTXN *txn;
-       ReorderBufferIterTXNState *iterstate = NULL;
-       ReorderBufferChange *change;
-
+       volatile Snapshot snapshot_now;
        volatile CommandId command_id = FirstCommandId;
-       volatile Snapshot snapshot_now = NULL;
-       volatile bool using_subtxn = false;
+       bool            using_subtxn;
+       ReorderBufferIterTXNState *volatile iterstate = NULL;
 
        txn = ReorderBufferTXNByXid(rb, xid, false, NULL, InvalidXLogRecPtr,
                                                                false);
@@ -1277,16 +1411,14 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
        txn->final_lsn = commit_lsn;
        txn->end_lsn = end_lsn;
        txn->commit_time = commit_time;
-
-       /* serialize the last bunch of changes if we need start earlier anyway */
-       if (txn->nentries_mem != txn->nentries)
-               ReorderBufferSerializeTXN(rb, txn);
+       txn->origin_id = origin_id;
+       txn->origin_lsn = origin_lsn;
 
        /*
-        * 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)
        {
@@ -1303,20 +1435,22 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
        /* setup the initial snapshot */
        SetupHistoricSnapshot(snapshot_now, txn->tuplecid_hash);
 
+       /*
+        * Decoding needs access to syscaches et al., which in turn use
+        * heavyweight locks and such. Thus we need to have enough state around to
+        * keep track of those.  The easiest way is to simply use a transaction
+        * internally.  That also allows us to easily enforce that nothing writes
+        * to the database by checking for xid assignments.
+        *
+        * When we're called via the SQL SRF there's already a transaction
+        * started, so start an explicit subtransaction there.
+        */
+       using_subtxn = IsTransactionOrTransactionBlock();
+
        PG_TRY();
        {
-
-               /*
-                * Decoding needs access to syscaches et al., which in turn use
-                * heavyweight locks and such. Thus we need to have enough state
-                * around to keep track of those. The easiest way is to simply use a
-                * transaction internally. That also allows us to easily enforce that
-                * nothing writes to the database by checking for xid assignments.
-                *
-                * When we're called via the SQL SRF there's already a transaction
-                * started, so start an explicit subtransaction there.
-                */
-               using_subtxn = IsTransactionOrTransactionBlock();
+               ReorderBufferChange *change;
+               ReorderBufferChange *specinsert = NULL;
 
                if (using_subtxn)
                        BeginInternalSubTransaction("replay");
@@ -1326,20 +1460,34 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
                rb->begin(rb, txn);
 
                iterstate = ReorderBufferIterTXNInit(rb, txn);
-               while ((change = ReorderBufferIterTXNNext(rb, iterstate)))
+               while ((change = ReorderBufferIterTXNNext(rb, iterstate)) != NULL)
                {
                        Relation        relation = NULL;
                        Oid                     reloid;
 
                        switch (change->action)
                        {
+                               case REORDER_BUFFER_CHANGE_INTERNAL_SPEC_CONFIRM:
+
+                                       /*
+                                        * Confirmation for speculative insertion arrived. Simply
+                                        * 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;
+
+                                       /* intentionally fall through */
                                case REORDER_BUFFER_CHANGE_INSERT:
                                case REORDER_BUFFER_CHANGE_UPDATE:
                                case REORDER_BUFFER_CHANGE_DELETE:
                                        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
@@ -1348,7 +1496,7 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
                                        if (reloid == InvalidOid &&
                                                change->data.tp.newtuple == NULL &&
                                                change->data.tp.oldtuple == NULL)
-                                               continue;
+                                               goto change_done;
                                        else if (reloid == InvalidOid)
                                                elog(ERROR, "could not map filenode \"%s\" to relation OID",
                                                         relpathperm(change->data.tp.relnode,
@@ -1362,50 +1510,140 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
                                                         relpathperm(change->data.tp.relnode,
                                                                                 MAIN_FORKNUM));
 
-                                       if (RelationIsLogicallyLogged(relation))
+                                       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
+                                        * understand, so it doesn't make sense to handle the few
+                                        * cases we do.
+                                        */
+                                       if (relation->rd_rel->relkind == RELKIND_SEQUENCE)
+                                               goto change_done;
+
+                                       /* user-triggered change */
+                                       if (!IsToastRelation(relation))
                                        {
+                                               ReorderBufferToastReplace(rb, txn, relation, change);
+                                               rb->apply_change(rb, txn, relation, change);
+
                                                /*
-                                                * For now ignore sequence changes entirely. Most of
-                                                * the time they don't log changes using records we
-                                                * understand, so it doesn't make sense to handle the
-                                                * few cases we do.
+                                                * Only clear reassembled toast chunks if we're sure
+                                                * they're not required anymore. The creator of the
+                                                * tuple tells us.
                                                 */
-                                               if (relation->rd_rel->relkind == RELKIND_SEQUENCE)
-                                               {
-                                               }
-                                               /* user-triggered change */
-                                               else if (!IsToastRelation(relation))
-                                               {
-                                                       ReorderBufferToastReplace(rb, txn, relation, change);
-                                                       rb->apply_change(rb, txn, relation, change);
-
-                                                       /*
-                                                        * Only clear reassembled toast chunks if we're
-                                                        * sure they're not required anymore. The creator
-                                                        * of the tuple tells us.
-                                                        */
-                                                       if (change->data.tp.clear_toast_afterwards)
-                                                               ReorderBufferToastReset(rb, txn);
-                                               }
-                                               /* we're not interested in toast deletions */
-                                               else if (change->action == REORDER_BUFFER_CHANGE_INSERT)
+                                               if (change->data.tp.clear_toast_afterwards)
+                                                       ReorderBufferToastReset(rb, txn);
+                                       }
+                                       /* we're not interested in toast deletions */
+                                       else if (change->action == REORDER_BUFFER_CHANGE_INSERT)
+                                       {
+                                               /*
+                                                * Need to reassemble the full toasted Datum in
+                                                * memory, to ensure the chunks don't get reused till
+                                                * we're done remove it from the list of this
+                                                * transaction's changes. Otherwise it will get
+                                                * freed/reused while restoring spooled data from
+                                                * disk.
+                                                */
+                                               dlist_delete(&change->node);
+                                               ReorderBufferToastAppendChunk(rb, txn, relation,
+                                                                                                         change);
+                                       }
+
+                       change_done:
+
+                                       /*
+                                        * Either speculative insertion was confirmed, or it was
+                                        * unsuccessful and the record isn't needed anymore.
+                                        */
+                                       if (specinsert != NULL)
+                                       {
+                                               ReorderBufferReturnChange(rb, specinsert);
+                                               specinsert = NULL;
+                                       }
+
+                                       if (relation != NULL)
+                                       {
+                                               RelationClose(relation);
+                                               relation = NULL;
+                                       }
+                                       break;
+
+                               case REORDER_BUFFER_CHANGE_INTERNAL_SPEC_INSERT:
+
+                                       /*
+                                        * Speculative insertions are dealt with by delaying the
+                                        * processing of the insert until the confirmation record
+                                        * arrives. For that we simply unlink the record from the
+                                        * chain, so it does not get freed/reused while restoring
+                                        * spooled data from disk.
+                                        *
+                                        * This is safe in the face of concurrent catalog changes
+                                        * because the relevant relation can't be changed between
+                                        * speculative insertion and confirmation due to
+                                        * CheckTableNotInUse() and locking.
+                                        */
+
+                                       /* clear out a pending (and thus failed) speculation */
+                                       if (specinsert != NULL)
+                                       {
+                                               ReorderBufferReturnChange(rb, specinsert);
+                                               specinsert = NULL;
+                                       }
+
+                                       /* and memorize the pending insertion */
+                                       dlist_delete(&change->node);
+                                       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++)
                                                {
-                                                       /*
-                                                        * Need to reassemble the full toasted Datum in
-                                                        * memory, to ensure the chunks don't get reused
-                                                        * till we're done remove it from the list of this
-                                                        * transaction's changes. Otherwise it will get
-                                                        * freed/reused while restoring spooled data from
-                                                        * disk.
-                                                        */
-                                                       dlist_delete(&change->node);
-                                                       ReorderBufferToastAppendChunk(rb, txn, relation,
-                                                                                                                 change);
+                                                       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;
                                        }
-                                       RelationClose(relation);
+
+                               case REORDER_BUFFER_CHANGE_MESSAGE:
+                                       rb->message(rb, txn, change->lsn, true,
+                                                               change->data.msg.prefix,
+                                                               change->data.msg.message_size,
+                                                               change->data.msg.message);
                                        break;
+
                                case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
                                        /* get rid of the old */
                                        TeardownHistoricSnapshot(false);
@@ -1450,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;
@@ -1474,7 +1712,20 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
                        }
                }
 
+               /*
+                * There's a speculative insertion remaining, just clean in up, it
+                * can't have been successful, otherwise we'd gotten a confirmation
+                * record.
+                */
+               if (specinsert)
+               {
+                       ReorderBufferReturnChange(rb, specinsert);
+                       specinsert = NULL;
+               }
+
+               /* clean up the iterator */
                ReorderBufferIterTXNFinish(rb, iterstate);
+               iterstate = NULL;
 
                /* call commit callback */
                rb->commit(rb, txn, commit_lsn);
@@ -1544,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
@@ -1585,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)
        {
@@ -1596,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);
@@ -1612,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
@@ -1635,31 +1901,13 @@ ReorderBufferForget(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn)
        txn->final_lsn = lsn;
 
        /*
-        * Proccess cache invalidation messages if there are any. Even if we're
-        * not interested in the transaction's contents, it could have manipulated
-        * the catalog and we need to update the caches according to that.
+        * Process cache invalidation messages if there are any. Even if we're not
+        * interested in the transaction's contents, it could have manipulated the
+        * 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);
 
@@ -1667,18 +1915,54 @@ ReorderBufferForget(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn)
        ReorderBufferCleanupTXN(rb, txn);
 }
 
-
 /*
- * Check whether a transaction is already known in this module.xs
+ * 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()).
  */
-bool
-ReorderBufferIsXidKnown(ReorderBuffer *rb, TransactionId xid)
+void
+ReorderBufferImmediateInvalidation(ReorderBuffer *rb, uint32 ninvalidations,
+                                                                  SharedInvalidationMessage *invalidations)
 {
-       ReorderBufferTXN *txn;
+       bool            use_subtxn = IsTransactionOrTransactionBlock();
+       int                     i;
 
-       txn = ReorderBufferTXNByXid(rb, xid, false, NULL, InvalidXLogRecPtr,
-                                                               false);
-       return txn != NULL;
+       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
+ * least once for every xid in XLogRecord->xl_xid (other places in records
+ * may, but do not have to be passed through here).
+ *
+ * Reorderbuffer keeps some datastructures about transactions in LSN order,
+ * for efficiency. To do that it has to know about when transactions are seen
+ * first in the WAL. As many types of records are not actually interesting for
+ * logical decoding, they do not necessarily pass though here.
+ */
+void
+ReorderBufferProcessXid(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn)
+{
+       /* many records won't have an xid assigned, centralize check here */
+       if (xid != InvalidTransactionId)
+               ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
 }
 
 /*
@@ -1699,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,
@@ -1713,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);
 }
 
 /*
@@ -1837,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;
 }
 
@@ -1912,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);
@@ -1937,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);
@@ -1977,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);
@@ -1999,11 +2290,11 @@ ReorderBufferSerializeChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
 
        switch (change->action)
        {
+                       /* fall through these, they're all similar enough */
                case REORDER_BUFFER_CHANGE_INSERT:
-                       /* fall through */
                case REORDER_BUFFER_CHANGE_UPDATE:
-                       /* fall through */
                case REORDER_BUFFER_CHANGE_DELETE:
+               case REORDER_BUFFER_CHANGE_INTERNAL_SPEC_INSERT:
                        {
                                char       *data;
                                ReorderBufferTupleBuf *oldtup,
@@ -2015,17 +2306,18 @@ ReorderBufferSerializeChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
                                newtup = change->data.tp.newtuple;
 
                                if (oldtup)
-                                       oldlen = offsetof(ReorderBufferTupleBuf, data)
-                                               +oldtup->tuple.t_len
-                                               - offsetof(HeapTupleHeaderData, t_bits);
+                               {
+                                       sz += sizeof(HeapTupleData);
+                                       oldlen = oldtup->tuple.t_len;
+                                       sz += oldlen;
+                               }
 
                                if (newtup)
-                                       newlen = offsetof(ReorderBufferTupleBuf, data)
-                                               +newtup->tuple.t_len
-                                               - offsetof(HeapTupleHeaderData, t_bits);
-
-                               sz += oldlen;
-                               sz += newlen;
+                               {
+                                       sz += sizeof(HeapTupleData);
+                                       newlen = newtup->tuple.t_len;
+                                       sz += newlen;
+                               }
 
                                /* make sure we have enough space */
                                ReorderBufferSerializeReserve(rb, sz);
@@ -2036,15 +2328,51 @@ ReorderBufferSerializeChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
 
                                if (oldlen)
                                {
-                                       memcpy(data, oldtup, oldlen);
+                                       memcpy(data, &oldtup->tuple, sizeof(HeapTupleData));
+                                       data += sizeof(HeapTupleData);
+
+                                       memcpy(data, oldtup->tuple.t_data, oldlen);
                                        data += oldlen;
                                }
 
                                if (newlen)
                                {
-                                       memcpy(data, newtup, newlen);
+                                       memcpy(data, &newtup->tuple, sizeof(HeapTupleData));
+                                       data += sizeof(HeapTupleData);
+
+                                       memcpy(data, newtup->tuple.t_data, newlen);
                                        data += newlen;
                                }
+                               break;
+                       }
+               case REORDER_BUFFER_CHANGE_MESSAGE:
+                       {
+                               char       *data;
+                               Size            prefix_size = strlen(change->data.msg.prefix) + 1;
+
+                               sz += prefix_size + change->data.msg.message_size +
+                                       sizeof(Size) + sizeof(Size);
+                               ReorderBufferSerializeReserve(rb, sz);
+
+                               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);
+                               memcpy(data, change->data.msg.prefix,
+                                          prefix_size);
+                               data += prefix_size;
+
+                               /* write the message including the size */
+                               memcpy(data, &change->data.msg.message_size, sizeof(Size));
+                               data += sizeof(Size);
+                               memcpy(data, change->data.msg.message,
+                                          change->data.msg.message_size);
+                               data += change->data.msg.message_size;
+
                                break;
                        }
                case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
@@ -2083,9 +2411,9 @@ 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:
-                       /* ReorderBufferChange contains everything important */
-                       break;
                case REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID:
                        /* ReorderBufferChange contains everything important */
                        break;
@@ -2093,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);
 }
@@ -2131,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)
        {
@@ -2140,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;
@@ -2172,7 +2503,6 @@ ReorderBufferRestoreChanges(ReorderBuffer *rb, ReorderBufferTXN *txn,
                                                (errcode_for_file_access(),
                                                 errmsg("could not open file \"%s\": %m",
                                                                path)));
-
                }
 
                /*
@@ -2181,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)
@@ -2194,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(),
@@ -2205,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
@@ -2236,6 +2570,10 @@ ReorderBufferRestoreChanges(ReorderBuffer *rb, ReorderBufferTXN *txn,
 /*
  * Convert change from its on-disk format to in-memory format and queue it onto
  * the TXN's ->changes list.
+ *
+ * Note: although "data" is declared char*, at entry it points to a
+ * maxalign'd buffer, making it safe in most of this function to assume
+ * that the pointed-to data is suitably aligned for direct access.
  */
 static void
 ReorderBufferRestoreChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
@@ -2256,37 +2594,82 @@ ReorderBufferRestoreChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
        /* restore individual stuff */
        switch (change->action)
        {
+                       /* fall through these, they're all similar enough */
                case REORDER_BUFFER_CHANGE_INSERT:
-                       /* fall through */
                case REORDER_BUFFER_CHANGE_UPDATE:
-                       /* fall through */
                case REORDER_BUFFER_CHANGE_DELETE:
+               case REORDER_BUFFER_CHANGE_INTERNAL_SPEC_INSERT:
+                       if (change->data.tp.oldtuple)
+                       {
+                               uint32          tuplelen = ((HeapTuple) data)->t_len;
+
+                               change->data.tp.oldtuple =
+                                       ReorderBufferGetTupleBuf(rb, tuplelen - SizeofHeapTupleHeader);
+
+                               /* restore ->tuple */
+                               memcpy(&change->data.tp.oldtuple->tuple, data,
+                                          sizeof(HeapTupleData));
+                               data += sizeof(HeapTupleData);
+
+                               /* reset t_data pointer into the new tuplebuf */
+                               change->data.tp.oldtuple->tuple.t_data =
+                                       ReorderBufferTupleBufData(change->data.tp.oldtuple);
+
+                               /* restore tuple data itself */
+                               memcpy(change->data.tp.oldtuple->tuple.t_data, data, tuplelen);
+                               data += tuplelen;
+                       }
+
                        if (change->data.tp.newtuple)
                        {
-                               Size            len = offsetof(ReorderBufferTupleBuf, data)
-                               +((ReorderBufferTupleBuf *) data)->tuple.t_len
-                               - offsetof(HeapTupleHeaderData, t_bits);
+                               /* here, data might not be suitably aligned! */
+                               uint32          tuplelen;
+
+                               memcpy(&tuplelen, data + offsetof(HeapTupleData, t_len),
+                                          sizeof(uint32));
 
-                               change->data.tp.newtuple = ReorderBufferGetTupleBuf(rb);
-                               memcpy(change->data.tp.newtuple, data, len);
+                               change->data.tp.newtuple =
+                                       ReorderBufferGetTupleBuf(rb, tuplelen - SizeofHeapTupleHeader);
+
+                               /* restore ->tuple */
+                               memcpy(&change->data.tp.newtuple->tuple, data,
+                                          sizeof(HeapTupleData));
+                               data += sizeof(HeapTupleData);
+
+                               /* reset t_data pointer into the new tuplebuf */
                                change->data.tp.newtuple->tuple.t_data =
-                                       &change->data.tp.newtuple->header;
-                               data += len;
+                                       ReorderBufferTupleBufData(change->data.tp.newtuple);
+
+                               /* restore tuple data itself */
+                               memcpy(change->data.tp.newtuple->tuple.t_data, data, tuplelen);
+                               data += tuplelen;
                        }
 
-                       if (change->data.tp.oldtuple)
+                       break;
+               case REORDER_BUFFER_CHANGE_MESSAGE:
                        {
-                               Size            len = offsetof(ReorderBufferTupleBuf, data)
-                               +((ReorderBufferTupleBuf *) data)->tuple.t_len
-                               - offsetof(HeapTupleHeaderData, t_bits);
+                               Size            prefix_size;
+
+                               /* read prefix */
+                               memcpy(&prefix_size, data, sizeof(Size));
+                               data += sizeof(Size);
+                               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');
+                               data += prefix_size;
+
+                               /* 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);
+                               memcpy(change->data.msg.message, data,
+                                          change->data.msg.message_size);
+                               data += change->data.msg.message_size;
 
-                               change->data.tp.oldtuple = ReorderBufferGetTupleBuf(rb);
-                               memcpy(change->data.tp.oldtuple, data, len);
-                               change->data.tp.oldtuple->tuple.t_data =
-                                       &change->data.tp.oldtuple->header;
-                               data += len;
+                               break;
                        }
-                       break;
                case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
                        {
                                Snapshot        oldsnap;
@@ -2311,6 +2694,8 @@ 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:
                        break;
@@ -2333,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(),
@@ -2354,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.
@@ -2364,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;
@@ -2383,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);
 }
@@ -2434,10 +2839,9 @@ ReorderBufferToastInitHash(ReorderBuffer *rb, ReorderBufferTXN *txn)
        memset(&hash_ctl, 0, sizeof(hash_ctl));
        hash_ctl.keysize = sizeof(Oid);
        hash_ctl.entrysize = sizeof(ReorderBufferToastEnt);
-       hash_ctl.hash = tag_hash;
        hash_ctl.hcxt = rb->context;
        txn->toast_hash = hash_create("ReorderBufferToastHash", 5, &hash_ctl,
-                                                                 HASH_ELEM | HASH_FUNCTION | HASH_CONTEXT);
+                                                                 HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
 }
 
 /*
@@ -2559,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;
 
@@ -2626,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));
@@ -2662,7 +3066,7 @@ ReorderBufferToastReplace(ReorderBuffer *rb, ReorderBufferTXN *txn,
         */
        tmphtup = heap_form_tuple(desc, attrs, isnull);
        Assert(newtup->tuple.t_len <= MaxHeapTupleSize);
-       Assert(&newtup->header == newtup->tuple.t_data);
+       Assert(ReorderBufferTupleBufData(newtup) == newtup->tuple.t_data);
 
        memcpy(newtup->tuple.t_data, tmphtup->t_data, tmphtup->t_len);
        newtup->tuple.t_len = tmphtup->t_len;
@@ -2772,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
                        );
@@ -2796,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)
        {
@@ -2812,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,
@@ -2871,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)