]> granicus.if.org Git - postgresql/blob - src/backend/replication/logical/reorderbuffer.c
Allow memory contexts to have both fixed and variable ident strings.
[postgresql] / src / backend / replication / logical / reorderbuffer.c
1 /*-------------------------------------------------------------------------
2  *
3  * reorderbuffer.c
4  *        PostgreSQL logical replay/reorder buffer management
5  *
6  *
7  * Copyright (c) 2012-2018, PostgreSQL Global Development Group
8  *
9  *
10  * IDENTIFICATION
11  *        src/backend/replication/reorderbuffer.c
12  *
13  * NOTES
14  *        This module gets handed individual pieces of transactions in the order
15  *        they are written to the WAL and is responsible to reassemble them into
16  *        toplevel transaction sized pieces. When a transaction is completely
17  *        reassembled - signalled by reading the transaction commit record - it
18  *        will then call the output plugin (cf. ReorderBufferCommit()) with the
19  *        individual changes. The output plugins rely on snapshots built by
20  *        snapbuild.c which hands them to us.
21  *
22  *        Transactions and subtransactions/savepoints in postgres are not
23  *        immediately linked to each other from outside the performing
24  *        backend. Only at commit/abort (or special xact_assignment records) they
25  *        are linked together. Which means that we will have to splice together a
26  *        toplevel transaction from its subtransactions. To do that efficiently we
27  *        build a binary heap indexed by the smallest current lsn of the individual
28  *        subtransactions' changestreams. As the individual streams are inherently
29  *        ordered by LSN - since that is where we build them from - the transaction
30  *        can easily be reassembled by always using the subtransaction with the
31  *        smallest current LSN from the heap.
32  *
33  *        In order to cope with large transactions - which can be several times as
34  *        big as the available memory - this module supports spooling the contents
35  *        of a large transactions to disk. When the transaction is replayed the
36  *        contents of individual (sub-)transactions will be read from disk in
37  *        chunks.
38  *
39  *        This module also has to deal with reassembling toast records from the
40  *        individual chunks stored in WAL. When a new (or initial) version of a
41  *        tuple is stored in WAL it will always be preceded by the toast chunks
42  *        emitted for the columns stored out of line. Within a single toplevel
43  *        transaction there will be no other data carrying records between a row's
44  *        toast chunks and the row data itself. See ReorderBufferToast* for
45  *        details.
46  *
47  *        ReorderBuffer uses two special memory context types - SlabContext for
48  *        allocations of fixed-length structures (changes and transactions), and
49  *        GenerationContext for the variable-length transaction data (allocated
50  *        and freed in groups with similar lifespan).
51  *
52  * -------------------------------------------------------------------------
53  */
54 #include "postgres.h"
55
56 #include <unistd.h>
57 #include <sys/stat.h>
58
59 #include "access/rewriteheap.h"
60 #include "access/transam.h"
61 #include "access/tuptoaster.h"
62 #include "access/xact.h"
63 #include "access/xlog_internal.h"
64 #include "catalog/catalog.h"
65 #include "lib/binaryheap.h"
66 #include "miscadmin.h"
67 #include "pgstat.h"
68 #include "replication/logical.h"
69 #include "replication/reorderbuffer.h"
70 #include "replication/slot.h"
71 #include "replication/snapbuild.h"      /* just for SnapBuildSnapDecRefcount */
72 #include "storage/bufmgr.h"
73 #include "storage/fd.h"
74 #include "storage/sinval.h"
75 #include "utils/builtins.h"
76 #include "utils/combocid.h"
77 #include "utils/memdebug.h"
78 #include "utils/memutils.h"
79 #include "utils/rel.h"
80 #include "utils/relfilenodemap.h"
81 #include "utils/tqual.h"
82
83
84 /* entry for a hash table we use to map from xid to our transaction state */
85 typedef struct ReorderBufferTXNByIdEnt
86 {
87         TransactionId xid;
88         ReorderBufferTXN *txn;
89 } ReorderBufferTXNByIdEnt;
90
91 /* data structures for (relfilenode, ctid) => (cmin, cmax) mapping */
92 typedef struct ReorderBufferTupleCidKey
93 {
94         RelFileNode relnode;
95         ItemPointerData tid;
96 } ReorderBufferTupleCidKey;
97
98 typedef struct ReorderBufferTupleCidEnt
99 {
100         ReorderBufferTupleCidKey key;
101         CommandId       cmin;
102         CommandId       cmax;
103         CommandId       combocid;               /* just for debugging */
104 } ReorderBufferTupleCidEnt;
105
106 /* k-way in-order change iteration support structures */
107 typedef struct ReorderBufferIterTXNEntry
108 {
109         XLogRecPtr      lsn;
110         ReorderBufferChange *change;
111         ReorderBufferTXN *txn;
112         int                     fd;
113         XLogSegNo       segno;
114 } ReorderBufferIterTXNEntry;
115
116 typedef struct ReorderBufferIterTXNState
117 {
118         binaryheap *heap;
119         Size            nr_txns;
120         dlist_head      old_change;
121         ReorderBufferIterTXNEntry entries[FLEXIBLE_ARRAY_MEMBER];
122 } ReorderBufferIterTXNState;
123
124 /* toast datastructures */
125 typedef struct ReorderBufferToastEnt
126 {
127         Oid                     chunk_id;               /* toast_table.chunk_id */
128         int32           last_chunk_seq; /* toast_table.chunk_seq of the last chunk we
129                                                                  * have seen */
130         Size            num_chunks;             /* number of chunks we've already seen */
131         Size            size;                   /* combined size of chunks seen */
132         dlist_head      chunks;                 /* linked list of chunks */
133         struct varlena *reconstructed;  /* reconstructed varlena now pointed to in
134                                                                          * main tup */
135 } ReorderBufferToastEnt;
136
137 /* Disk serialization support datastructures */
138 typedef struct ReorderBufferDiskChange
139 {
140         Size            size;
141         ReorderBufferChange change;
142         /* data follows */
143 } ReorderBufferDiskChange;
144
145 /*
146  * Maximum number of changes kept in memory, per transaction. After that,
147  * changes are spooled to disk.
148  *
149  * The current value should be sufficient to decode the entire transaction
150  * without hitting disk in OLTP workloads, while starting to spool to disk in
151  * other workloads reasonably fast.
152  *
153  * At some point in the future it probably makes sense to have a more elaborate
154  * resource management here, but it's not entirely clear what that would look
155  * like.
156  */
157 static const Size max_changes_in_memory = 4096;
158
159 /* ---------------------------------------
160  * primary reorderbuffer support routines
161  * ---------------------------------------
162  */
163 static ReorderBufferTXN *ReorderBufferGetTXN(ReorderBuffer *rb);
164 static void ReorderBufferReturnTXN(ReorderBuffer *rb, ReorderBufferTXN *txn);
165 static ReorderBufferTXN *ReorderBufferTXNByXid(ReorderBuffer *rb,
166                                           TransactionId xid, bool create, bool *is_new,
167                                           XLogRecPtr lsn, bool create_as_top);
168
169 static void AssertTXNLsnOrder(ReorderBuffer *rb);
170
171 /* ---------------------------------------
172  * support functions for lsn-order iterating over the ->changes of a
173  * transaction and its subtransactions
174  *
175  * used for iteration over the k-way heap merge of a transaction and its
176  * subtransactions
177  * ---------------------------------------
178  */
179 static ReorderBufferIterTXNState *ReorderBufferIterTXNInit(ReorderBuffer *rb, ReorderBufferTXN *txn);
180 static ReorderBufferChange *ReorderBufferIterTXNNext(ReorderBuffer *rb, ReorderBufferIterTXNState *state);
181 static void ReorderBufferIterTXNFinish(ReorderBuffer *rb,
182                                                    ReorderBufferIterTXNState *state);
183 static void ReorderBufferExecuteInvalidations(ReorderBuffer *rb, ReorderBufferTXN *txn);
184
185 /*
186  * ---------------------------------------
187  * Disk serialization support functions
188  * ---------------------------------------
189  */
190 static void ReorderBufferCheckSerializeTXN(ReorderBuffer *rb, ReorderBufferTXN *txn);
191 static void ReorderBufferSerializeTXN(ReorderBuffer *rb, ReorderBufferTXN *txn);
192 static void ReorderBufferSerializeChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
193                                                          int fd, ReorderBufferChange *change);
194 static Size ReorderBufferRestoreChanges(ReorderBuffer *rb, ReorderBufferTXN *txn,
195                                                         int *fd, XLogSegNo *segno);
196 static void ReorderBufferRestoreChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
197                                                    char *change);
198 static void ReorderBufferRestoreCleanup(ReorderBuffer *rb, ReorderBufferTXN *txn);
199 static void ReorderBufferCleanupSerializedTXNs(const char *slotname);
200 static void ReorderBufferSerializedPath(char *path, ReplicationSlot *slot,
201                                                         TransactionId xid, XLogSegNo segno);
202
203 static void ReorderBufferFreeSnap(ReorderBuffer *rb, Snapshot snap);
204 static Snapshot ReorderBufferCopySnap(ReorderBuffer *rb, Snapshot orig_snap,
205                                           ReorderBufferTXN *txn, CommandId cid);
206
207 /* ---------------------------------------
208  * toast reassembly support
209  * ---------------------------------------
210  */
211 static void ReorderBufferToastInitHash(ReorderBuffer *rb, ReorderBufferTXN *txn);
212 static void ReorderBufferToastReset(ReorderBuffer *rb, ReorderBufferTXN *txn);
213 static void ReorderBufferToastReplace(ReorderBuffer *rb, ReorderBufferTXN *txn,
214                                                   Relation relation, ReorderBufferChange *change);
215 static void ReorderBufferToastAppendChunk(ReorderBuffer *rb, ReorderBufferTXN *txn,
216                                                           Relation relation, ReorderBufferChange *change);
217
218
219 /*
220  * Allocate a new ReorderBuffer and clean out any old serialized state from
221  * prior ReorderBuffer instances for the same slot.
222  */
223 ReorderBuffer *
224 ReorderBufferAllocate(void)
225 {
226         ReorderBuffer *buffer;
227         HASHCTL         hash_ctl;
228         MemoryContext new_ctx;
229
230         Assert(MyReplicationSlot != NULL);
231
232         /* allocate memory in own context, to have better accountability */
233         new_ctx = AllocSetContextCreate(CurrentMemoryContext,
234                                                                         "ReorderBuffer",
235                                                                         ALLOCSET_DEFAULT_SIZES);
236
237         buffer =
238                 (ReorderBuffer *) MemoryContextAlloc(new_ctx, sizeof(ReorderBuffer));
239
240         memset(&hash_ctl, 0, sizeof(hash_ctl));
241
242         buffer->context = new_ctx;
243
244         buffer->change_context = SlabContextCreate(new_ctx,
245                                                                                            "Change",
246                                                                                            SLAB_DEFAULT_BLOCK_SIZE,
247                                                                                            sizeof(ReorderBufferChange));
248
249         buffer->txn_context = SlabContextCreate(new_ctx,
250                                                                                         "TXN",
251                                                                                         SLAB_DEFAULT_BLOCK_SIZE,
252                                                                                         sizeof(ReorderBufferTXN));
253
254         buffer->tup_context = GenerationContextCreate(new_ctx,
255                                                                                                   "Tuples",
256                                                                                                   SLAB_LARGE_BLOCK_SIZE);
257
258         hash_ctl.keysize = sizeof(TransactionId);
259         hash_ctl.entrysize = sizeof(ReorderBufferTXNByIdEnt);
260         hash_ctl.hcxt = buffer->context;
261
262         buffer->by_txn = hash_create("ReorderBufferByXid", 1000, &hash_ctl,
263                                                                  HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
264
265         buffer->by_txn_last_xid = InvalidTransactionId;
266         buffer->by_txn_last_txn = NULL;
267
268         buffer->outbuf = NULL;
269         buffer->outbufsize = 0;
270
271         buffer->current_restart_decoding_lsn = InvalidXLogRecPtr;
272
273         dlist_init(&buffer->toplevel_by_lsn);
274
275         /*
276          * Ensure there's no stale data from prior uses of this slot, in case some
277          * prior exit avoided calling ReorderBufferFree. Failure to do this can
278          * produce duplicated txns, and it's very cheap if there's nothing there.
279          */
280         ReorderBufferCleanupSerializedTXNs(NameStr(MyReplicationSlot->data.name));
281
282         return buffer;
283 }
284
285 /*
286  * Free a ReorderBuffer
287  */
288 void
289 ReorderBufferFree(ReorderBuffer *rb)
290 {
291         MemoryContext context = rb->context;
292
293         /*
294          * We free separately allocated data by entirely scrapping reorderbuffer's
295          * memory context.
296          */
297         MemoryContextDelete(context);
298
299         /* Free disk space used by unconsumed reorder buffers */
300         ReorderBufferCleanupSerializedTXNs(NameStr(MyReplicationSlot->data.name));
301 }
302
303 /*
304  * Get an unused, possibly preallocated, ReorderBufferTXN.
305  */
306 static ReorderBufferTXN *
307 ReorderBufferGetTXN(ReorderBuffer *rb)
308 {
309         ReorderBufferTXN *txn;
310
311         txn = (ReorderBufferTXN *)
312                 MemoryContextAlloc(rb->txn_context, sizeof(ReorderBufferTXN));
313
314         memset(txn, 0, sizeof(ReorderBufferTXN));
315
316         dlist_init(&txn->changes);
317         dlist_init(&txn->tuplecids);
318         dlist_init(&txn->subtxns);
319
320         return txn;
321 }
322
323 /*
324  * Free a ReorderBufferTXN.
325  *
326  * Deallocation might be delayed for efficiency purposes, for details check
327  * the comments above max_cached_changes's definition.
328  */
329 static void
330 ReorderBufferReturnTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
331 {
332         /* clean the lookup cache if we were cached (quite likely) */
333         if (rb->by_txn_last_xid == txn->xid)
334         {
335                 rb->by_txn_last_xid = InvalidTransactionId;
336                 rb->by_txn_last_txn = NULL;
337         }
338
339         /* free data that's contained */
340
341         if (txn->tuplecid_hash != NULL)
342         {
343                 hash_destroy(txn->tuplecid_hash);
344                 txn->tuplecid_hash = NULL;
345         }
346
347         if (txn->invalidations)
348         {
349                 pfree(txn->invalidations);
350                 txn->invalidations = NULL;
351         }
352
353         pfree(txn);
354 }
355
356 /*
357  * Get an unused, possibly preallocated, ReorderBufferChange.
358  */
359 ReorderBufferChange *
360 ReorderBufferGetChange(ReorderBuffer *rb)
361 {
362         ReorderBufferChange *change;
363
364         change = (ReorderBufferChange *)
365                 MemoryContextAlloc(rb->change_context, sizeof(ReorderBufferChange));
366
367         memset(change, 0, sizeof(ReorderBufferChange));
368         return change;
369 }
370
371 /*
372  * Free an ReorderBufferChange.
373  *
374  * Deallocation might be delayed for efficiency purposes, for details check
375  * the comments above max_cached_changes's definition.
376  */
377 void
378 ReorderBufferReturnChange(ReorderBuffer *rb, ReorderBufferChange *change)
379 {
380         /* free contained data */
381         switch (change->action)
382         {
383                 case REORDER_BUFFER_CHANGE_INSERT:
384                 case REORDER_BUFFER_CHANGE_UPDATE:
385                 case REORDER_BUFFER_CHANGE_DELETE:
386                 case REORDER_BUFFER_CHANGE_INTERNAL_SPEC_INSERT:
387                         if (change->data.tp.newtuple)
388                         {
389                                 ReorderBufferReturnTupleBuf(rb, change->data.tp.newtuple);
390                                 change->data.tp.newtuple = NULL;
391                         }
392
393                         if (change->data.tp.oldtuple)
394                         {
395                                 ReorderBufferReturnTupleBuf(rb, change->data.tp.oldtuple);
396                                 change->data.tp.oldtuple = NULL;
397                         }
398                         break;
399                 case REORDER_BUFFER_CHANGE_MESSAGE:
400                         if (change->data.msg.prefix != NULL)
401                                 pfree(change->data.msg.prefix);
402                         change->data.msg.prefix = NULL;
403                         if (change->data.msg.message != NULL)
404                                 pfree(change->data.msg.message);
405                         change->data.msg.message = NULL;
406                         break;
407                 case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
408                         if (change->data.snapshot)
409                         {
410                                 ReorderBufferFreeSnap(rb, change->data.snapshot);
411                                 change->data.snapshot = NULL;
412                         }
413                         break;
414                         /* no data in addition to the struct itself */
415                 case REORDER_BUFFER_CHANGE_INTERNAL_SPEC_CONFIRM:
416                 case REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID:
417                 case REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID:
418                         break;
419         }
420
421         pfree(change);
422 }
423
424 /*
425  * Get an unused, possibly preallocated, ReorderBufferTupleBuf fitting at
426  * least a tuple of size tuple_len (excluding header overhead).
427  */
428 ReorderBufferTupleBuf *
429 ReorderBufferGetTupleBuf(ReorderBuffer *rb, Size tuple_len)
430 {
431         ReorderBufferTupleBuf *tuple;
432         Size            alloc_len;
433
434         alloc_len = tuple_len + SizeofHeapTupleHeader;
435
436         tuple = (ReorderBufferTupleBuf *)
437                 MemoryContextAlloc(rb->tup_context,
438                                                    sizeof(ReorderBufferTupleBuf) +
439                                                    MAXIMUM_ALIGNOF + alloc_len);
440         tuple->alloc_tuple_size = alloc_len;
441         tuple->tuple.t_data = ReorderBufferTupleBufData(tuple);
442
443         return tuple;
444 }
445
446 /*
447  * Free an ReorderBufferTupleBuf.
448  *
449  * Deallocation might be delayed for efficiency purposes, for details check
450  * the comments above max_cached_changes's definition.
451  */
452 void
453 ReorderBufferReturnTupleBuf(ReorderBuffer *rb, ReorderBufferTupleBuf *tuple)
454 {
455         pfree(tuple);
456 }
457
458 /*
459  * Return the ReorderBufferTXN from the given buffer, specified by Xid.
460  * If create is true, and a transaction doesn't already exist, create it
461  * (with the given LSN, and as top transaction if that's specified);
462  * when this happens, is_new is set to true.
463  */
464 static ReorderBufferTXN *
465 ReorderBufferTXNByXid(ReorderBuffer *rb, TransactionId xid, bool create,
466                                           bool *is_new, XLogRecPtr lsn, bool create_as_top)
467 {
468         ReorderBufferTXN *txn;
469         ReorderBufferTXNByIdEnt *ent;
470         bool            found;
471
472         Assert(TransactionIdIsValid(xid));
473         Assert(!create || lsn != InvalidXLogRecPtr);
474
475         /*
476          * Check the one-entry lookup cache first
477          */
478         if (TransactionIdIsValid(rb->by_txn_last_xid) &&
479                 rb->by_txn_last_xid == xid)
480         {
481                 txn = rb->by_txn_last_txn;
482
483                 if (txn != NULL)
484                 {
485                         /* found it, and it's valid */
486                         if (is_new)
487                                 *is_new = false;
488                         return txn;
489                 }
490
491                 /*
492                  * cached as non-existent, and asked not to create? Then nothing else
493                  * to do.
494                  */
495                 if (!create)
496                         return NULL;
497                 /* otherwise fall through to create it */
498         }
499
500         /*
501          * If the cache wasn't hit or it yielded an "does-not-exist" and we want
502          * to create an entry.
503          */
504
505         /* search the lookup table */
506         ent = (ReorderBufferTXNByIdEnt *)
507                 hash_search(rb->by_txn,
508                                         (void *) &xid,
509                                         create ? HASH_ENTER : HASH_FIND,
510                                         &found);
511         if (found)
512                 txn = ent->txn;
513         else if (create)
514         {
515                 /* initialize the new entry, if creation was requested */
516                 Assert(ent != NULL);
517
518                 ent->txn = ReorderBufferGetTXN(rb);
519                 ent->txn->xid = xid;
520                 txn = ent->txn;
521                 txn->first_lsn = lsn;
522                 txn->restart_decoding_lsn = rb->current_restart_decoding_lsn;
523
524                 if (create_as_top)
525                 {
526                         dlist_push_tail(&rb->toplevel_by_lsn, &txn->node);
527                         AssertTXNLsnOrder(rb);
528                 }
529         }
530         else
531                 txn = NULL;                             /* not found and not asked to create */
532
533         /* update cache */
534         rb->by_txn_last_xid = xid;
535         rb->by_txn_last_txn = txn;
536
537         if (is_new)
538                 *is_new = !found;
539
540         Assert(!create || txn != NULL);
541         return txn;
542 }
543
544 /*
545  * Queue a change into a transaction so it can be replayed upon commit.
546  */
547 void
548 ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
549                                                  ReorderBufferChange *change)
550 {
551         ReorderBufferTXN *txn;
552
553         txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
554
555         change->lsn = lsn;
556         Assert(InvalidXLogRecPtr != lsn);
557         dlist_push_tail(&txn->changes, &change->node);
558         txn->nentries++;
559         txn->nentries_mem++;
560
561         ReorderBufferCheckSerializeTXN(rb, txn);
562 }
563
564 /*
565  * Queue message into a transaction so it can be processed upon commit.
566  */
567 void
568 ReorderBufferQueueMessage(ReorderBuffer *rb, TransactionId xid,
569                                                   Snapshot snapshot, XLogRecPtr lsn,
570                                                   bool transactional, const char *prefix,
571                                                   Size message_size, const char *message)
572 {
573         if (transactional)
574         {
575                 MemoryContext oldcontext;
576                 ReorderBufferChange *change;
577
578                 Assert(xid != InvalidTransactionId);
579
580                 oldcontext = MemoryContextSwitchTo(rb->context);
581
582                 change = ReorderBufferGetChange(rb);
583                 change->action = REORDER_BUFFER_CHANGE_MESSAGE;
584                 change->data.msg.prefix = pstrdup(prefix);
585                 change->data.msg.message_size = message_size;
586                 change->data.msg.message = palloc(message_size);
587                 memcpy(change->data.msg.message, message, message_size);
588
589                 ReorderBufferQueueChange(rb, xid, lsn, change);
590
591                 MemoryContextSwitchTo(oldcontext);
592         }
593         else
594         {
595                 ReorderBufferTXN *txn = NULL;
596                 volatile Snapshot snapshot_now = snapshot;
597
598                 if (xid != InvalidTransactionId)
599                         txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
600
601                 /* setup snapshot to allow catalog access */
602                 SetupHistoricSnapshot(snapshot_now, NULL);
603                 PG_TRY();
604                 {
605                         rb->message(rb, txn, lsn, false, prefix, message_size, message);
606
607                         TeardownHistoricSnapshot(false);
608                 }
609                 PG_CATCH();
610                 {
611                         TeardownHistoricSnapshot(true);
612                         PG_RE_THROW();
613                 }
614                 PG_END_TRY();
615         }
616 }
617
618
619 static void
620 AssertTXNLsnOrder(ReorderBuffer *rb)
621 {
622 #ifdef USE_ASSERT_CHECKING
623         dlist_iter      iter;
624         XLogRecPtr      prev_first_lsn = InvalidXLogRecPtr;
625
626         dlist_foreach(iter, &rb->toplevel_by_lsn)
627         {
628                 ReorderBufferTXN *cur_txn;
629
630                 cur_txn = dlist_container(ReorderBufferTXN, node, iter.cur);
631                 Assert(cur_txn->first_lsn != InvalidXLogRecPtr);
632
633                 if (cur_txn->end_lsn != InvalidXLogRecPtr)
634                         Assert(cur_txn->first_lsn <= cur_txn->end_lsn);
635
636                 if (prev_first_lsn != InvalidXLogRecPtr)
637                         Assert(prev_first_lsn < cur_txn->first_lsn);
638
639                 Assert(!cur_txn->is_known_as_subxact);
640                 prev_first_lsn = cur_txn->first_lsn;
641         }
642 #endif
643 }
644
645 ReorderBufferTXN *
646 ReorderBufferGetOldestTXN(ReorderBuffer *rb)
647 {
648         ReorderBufferTXN *txn;
649
650         if (dlist_is_empty(&rb->toplevel_by_lsn))
651                 return NULL;
652
653         AssertTXNLsnOrder(rb);
654
655         txn = dlist_head_element(ReorderBufferTXN, node, &rb->toplevel_by_lsn);
656
657         Assert(!txn->is_known_as_subxact);
658         Assert(txn->first_lsn != InvalidXLogRecPtr);
659         return txn;
660 }
661
662 void
663 ReorderBufferSetRestartPoint(ReorderBuffer *rb, XLogRecPtr ptr)
664 {
665         rb->current_restart_decoding_lsn = ptr;
666 }
667
668 void
669 ReorderBufferAssignChild(ReorderBuffer *rb, TransactionId xid,
670                                                  TransactionId subxid, XLogRecPtr lsn)
671 {
672         ReorderBufferTXN *txn;
673         ReorderBufferTXN *subtxn;
674         bool            new_top;
675         bool            new_sub;
676
677         txn = ReorderBufferTXNByXid(rb, xid, true, &new_top, lsn, true);
678         subtxn = ReorderBufferTXNByXid(rb, subxid, true, &new_sub, lsn, false);
679
680         if (new_sub)
681         {
682                 /*
683                  * we assign subtransactions to top level transaction even if we don't
684                  * have data for it yet, assignment records frequently reference xids
685                  * that have not yet produced any records. Knowing those aren't top
686                  * level xids allows us to make processing cheaper in some places.
687                  */
688                 dlist_push_tail(&txn->subtxns, &subtxn->node);
689                 txn->nsubtxns++;
690         }
691         else if (!subtxn->is_known_as_subxact)
692         {
693                 subtxn->is_known_as_subxact = true;
694                 Assert(subtxn->nsubtxns == 0);
695
696                 /* remove from lsn order list of top-level transactions */
697                 dlist_delete(&subtxn->node);
698
699                 /* add to toplevel transaction */
700                 dlist_push_tail(&txn->subtxns, &subtxn->node);
701                 txn->nsubtxns++;
702         }
703         else if (new_top)
704         {
705                 elog(ERROR, "existing subxact assigned to unknown toplevel xact");
706         }
707 }
708
709 /*
710  * Associate a subtransaction with its toplevel transaction at commit
711  * time. There may be no further changes added after this.
712  */
713 void
714 ReorderBufferCommitChild(ReorderBuffer *rb, TransactionId xid,
715                                                  TransactionId subxid, XLogRecPtr commit_lsn,
716                                                  XLogRecPtr end_lsn)
717 {
718         ReorderBufferTXN *txn;
719         ReorderBufferTXN *subtxn;
720
721         subtxn = ReorderBufferTXNByXid(rb, subxid, false, NULL,
722                                                                    InvalidXLogRecPtr, false);
723
724         /*
725          * No need to do anything if that subtxn didn't contain any changes
726          */
727         if (!subtxn)
728                 return;
729
730         txn = ReorderBufferTXNByXid(rb, xid, false, NULL, commit_lsn, true);
731
732         if (txn == NULL)
733                 elog(ERROR, "subxact logged without previous toplevel record");
734
735         /*
736          * Pass our base snapshot to the parent transaction if it doesn't have
737          * one, or ours is older. That can happen if there are no changes in the
738          * toplevel transaction but in one of the child transactions. This allows
739          * the parent to simply use its base snapshot initially.
740          */
741         if (subtxn->base_snapshot != NULL &&
742                 (txn->base_snapshot == NULL ||
743                  txn->base_snapshot_lsn > subtxn->base_snapshot_lsn))
744         {
745                 txn->base_snapshot = subtxn->base_snapshot;
746                 txn->base_snapshot_lsn = subtxn->base_snapshot_lsn;
747                 subtxn->base_snapshot = NULL;
748                 subtxn->base_snapshot_lsn = InvalidXLogRecPtr;
749         }
750
751         subtxn->final_lsn = commit_lsn;
752         subtxn->end_lsn = end_lsn;
753
754         if (!subtxn->is_known_as_subxact)
755         {
756                 subtxn->is_known_as_subxact = true;
757                 Assert(subtxn->nsubtxns == 0);
758
759                 /* remove from lsn order list of top-level transactions */
760                 dlist_delete(&subtxn->node);
761
762                 /* add to subtransaction list */
763                 dlist_push_tail(&txn->subtxns, &subtxn->node);
764                 txn->nsubtxns++;
765         }
766 }
767
768
769 /*
770  * Support for efficiently iterating over a transaction's and its
771  * subtransactions' changes.
772  *
773  * We do by doing a k-way merge between transactions/subtransactions. For that
774  * we model the current heads of the different transactions as a binary heap
775  * so we easily know which (sub-)transaction has the change with the smallest
776  * lsn next.
777  *
778  * We assume the changes in individual transactions are already sorted by LSN.
779  */
780
781 /*
782  * Binary heap comparison function.
783  */
784 static int
785 ReorderBufferIterCompare(Datum a, Datum b, void *arg)
786 {
787         ReorderBufferIterTXNState *state = (ReorderBufferIterTXNState *) arg;
788         XLogRecPtr      pos_a = state->entries[DatumGetInt32(a)].lsn;
789         XLogRecPtr      pos_b = state->entries[DatumGetInt32(b)].lsn;
790
791         if (pos_a < pos_b)
792                 return 1;
793         else if (pos_a == pos_b)
794                 return 0;
795         return -1;
796 }
797
798 /*
799  * Allocate & initialize an iterator which iterates in lsn order over a
800  * transaction and all its subtransactions.
801  */
802 static ReorderBufferIterTXNState *
803 ReorderBufferIterTXNInit(ReorderBuffer *rb, ReorderBufferTXN *txn)
804 {
805         Size            nr_txns = 0;
806         ReorderBufferIterTXNState *state;
807         dlist_iter      cur_txn_i;
808         int32           off;
809
810         /*
811          * Calculate the size of our heap: one element for every transaction that
812          * contains changes.  (Besides the transactions already in the reorder
813          * buffer, we count the one we were directly passed.)
814          */
815         if (txn->nentries > 0)
816                 nr_txns++;
817
818         dlist_foreach(cur_txn_i, &txn->subtxns)
819         {
820                 ReorderBufferTXN *cur_txn;
821
822                 cur_txn = dlist_container(ReorderBufferTXN, node, cur_txn_i.cur);
823
824                 if (cur_txn->nentries > 0)
825                         nr_txns++;
826         }
827
828         /*
829          * TODO: Consider adding fastpath for the rather common nr_txns=1 case, no
830          * need to allocate/build a heap then.
831          */
832
833         /* allocate iteration state */
834         state = (ReorderBufferIterTXNState *)
835                 MemoryContextAllocZero(rb->context,
836                                                            sizeof(ReorderBufferIterTXNState) +
837                                                            sizeof(ReorderBufferIterTXNEntry) * nr_txns);
838
839         state->nr_txns = nr_txns;
840         dlist_init(&state->old_change);
841
842         for (off = 0; off < state->nr_txns; off++)
843         {
844                 state->entries[off].fd = -1;
845                 state->entries[off].segno = 0;
846         }
847
848         /* allocate heap */
849         state->heap = binaryheap_allocate(state->nr_txns,
850                                                                           ReorderBufferIterCompare,
851                                                                           state);
852
853         /*
854          * Now insert items into the binary heap, in an unordered fashion.  (We
855          * will run a heap assembly step at the end; this is more efficient.)
856          */
857
858         off = 0;
859
860         /* add toplevel transaction if it contains changes */
861         if (txn->nentries > 0)
862         {
863                 ReorderBufferChange *cur_change;
864
865                 if (txn->serialized)
866                 {
867                         /* serialize remaining changes */
868                         ReorderBufferSerializeTXN(rb, txn);
869                         ReorderBufferRestoreChanges(rb, txn, &state->entries[off].fd,
870                                                                                 &state->entries[off].segno);
871                 }
872
873                 cur_change = dlist_head_element(ReorderBufferChange, node,
874                                                                                 &txn->changes);
875
876                 state->entries[off].lsn = cur_change->lsn;
877                 state->entries[off].change = cur_change;
878                 state->entries[off].txn = txn;
879
880                 binaryheap_add_unordered(state->heap, Int32GetDatum(off++));
881         }
882
883         /* add subtransactions if they contain changes */
884         dlist_foreach(cur_txn_i, &txn->subtxns)
885         {
886                 ReorderBufferTXN *cur_txn;
887
888                 cur_txn = dlist_container(ReorderBufferTXN, node, cur_txn_i.cur);
889
890                 if (cur_txn->nentries > 0)
891                 {
892                         ReorderBufferChange *cur_change;
893
894                         if (cur_txn->serialized)
895                         {
896                                 /* serialize remaining changes */
897                                 ReorderBufferSerializeTXN(rb, cur_txn);
898                                 ReorderBufferRestoreChanges(rb, cur_txn,
899                                                                                         &state->entries[off].fd,
900                                                                                         &state->entries[off].segno);
901                         }
902                         cur_change = dlist_head_element(ReorderBufferChange, node,
903                                                                                         &cur_txn->changes);
904
905                         state->entries[off].lsn = cur_change->lsn;
906                         state->entries[off].change = cur_change;
907                         state->entries[off].txn = cur_txn;
908
909                         binaryheap_add_unordered(state->heap, Int32GetDatum(off++));
910                 }
911         }
912
913         /* assemble a valid binary heap */
914         binaryheap_build(state->heap);
915
916         return state;
917 }
918
919 /*
920  * Return the next change when iterating over a transaction and its
921  * subtransactions.
922  *
923  * Returns NULL when no further changes exist.
924  */
925 static ReorderBufferChange *
926 ReorderBufferIterTXNNext(ReorderBuffer *rb, ReorderBufferIterTXNState *state)
927 {
928         ReorderBufferChange *change;
929         ReorderBufferIterTXNEntry *entry;
930         int32           off;
931
932         /* nothing there anymore */
933         if (state->heap->bh_size == 0)
934                 return NULL;
935
936         off = DatumGetInt32(binaryheap_first(state->heap));
937         entry = &state->entries[off];
938
939         /* free memory we might have "leaked" in the previous *Next call */
940         if (!dlist_is_empty(&state->old_change))
941         {
942                 change = dlist_container(ReorderBufferChange, node,
943                                                                  dlist_pop_head_node(&state->old_change));
944                 ReorderBufferReturnChange(rb, change);
945                 Assert(dlist_is_empty(&state->old_change));
946         }
947
948         change = entry->change;
949
950         /*
951          * update heap with information about which transaction has the next
952          * relevant change in LSN order
953          */
954
955         /* there are in-memory changes */
956         if (dlist_has_next(&entry->txn->changes, &entry->change->node))
957         {
958                 dlist_node *next = dlist_next_node(&entry->txn->changes, &change->node);
959                 ReorderBufferChange *next_change =
960                 dlist_container(ReorderBufferChange, node, next);
961
962                 /* txn stays the same */
963                 state->entries[off].lsn = next_change->lsn;
964                 state->entries[off].change = next_change;
965
966                 binaryheap_replace_first(state->heap, Int32GetDatum(off));
967                 return change;
968         }
969
970         /* try to load changes from disk */
971         if (entry->txn->nentries != entry->txn->nentries_mem)
972         {
973                 /*
974                  * Ugly: restoring changes will reuse *Change records, thus delete the
975                  * current one from the per-tx list and only free in the next call.
976                  */
977                 dlist_delete(&change->node);
978                 dlist_push_tail(&state->old_change, &change->node);
979
980                 if (ReorderBufferRestoreChanges(rb, entry->txn, &entry->fd,
981                                                                                 &state->entries[off].segno))
982                 {
983                         /* successfully restored changes from disk */
984                         ReorderBufferChange *next_change =
985                         dlist_head_element(ReorderBufferChange, node,
986                                                            &entry->txn->changes);
987
988                         elog(DEBUG2, "restored %u/%u changes from disk",
989                                  (uint32) entry->txn->nentries_mem,
990                                  (uint32) entry->txn->nentries);
991
992                         Assert(entry->txn->nentries_mem);
993                         /* txn stays the same */
994                         state->entries[off].lsn = next_change->lsn;
995                         state->entries[off].change = next_change;
996                         binaryheap_replace_first(state->heap, Int32GetDatum(off));
997
998                         return change;
999                 }
1000         }
1001
1002         /* ok, no changes there anymore, remove */
1003         binaryheap_remove_first(state->heap);
1004
1005         return change;
1006 }
1007
1008 /*
1009  * Deallocate the iterator
1010  */
1011 static void
1012 ReorderBufferIterTXNFinish(ReorderBuffer *rb,
1013                                                    ReorderBufferIterTXNState *state)
1014 {
1015         int32           off;
1016
1017         for (off = 0; off < state->nr_txns; off++)
1018         {
1019                 if (state->entries[off].fd != -1)
1020                         CloseTransientFile(state->entries[off].fd);
1021         }
1022
1023         /* free memory we might have "leaked" in the last *Next call */
1024         if (!dlist_is_empty(&state->old_change))
1025         {
1026                 ReorderBufferChange *change;
1027
1028                 change = dlist_container(ReorderBufferChange, node,
1029                                                                  dlist_pop_head_node(&state->old_change));
1030                 ReorderBufferReturnChange(rb, change);
1031                 Assert(dlist_is_empty(&state->old_change));
1032         }
1033
1034         binaryheap_free(state->heap);
1035         pfree(state);
1036 }
1037
1038 /*
1039  * Cleanup the contents of a transaction, usually after the transaction
1040  * committed or aborted.
1041  */
1042 static void
1043 ReorderBufferCleanupTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
1044 {
1045         bool            found;
1046         dlist_mutable_iter iter;
1047
1048         /* cleanup subtransactions & their changes */
1049         dlist_foreach_modify(iter, &txn->subtxns)
1050         {
1051                 ReorderBufferTXN *subtxn;
1052
1053                 subtxn = dlist_container(ReorderBufferTXN, node, iter.cur);
1054
1055                 /*
1056                  * Subtransactions are always associated to the toplevel TXN, even if
1057                  * they originally were happening inside another subtxn, so we won't
1058                  * ever recurse more than one level deep here.
1059                  */
1060                 Assert(subtxn->is_known_as_subxact);
1061                 Assert(subtxn->nsubtxns == 0);
1062
1063                 ReorderBufferCleanupTXN(rb, subtxn);
1064         }
1065
1066         /* cleanup changes in the toplevel txn */
1067         dlist_foreach_modify(iter, &txn->changes)
1068         {
1069                 ReorderBufferChange *change;
1070
1071                 change = dlist_container(ReorderBufferChange, node, iter.cur);
1072
1073                 ReorderBufferReturnChange(rb, change);
1074         }
1075
1076         /*
1077          * Cleanup the tuplecids we stored for decoding catalog snapshot access.
1078          * They are always stored in the toplevel transaction.
1079          */
1080         dlist_foreach_modify(iter, &txn->tuplecids)
1081         {
1082                 ReorderBufferChange *change;
1083
1084                 change = dlist_container(ReorderBufferChange, node, iter.cur);
1085                 Assert(change->action == REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID);
1086                 ReorderBufferReturnChange(rb, change);
1087         }
1088
1089         if (txn->base_snapshot != NULL)
1090         {
1091                 SnapBuildSnapDecRefcount(txn->base_snapshot);
1092                 txn->base_snapshot = NULL;
1093                 txn->base_snapshot_lsn = InvalidXLogRecPtr;
1094         }
1095
1096         /*
1097          * Remove TXN from its containing list.
1098          *
1099          * Note: if txn->is_known_as_subxact, we are deleting the TXN from its
1100          * parent's list of known subxacts; this leaves the parent's nsubxacts
1101          * count too high, but we don't care.  Otherwise, we are deleting the TXN
1102          * from the LSN-ordered list of toplevel TXNs.
1103          */
1104         dlist_delete(&txn->node);
1105
1106         /* now remove reference from buffer */
1107         hash_search(rb->by_txn,
1108                                 (void *) &txn->xid,
1109                                 HASH_REMOVE,
1110                                 &found);
1111         Assert(found);
1112
1113         /* remove entries spilled to disk */
1114         if (txn->serialized)
1115                 ReorderBufferRestoreCleanup(rb, txn);
1116
1117         /* deallocate */
1118         ReorderBufferReturnTXN(rb, txn);
1119 }
1120
1121 /*
1122  * Build a hash with a (relfilenode, ctid) -> (cmin, cmax) mapping for use by
1123  * tqual.c's HeapTupleSatisfiesHistoricMVCC.
1124  */
1125 static void
1126 ReorderBufferBuildTupleCidHash(ReorderBuffer *rb, ReorderBufferTXN *txn)
1127 {
1128         dlist_iter      iter;
1129         HASHCTL         hash_ctl;
1130
1131         if (!txn->has_catalog_changes || dlist_is_empty(&txn->tuplecids))
1132                 return;
1133
1134         memset(&hash_ctl, 0, sizeof(hash_ctl));
1135
1136         hash_ctl.keysize = sizeof(ReorderBufferTupleCidKey);
1137         hash_ctl.entrysize = sizeof(ReorderBufferTupleCidEnt);
1138         hash_ctl.hcxt = rb->context;
1139
1140         /*
1141          * create the hash with the exact number of to-be-stored tuplecids from
1142          * the start
1143          */
1144         txn->tuplecid_hash =
1145                 hash_create("ReorderBufferTupleCid", txn->ntuplecids, &hash_ctl,
1146                                         HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
1147
1148         dlist_foreach(iter, &txn->tuplecids)
1149         {
1150                 ReorderBufferTupleCidKey key;
1151                 ReorderBufferTupleCidEnt *ent;
1152                 bool            found;
1153                 ReorderBufferChange *change;
1154
1155                 change = dlist_container(ReorderBufferChange, node, iter.cur);
1156
1157                 Assert(change->action == REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID);
1158
1159                 /* be careful about padding */
1160                 memset(&key, 0, sizeof(ReorderBufferTupleCidKey));
1161
1162                 key.relnode = change->data.tuplecid.node;
1163
1164                 ItemPointerCopy(&change->data.tuplecid.tid,
1165                                                 &key.tid);
1166
1167                 ent = (ReorderBufferTupleCidEnt *)
1168                         hash_search(txn->tuplecid_hash,
1169                                                 (void *) &key,
1170                                                 HASH_ENTER | HASH_FIND,
1171                                                 &found);
1172                 if (!found)
1173                 {
1174                         ent->cmin = change->data.tuplecid.cmin;
1175                         ent->cmax = change->data.tuplecid.cmax;
1176                         ent->combocid = change->data.tuplecid.combocid;
1177                 }
1178                 else
1179                 {
1180                         Assert(ent->cmin == change->data.tuplecid.cmin);
1181                         Assert(ent->cmax == InvalidCommandId ||
1182                                    ent->cmax == change->data.tuplecid.cmax);
1183
1184                         /*
1185                          * if the tuple got valid in this transaction and now got deleted
1186                          * we already have a valid cmin stored. The cmax will be
1187                          * InvalidCommandId though.
1188                          */
1189                         ent->cmax = change->data.tuplecid.cmax;
1190                 }
1191         }
1192 }
1193
1194 /*
1195  * Copy a provided snapshot so we can modify it privately. This is needed so
1196  * that catalog modifying transactions can look into intermediate catalog
1197  * states.
1198  */
1199 static Snapshot
1200 ReorderBufferCopySnap(ReorderBuffer *rb, Snapshot orig_snap,
1201                                           ReorderBufferTXN *txn, CommandId cid)
1202 {
1203         Snapshot        snap;
1204         dlist_iter      iter;
1205         int                     i = 0;
1206         Size            size;
1207
1208         size = sizeof(SnapshotData) +
1209                 sizeof(TransactionId) * orig_snap->xcnt +
1210                 sizeof(TransactionId) * (txn->nsubtxns + 1);
1211
1212         snap = MemoryContextAllocZero(rb->context, size);
1213         memcpy(snap, orig_snap, sizeof(SnapshotData));
1214
1215         snap->copied = true;
1216         snap->active_count = 1;         /* mark as active so nobody frees it */
1217         snap->regd_count = 0;
1218         snap->xip = (TransactionId *) (snap + 1);
1219
1220         memcpy(snap->xip, orig_snap->xip, sizeof(TransactionId) * snap->xcnt);
1221
1222         /*
1223          * snap->subxip contains all txids that belong to our transaction which we
1224          * need to check via cmin/cmax. That's why we store the toplevel
1225          * transaction in there as well.
1226          */
1227         snap->subxip = snap->xip + snap->xcnt;
1228         snap->subxip[i++] = txn->xid;
1229
1230         /*
1231          * nsubxcnt isn't decreased when subtransactions abort, so count manually.
1232          * Since it's an upper boundary it is safe to use it for the allocation
1233          * above.
1234          */
1235         snap->subxcnt = 1;
1236
1237         dlist_foreach(iter, &txn->subtxns)
1238         {
1239                 ReorderBufferTXN *sub_txn;
1240
1241                 sub_txn = dlist_container(ReorderBufferTXN, node, iter.cur);
1242                 snap->subxip[i++] = sub_txn->xid;
1243                 snap->subxcnt++;
1244         }
1245
1246         /* sort so we can bsearch() later */
1247         qsort(snap->subxip, snap->subxcnt, sizeof(TransactionId), xidComparator);
1248
1249         /* store the specified current CommandId */
1250         snap->curcid = cid;
1251
1252         return snap;
1253 }
1254
1255 /*
1256  * Free a previously ReorderBufferCopySnap'ed snapshot
1257  */
1258 static void
1259 ReorderBufferFreeSnap(ReorderBuffer *rb, Snapshot snap)
1260 {
1261         if (snap->copied)
1262                 pfree(snap);
1263         else
1264                 SnapBuildSnapDecRefcount(snap);
1265 }
1266
1267 /*
1268  * Perform the replay of a transaction and it's non-aborted subtransactions.
1269  *
1270  * Subtransactions previously have to be processed by
1271  * ReorderBufferCommitChild(), even if previously assigned to the toplevel
1272  * transaction with ReorderBufferAssignChild.
1273  *
1274  * We currently can only decode a transaction's contents in when their commit
1275  * record is read because that's currently the only place where we know about
1276  * cache invalidations. Thus, once a toplevel commit is read, we iterate over
1277  * the top and subtransactions (using a k-way merge) and replay the changes in
1278  * lsn order.
1279  */
1280 void
1281 ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
1282                                         XLogRecPtr commit_lsn, XLogRecPtr end_lsn,
1283                                         TimestampTz commit_time,
1284                                         RepOriginId origin_id, XLogRecPtr origin_lsn)
1285 {
1286         ReorderBufferTXN *txn;
1287         volatile Snapshot snapshot_now;
1288         volatile CommandId command_id = FirstCommandId;
1289         bool            using_subtxn;
1290         ReorderBufferIterTXNState *volatile iterstate = NULL;
1291
1292         txn = ReorderBufferTXNByXid(rb, xid, false, NULL, InvalidXLogRecPtr,
1293                                                                 false);
1294
1295         /* unknown transaction, nothing to replay */
1296         if (txn == NULL)
1297                 return;
1298
1299         txn->final_lsn = commit_lsn;
1300         txn->end_lsn = end_lsn;
1301         txn->commit_time = commit_time;
1302         txn->origin_id = origin_id;
1303         txn->origin_lsn = origin_lsn;
1304
1305         /*
1306          * If this transaction didn't have any real changes in our database, it's
1307          * OK not to have a snapshot. Note that ReorderBufferCommitChild will have
1308          * transferred its snapshot to this transaction if it had one and the
1309          * toplevel tx didn't.
1310          */
1311         if (txn->base_snapshot == NULL)
1312         {
1313                 Assert(txn->ninvalidations == 0);
1314                 ReorderBufferCleanupTXN(rb, txn);
1315                 return;
1316         }
1317
1318         snapshot_now = txn->base_snapshot;
1319
1320         /* build data to be able to lookup the CommandIds of catalog tuples */
1321         ReorderBufferBuildTupleCidHash(rb, txn);
1322
1323         /* setup the initial snapshot */
1324         SetupHistoricSnapshot(snapshot_now, txn->tuplecid_hash);
1325
1326         /*
1327          * Decoding needs access to syscaches et al., which in turn use
1328          * heavyweight locks and such. Thus we need to have enough state around to
1329          * keep track of those.  The easiest way is to simply use a transaction
1330          * internally.  That also allows us to easily enforce that nothing writes
1331          * to the database by checking for xid assignments.
1332          *
1333          * When we're called via the SQL SRF there's already a transaction
1334          * started, so start an explicit subtransaction there.
1335          */
1336         using_subtxn = IsTransactionOrTransactionBlock();
1337
1338         PG_TRY();
1339         {
1340                 ReorderBufferChange *change;
1341                 ReorderBufferChange *specinsert = NULL;
1342
1343                 if (using_subtxn)
1344                         BeginInternalSubTransaction("replay");
1345                 else
1346                         StartTransactionCommand();
1347
1348                 rb->begin(rb, txn);
1349
1350                 iterstate = ReorderBufferIterTXNInit(rb, txn);
1351                 while ((change = ReorderBufferIterTXNNext(rb, iterstate)) != NULL)
1352                 {
1353                         Relation        relation = NULL;
1354                         Oid                     reloid;
1355
1356                         switch (change->action)
1357                         {
1358                                 case REORDER_BUFFER_CHANGE_INTERNAL_SPEC_CONFIRM:
1359
1360                                         /*
1361                                          * Confirmation for speculative insertion arrived. Simply
1362                                          * use as a normal record. It'll be cleaned up at the end
1363                                          * of INSERT processing.
1364                                          */
1365                                         Assert(specinsert->data.tp.oldtuple == NULL);
1366                                         change = specinsert;
1367                                         change->action = REORDER_BUFFER_CHANGE_INSERT;
1368
1369                                         /* intentionally fall through */
1370                                 case REORDER_BUFFER_CHANGE_INSERT:
1371                                 case REORDER_BUFFER_CHANGE_UPDATE:
1372                                 case REORDER_BUFFER_CHANGE_DELETE:
1373                                         Assert(snapshot_now);
1374
1375                                         reloid = RelidByRelfilenode(change->data.tp.relnode.spcNode,
1376                                                                                                 change->data.tp.relnode.relNode);
1377
1378                                         /*
1379                                          * Catalog tuple without data, emitted while catalog was
1380                                          * in the process of being rewritten.
1381                                          */
1382                                         if (reloid == InvalidOid &&
1383                                                 change->data.tp.newtuple == NULL &&
1384                                                 change->data.tp.oldtuple == NULL)
1385                                                 goto change_done;
1386                                         else if (reloid == InvalidOid)
1387                                                 elog(ERROR, "could not map filenode \"%s\" to relation OID",
1388                                                          relpathperm(change->data.tp.relnode,
1389                                                                                  MAIN_FORKNUM));
1390
1391                                         relation = RelationIdGetRelation(reloid);
1392
1393                                         if (relation == NULL)
1394                                                 elog(ERROR, "could not open relation with OID %u (for filenode \"%s\")",
1395                                                          reloid,
1396                                                          relpathperm(change->data.tp.relnode,
1397                                                                                  MAIN_FORKNUM));
1398
1399                                         if (!RelationIsLogicallyLogged(relation))
1400                                                 goto change_done;
1401
1402                                         /*
1403                                          * Ignore temporary heaps created during DDL unless the
1404                                          * plugin has asked for them.
1405                                          */
1406                                         if (relation->rd_rel->relrewrite && !rb->output_rewrites)
1407                                                 goto change_done;
1408
1409                                         /*
1410                                          * For now ignore sequence changes entirely. Most of the
1411                                          * time they don't log changes using records we
1412                                          * understand, so it doesn't make sense to handle the few
1413                                          * cases we do.
1414                                          */
1415                                         if (relation->rd_rel->relkind == RELKIND_SEQUENCE)
1416                                                 goto change_done;
1417
1418                                         /* user-triggered change */
1419                                         if (!IsToastRelation(relation))
1420                                         {
1421                                                 ReorderBufferToastReplace(rb, txn, relation, change);
1422                                                 rb->apply_change(rb, txn, relation, change);
1423
1424                                                 /*
1425                                                  * Only clear reassembled toast chunks if we're sure
1426                                                  * they're not required anymore. The creator of the
1427                                                  * tuple tells us.
1428                                                  */
1429                                                 if (change->data.tp.clear_toast_afterwards)
1430                                                         ReorderBufferToastReset(rb, txn);
1431                                         }
1432                                         /* we're not interested in toast deletions */
1433                                         else if (change->action == REORDER_BUFFER_CHANGE_INSERT)
1434                                         {
1435                                                 /*
1436                                                  * Need to reassemble the full toasted Datum in
1437                                                  * memory, to ensure the chunks don't get reused till
1438                                                  * we're done remove it from the list of this
1439                                                  * transaction's changes. Otherwise it will get
1440                                                  * freed/reused while restoring spooled data from
1441                                                  * disk.
1442                                                  */
1443                                                 dlist_delete(&change->node);
1444                                                 ReorderBufferToastAppendChunk(rb, txn, relation,
1445                                                                                                           change);
1446                                         }
1447
1448                         change_done:
1449
1450                                         /*
1451                                          * Either speculative insertion was confirmed, or it was
1452                                          * unsuccessful and the record isn't needed anymore.
1453                                          */
1454                                         if (specinsert != NULL)
1455                                         {
1456                                                 ReorderBufferReturnChange(rb, specinsert);
1457                                                 specinsert = NULL;
1458                                         }
1459
1460                                         if (relation != NULL)
1461                                         {
1462                                                 RelationClose(relation);
1463                                                 relation = NULL;
1464                                         }
1465                                         break;
1466
1467                                 case REORDER_BUFFER_CHANGE_INTERNAL_SPEC_INSERT:
1468
1469                                         /*
1470                                          * Speculative insertions are dealt with by delaying the
1471                                          * processing of the insert until the confirmation record
1472                                          * arrives. For that we simply unlink the record from the
1473                                          * chain, so it does not get freed/reused while restoring
1474                                          * spooled data from disk.
1475                                          *
1476                                          * This is safe in the face of concurrent catalog changes
1477                                          * because the relevant relation can't be changed between
1478                                          * speculative insertion and confirmation due to
1479                                          * CheckTableNotInUse() and locking.
1480                                          */
1481
1482                                         /* clear out a pending (and thus failed) speculation */
1483                                         if (specinsert != NULL)
1484                                         {
1485                                                 ReorderBufferReturnChange(rb, specinsert);
1486                                                 specinsert = NULL;
1487                                         }
1488
1489                                         /* and memorize the pending insertion */
1490                                         dlist_delete(&change->node);
1491                                         specinsert = change;
1492                                         break;
1493
1494                                 case REORDER_BUFFER_CHANGE_MESSAGE:
1495                                         rb->message(rb, txn, change->lsn, true,
1496                                                                 change->data.msg.prefix,
1497                                                                 change->data.msg.message_size,
1498                                                                 change->data.msg.message);
1499                                         break;
1500
1501                                 case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
1502                                         /* get rid of the old */
1503                                         TeardownHistoricSnapshot(false);
1504
1505                                         if (snapshot_now->copied)
1506                                         {
1507                                                 ReorderBufferFreeSnap(rb, snapshot_now);
1508                                                 snapshot_now =
1509                                                         ReorderBufferCopySnap(rb, change->data.snapshot,
1510                                                                                                   txn, command_id);
1511                                         }
1512
1513                                         /*
1514                                          * Restored from disk, need to be careful not to double
1515                                          * free. We could introduce refcounting for that, but for
1516                                          * now this seems infrequent enough not to care.
1517                                          */
1518                                         else if (change->data.snapshot->copied)
1519                                         {
1520                                                 snapshot_now =
1521                                                         ReorderBufferCopySnap(rb, change->data.snapshot,
1522                                                                                                   txn, command_id);
1523                                         }
1524                                         else
1525                                         {
1526                                                 snapshot_now = change->data.snapshot;
1527                                         }
1528
1529
1530                                         /* and continue with the new one */
1531                                         SetupHistoricSnapshot(snapshot_now, txn->tuplecid_hash);
1532                                         break;
1533
1534                                 case REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID:
1535                                         Assert(change->data.command_id != InvalidCommandId);
1536
1537                                         if (command_id < change->data.command_id)
1538                                         {
1539                                                 command_id = change->data.command_id;
1540
1541                                                 if (!snapshot_now->copied)
1542                                                 {
1543                                                         /* we don't use the global one anymore */
1544                                                         snapshot_now = ReorderBufferCopySnap(rb, snapshot_now,
1545                                                                                                                                  txn, command_id);
1546                                                 }
1547
1548                                                 snapshot_now->curcid = command_id;
1549
1550                                                 TeardownHistoricSnapshot(false);
1551                                                 SetupHistoricSnapshot(snapshot_now, txn->tuplecid_hash);
1552
1553                                                 /*
1554                                                  * Every time the CommandId is incremented, we could
1555                                                  * see new catalog contents, so execute all
1556                                                  * invalidations.
1557                                                  */
1558                                                 ReorderBufferExecuteInvalidations(rb, txn);
1559                                         }
1560
1561                                         break;
1562
1563                                 case REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID:
1564                                         elog(ERROR, "tuplecid value in changequeue");
1565                                         break;
1566                         }
1567                 }
1568
1569                 /*
1570                  * There's a speculative insertion remaining, just clean in up, it
1571                  * can't have been successful, otherwise we'd gotten a confirmation
1572                  * record.
1573                  */
1574                 if (specinsert)
1575                 {
1576                         ReorderBufferReturnChange(rb, specinsert);
1577                         specinsert = NULL;
1578                 }
1579
1580                 /* clean up the iterator */
1581                 ReorderBufferIterTXNFinish(rb, iterstate);
1582                 iterstate = NULL;
1583
1584                 /* call commit callback */
1585                 rb->commit(rb, txn, commit_lsn);
1586
1587                 /* this is just a sanity check against bad output plugin behaviour */
1588                 if (GetCurrentTransactionIdIfAny() != InvalidTransactionId)
1589                         elog(ERROR, "output plugin used XID %u",
1590                                  GetCurrentTransactionId());
1591
1592                 /* cleanup */
1593                 TeardownHistoricSnapshot(false);
1594
1595                 /*
1596                  * Aborting the current (sub-)transaction as a whole has the right
1597                  * semantics. We want all locks acquired in here to be released, not
1598                  * reassigned to the parent and we do not want any database access
1599                  * have persistent effects.
1600                  */
1601                 AbortCurrentTransaction();
1602
1603                 /* make sure there's no cache pollution */
1604                 ReorderBufferExecuteInvalidations(rb, txn);
1605
1606                 if (using_subtxn)
1607                         RollbackAndReleaseCurrentSubTransaction();
1608
1609                 if (snapshot_now->copied)
1610                         ReorderBufferFreeSnap(rb, snapshot_now);
1611
1612                 /* remove potential on-disk data, and deallocate */
1613                 ReorderBufferCleanupTXN(rb, txn);
1614         }
1615         PG_CATCH();
1616         {
1617                 /* TODO: Encapsulate cleanup from the PG_TRY and PG_CATCH blocks */
1618                 if (iterstate)
1619                         ReorderBufferIterTXNFinish(rb, iterstate);
1620
1621                 TeardownHistoricSnapshot(true);
1622
1623                 /*
1624                  * Force cache invalidation to happen outside of a valid transaction
1625                  * to prevent catalog access as we just caught an error.
1626                  */
1627                 AbortCurrentTransaction();
1628
1629                 /* make sure there's no cache pollution */
1630                 ReorderBufferExecuteInvalidations(rb, txn);
1631
1632                 if (using_subtxn)
1633                         RollbackAndReleaseCurrentSubTransaction();
1634
1635                 if (snapshot_now->copied)
1636                         ReorderBufferFreeSnap(rb, snapshot_now);
1637
1638                 /* remove potential on-disk data, and deallocate */
1639                 ReorderBufferCleanupTXN(rb, txn);
1640
1641                 PG_RE_THROW();
1642         }
1643         PG_END_TRY();
1644 }
1645
1646 /*
1647  * Abort a transaction that possibly has previous changes. Needs to be first
1648  * called for subtransactions and then for the toplevel xid.
1649  *
1650  * NB: Transactions handled here have to have actively aborted (i.e. have
1651  * produced an abort record). Implicitly aborted transactions are handled via
1652  * ReorderBufferAbortOld(); transactions we're just not interested in, but
1653  * which have committed are handled in ReorderBufferForget().
1654  *
1655  * This function purges this transaction and its contents from memory and
1656  * disk.
1657  */
1658 void
1659 ReorderBufferAbort(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn)
1660 {
1661         ReorderBufferTXN *txn;
1662
1663         txn = ReorderBufferTXNByXid(rb, xid, false, NULL, InvalidXLogRecPtr,
1664                                                                 false);
1665
1666         /* unknown, nothing to remove */
1667         if (txn == NULL)
1668                 return;
1669
1670         /* cosmetic... */
1671         txn->final_lsn = lsn;
1672
1673         /* remove potential on-disk data, and deallocate */
1674         ReorderBufferCleanupTXN(rb, txn);
1675 }
1676
1677 /*
1678  * Abort all transactions that aren't actually running anymore because the
1679  * server restarted.
1680  *
1681  * NB: These really have to be transactions that have aborted due to a server
1682  * crash/immediate restart, as we don't deal with invalidations here.
1683  */
1684 void
1685 ReorderBufferAbortOld(ReorderBuffer *rb, TransactionId oldestRunningXid)
1686 {
1687         dlist_mutable_iter it;
1688
1689         /*
1690          * Iterate through all (potential) toplevel TXNs and abort all that are
1691          * older than what possibly can be running. Once we've found the first
1692          * that is alive we stop, there might be some that acquired an xid earlier
1693          * but started writing later, but it's unlikely and they will be cleaned
1694          * up in a later call to this function.
1695          */
1696         dlist_foreach_modify(it, &rb->toplevel_by_lsn)
1697         {
1698                 ReorderBufferTXN *txn;
1699
1700                 txn = dlist_container(ReorderBufferTXN, node, it.cur);
1701
1702                 if (TransactionIdPrecedes(txn->xid, oldestRunningXid))
1703                 {
1704                         /*
1705                          * We set final_lsn on a transaction when we decode its commit or
1706                          * abort record, but we never see those records for crashed
1707                          * transactions.  To ensure cleanup of these transactions, set
1708                          * final_lsn to that of their last change; this causes
1709                          * ReorderBufferRestoreCleanup to do the right thing.
1710                          */
1711                         if (txn->serialized && txn->final_lsn == 0)
1712                         {
1713                                 ReorderBufferChange *last =
1714                                         dlist_tail_element(ReorderBufferChange, node, &txn->changes);
1715
1716                                 txn->final_lsn = last->lsn;
1717                         }
1718
1719                         elog(DEBUG2, "aborting old transaction %u", txn->xid);
1720
1721                         /* remove potential on-disk data, and deallocate this tx */
1722                         ReorderBufferCleanupTXN(rb, txn);
1723                 }
1724                 else
1725                         return;
1726         }
1727 }
1728
1729 /*
1730  * Forget the contents of a transaction if we aren't interested in it's
1731  * contents. Needs to be first called for subtransactions and then for the
1732  * toplevel xid.
1733  *
1734  * This is significantly different to ReorderBufferAbort() because
1735  * transactions that have committed need to be treated differently from aborted
1736  * ones since they may have modified the catalog.
1737  *
1738  * Note that this is only allowed to be called in the moment a transaction
1739  * commit has just been read, not earlier; otherwise later records referring
1740  * to this xid might re-create the transaction incompletely.
1741  */
1742 void
1743 ReorderBufferForget(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn)
1744 {
1745         ReorderBufferTXN *txn;
1746
1747         txn = ReorderBufferTXNByXid(rb, xid, false, NULL, InvalidXLogRecPtr,
1748                                                                 false);
1749
1750         /* unknown, nothing to forget */
1751         if (txn == NULL)
1752                 return;
1753
1754         /* cosmetic... */
1755         txn->final_lsn = lsn;
1756
1757         /*
1758          * Process cache invalidation messages if there are any. Even if we're not
1759          * interested in the transaction's contents, it could have manipulated the
1760          * catalog and we need to update the caches according to that.
1761          */
1762         if (txn->base_snapshot != NULL && txn->ninvalidations > 0)
1763                 ReorderBufferImmediateInvalidation(rb, txn->ninvalidations,
1764                                                                                    txn->invalidations);
1765         else
1766                 Assert(txn->ninvalidations == 0);
1767
1768         /* remove potential on-disk data, and deallocate */
1769         ReorderBufferCleanupTXN(rb, txn);
1770 }
1771
1772 /*
1773  * Execute invalidations happening outside the context of a decoded
1774  * transaction. That currently happens either for xid-less commits
1775  * (cf. RecordTransactionCommit()) or for invalidations in uninteresting
1776  * transactions (via ReorderBufferForget()).
1777  */
1778 void
1779 ReorderBufferImmediateInvalidation(ReorderBuffer *rb, uint32 ninvalidations,
1780                                                                    SharedInvalidationMessage *invalidations)
1781 {
1782         bool            use_subtxn = IsTransactionOrTransactionBlock();
1783         int                     i;
1784
1785         if (use_subtxn)
1786                 BeginInternalSubTransaction("replay");
1787
1788         /*
1789          * Force invalidations to happen outside of a valid transaction - that way
1790          * entries will just be marked as invalid without accessing the catalog.
1791          * That's advantageous because we don't need to setup the full state
1792          * necessary for catalog access.
1793          */
1794         if (use_subtxn)
1795                 AbortCurrentTransaction();
1796
1797         for (i = 0; i < ninvalidations; i++)
1798                 LocalExecuteInvalidationMessage(&invalidations[i]);
1799
1800         if (use_subtxn)
1801                 RollbackAndReleaseCurrentSubTransaction();
1802 }
1803
1804 /*
1805  * Tell reorderbuffer about an xid seen in the WAL stream. Has to be called at
1806  * least once for every xid in XLogRecord->xl_xid (other places in records
1807  * may, but do not have to be passed through here).
1808  *
1809  * Reorderbuffer keeps some datastructures about transactions in LSN order,
1810  * for efficiency. To do that it has to know about when transactions are seen
1811  * first in the WAL. As many types of records are not actually interesting for
1812  * logical decoding, they do not necessarily pass though here.
1813  */
1814 void
1815 ReorderBufferProcessXid(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn)
1816 {
1817         /* many records won't have an xid assigned, centralize check here */
1818         if (xid != InvalidTransactionId)
1819                 ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
1820 }
1821
1822 /*
1823  * Add a new snapshot to this transaction that may only used after lsn 'lsn'
1824  * because the previous snapshot doesn't describe the catalog correctly for
1825  * following rows.
1826  */
1827 void
1828 ReorderBufferAddSnapshot(ReorderBuffer *rb, TransactionId xid,
1829                                                  XLogRecPtr lsn, Snapshot snap)
1830 {
1831         ReorderBufferChange *change = ReorderBufferGetChange(rb);
1832
1833         change->data.snapshot = snap;
1834         change->action = REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT;
1835
1836         ReorderBufferQueueChange(rb, xid, lsn, change);
1837 }
1838
1839 /*
1840  * Setup the base snapshot of a transaction. The base snapshot is the snapshot
1841  * that is used to decode all changes until either this transaction modifies
1842  * the catalog or another catalog modifying transaction commits.
1843  *
1844  * Needs to be called before any changes are added with
1845  * ReorderBufferQueueChange().
1846  */
1847 void
1848 ReorderBufferSetBaseSnapshot(ReorderBuffer *rb, TransactionId xid,
1849                                                          XLogRecPtr lsn, Snapshot snap)
1850 {
1851         ReorderBufferTXN *txn;
1852         bool            is_new;
1853
1854         txn = ReorderBufferTXNByXid(rb, xid, true, &is_new, lsn, true);
1855         Assert(txn->base_snapshot == NULL);
1856         Assert(snap != NULL);
1857
1858         txn->base_snapshot = snap;
1859         txn->base_snapshot_lsn = lsn;
1860 }
1861
1862 /*
1863  * Access the catalog with this CommandId at this point in the changestream.
1864  *
1865  * May only be called for command ids > 1
1866  */
1867 void
1868 ReorderBufferAddNewCommandId(ReorderBuffer *rb, TransactionId xid,
1869                                                          XLogRecPtr lsn, CommandId cid)
1870 {
1871         ReorderBufferChange *change = ReorderBufferGetChange(rb);
1872
1873         change->data.command_id = cid;
1874         change->action = REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID;
1875
1876         ReorderBufferQueueChange(rb, xid, lsn, change);
1877 }
1878
1879
1880 /*
1881  * Add new (relfilenode, tid) -> (cmin, cmax) mappings.
1882  */
1883 void
1884 ReorderBufferAddNewTupleCids(ReorderBuffer *rb, TransactionId xid,
1885                                                          XLogRecPtr lsn, RelFileNode node,
1886                                                          ItemPointerData tid, CommandId cmin,
1887                                                          CommandId cmax, CommandId combocid)
1888 {
1889         ReorderBufferChange *change = ReorderBufferGetChange(rb);
1890         ReorderBufferTXN *txn;
1891
1892         txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
1893
1894         change->data.tuplecid.node = node;
1895         change->data.tuplecid.tid = tid;
1896         change->data.tuplecid.cmin = cmin;
1897         change->data.tuplecid.cmax = cmax;
1898         change->data.tuplecid.combocid = combocid;
1899         change->lsn = lsn;
1900         change->action = REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID;
1901
1902         dlist_push_tail(&txn->tuplecids, &change->node);
1903         txn->ntuplecids++;
1904 }
1905
1906 /*
1907  * Setup the invalidation of the toplevel transaction.
1908  *
1909  * This needs to be done before ReorderBufferCommit is called!
1910  */
1911 void
1912 ReorderBufferAddInvalidations(ReorderBuffer *rb, TransactionId xid,
1913                                                           XLogRecPtr lsn, Size nmsgs,
1914                                                           SharedInvalidationMessage *msgs)
1915 {
1916         ReorderBufferTXN *txn;
1917
1918         txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
1919
1920         if (txn->ninvalidations != 0)
1921                 elog(ERROR, "only ever add one set of invalidations");
1922
1923         Assert(nmsgs > 0);
1924
1925         txn->ninvalidations = nmsgs;
1926         txn->invalidations = (SharedInvalidationMessage *)
1927                 MemoryContextAlloc(rb->context,
1928                                                    sizeof(SharedInvalidationMessage) * nmsgs);
1929         memcpy(txn->invalidations, msgs,
1930                    sizeof(SharedInvalidationMessage) * nmsgs);
1931 }
1932
1933 /*
1934  * Apply all invalidations we know. Possibly we only need parts at this point
1935  * in the changestream but we don't know which those are.
1936  */
1937 static void
1938 ReorderBufferExecuteInvalidations(ReorderBuffer *rb, ReorderBufferTXN *txn)
1939 {
1940         int                     i;
1941
1942         for (i = 0; i < txn->ninvalidations; i++)
1943                 LocalExecuteInvalidationMessage(&txn->invalidations[i]);
1944 }
1945
1946 /*
1947  * Mark a transaction as containing catalog changes
1948  */
1949 void
1950 ReorderBufferXidSetCatalogChanges(ReorderBuffer *rb, TransactionId xid,
1951                                                                   XLogRecPtr lsn)
1952 {
1953         ReorderBufferTXN *txn;
1954
1955         txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
1956
1957         txn->has_catalog_changes = true;
1958 }
1959
1960 /*
1961  * Query whether a transaction is already *known* to contain catalog
1962  * changes. This can be wrong until directly before the commit!
1963  */
1964 bool
1965 ReorderBufferXidHasCatalogChanges(ReorderBuffer *rb, TransactionId xid)
1966 {
1967         ReorderBufferTXN *txn;
1968
1969         txn = ReorderBufferTXNByXid(rb, xid, false, NULL, InvalidXLogRecPtr,
1970                                                                 false);
1971         if (txn == NULL)
1972                 return false;
1973
1974         return txn->has_catalog_changes;
1975 }
1976
1977 /*
1978  * Have we already added the first snapshot?
1979  */
1980 bool
1981 ReorderBufferXidHasBaseSnapshot(ReorderBuffer *rb, TransactionId xid)
1982 {
1983         ReorderBufferTXN *txn;
1984
1985         txn = ReorderBufferTXNByXid(rb, xid, false, NULL, InvalidXLogRecPtr,
1986                                                                 false);
1987
1988         /* transaction isn't known yet, ergo no snapshot */
1989         if (txn == NULL)
1990                 return false;
1991
1992         /*
1993          * TODO: It would be a nice improvement if we would check the toplevel
1994          * transaction in subtransactions, but we'd need to keep track of a bit
1995          * more state.
1996          */
1997         return txn->base_snapshot != NULL;
1998 }
1999
2000
2001 /*
2002  * ---------------------------------------
2003  * Disk serialization support
2004  * ---------------------------------------
2005  */
2006
2007 /*
2008  * Ensure the IO buffer is >= sz.
2009  */
2010 static void
2011 ReorderBufferSerializeReserve(ReorderBuffer *rb, Size sz)
2012 {
2013         if (!rb->outbufsize)
2014         {
2015                 rb->outbuf = MemoryContextAlloc(rb->context, sz);
2016                 rb->outbufsize = sz;
2017         }
2018         else if (rb->outbufsize < sz)
2019         {
2020                 rb->outbuf = repalloc(rb->outbuf, sz);
2021                 rb->outbufsize = sz;
2022         }
2023 }
2024
2025 /*
2026  * Check whether the transaction tx should spill its data to disk.
2027  */
2028 static void
2029 ReorderBufferCheckSerializeTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
2030 {
2031         /*
2032          * TODO: improve accounting so we cheaply can take subtransactions into
2033          * account here.
2034          */
2035         if (txn->nentries_mem >= max_changes_in_memory)
2036         {
2037                 ReorderBufferSerializeTXN(rb, txn);
2038                 Assert(txn->nentries_mem == 0);
2039         }
2040 }
2041
2042 /*
2043  * Spill data of a large transaction (and its subtransactions) to disk.
2044  */
2045 static void
2046 ReorderBufferSerializeTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
2047 {
2048         dlist_iter      subtxn_i;
2049         dlist_mutable_iter change_i;
2050         int                     fd = -1;
2051         XLogSegNo       curOpenSegNo = 0;
2052         Size            spilled = 0;
2053
2054         elog(DEBUG2, "spill %u changes in XID %u to disk",
2055                  (uint32) txn->nentries_mem, txn->xid);
2056
2057         /* do the same to all child TXs */
2058         dlist_foreach(subtxn_i, &txn->subtxns)
2059         {
2060                 ReorderBufferTXN *subtxn;
2061
2062                 subtxn = dlist_container(ReorderBufferTXN, node, subtxn_i.cur);
2063                 ReorderBufferSerializeTXN(rb, subtxn);
2064         }
2065
2066         /* serialize changestream */
2067         dlist_foreach_modify(change_i, &txn->changes)
2068         {
2069                 ReorderBufferChange *change;
2070
2071                 change = dlist_container(ReorderBufferChange, node, change_i.cur);
2072
2073                 /*
2074                  * store in segment in which it belongs by start lsn, don't split over
2075                  * multiple segments tho
2076                  */
2077                 if (fd == -1 ||
2078                         !XLByteInSeg(change->lsn, curOpenSegNo, wal_segment_size))
2079                 {
2080                         char            path[MAXPGPATH];
2081
2082                         if (fd != -1)
2083                                 CloseTransientFile(fd);
2084
2085                         XLByteToSeg(change->lsn, curOpenSegNo, wal_segment_size);
2086
2087                         /*
2088                          * No need to care about TLIs here, only used during a single run,
2089                          * so each LSN only maps to a specific WAL record.
2090                          */
2091                         ReorderBufferSerializedPath(path, MyReplicationSlot, txn->xid,
2092                                                                                 curOpenSegNo);
2093
2094                         /* open segment, create it if necessary */
2095                         fd = OpenTransientFile(path,
2096                                                                    O_CREAT | O_WRONLY | O_APPEND | PG_BINARY);
2097
2098                         if (fd < 0)
2099                                 ereport(ERROR,
2100                                                 (errcode_for_file_access(),
2101                                                  errmsg("could not open file \"%s\": %m", path)));
2102                 }
2103
2104                 ReorderBufferSerializeChange(rb, txn, fd, change);
2105                 dlist_delete(&change->node);
2106                 ReorderBufferReturnChange(rb, change);
2107
2108                 spilled++;
2109         }
2110
2111         Assert(spilled == txn->nentries_mem);
2112         Assert(dlist_is_empty(&txn->changes));
2113         txn->nentries_mem = 0;
2114         txn->serialized = true;
2115
2116         if (fd != -1)
2117                 CloseTransientFile(fd);
2118 }
2119
2120 /*
2121  * Serialize individual change to disk.
2122  */
2123 static void
2124 ReorderBufferSerializeChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
2125                                                          int fd, ReorderBufferChange *change)
2126 {
2127         ReorderBufferDiskChange *ondisk;
2128         Size            sz = sizeof(ReorderBufferDiskChange);
2129
2130         ReorderBufferSerializeReserve(rb, sz);
2131
2132         ondisk = (ReorderBufferDiskChange *) rb->outbuf;
2133         memcpy(&ondisk->change, change, sizeof(ReorderBufferChange));
2134
2135         switch (change->action)
2136         {
2137                         /* fall through these, they're all similar enough */
2138                 case REORDER_BUFFER_CHANGE_INSERT:
2139                 case REORDER_BUFFER_CHANGE_UPDATE:
2140                 case REORDER_BUFFER_CHANGE_DELETE:
2141                 case REORDER_BUFFER_CHANGE_INTERNAL_SPEC_INSERT:
2142                         {
2143                                 char       *data;
2144                                 ReorderBufferTupleBuf *oldtup,
2145                                                    *newtup;
2146                                 Size            oldlen = 0;
2147                                 Size            newlen = 0;
2148
2149                                 oldtup = change->data.tp.oldtuple;
2150                                 newtup = change->data.tp.newtuple;
2151
2152                                 if (oldtup)
2153                                 {
2154                                         sz += sizeof(HeapTupleData);
2155                                         oldlen = oldtup->tuple.t_len;
2156                                         sz += oldlen;
2157                                 }
2158
2159                                 if (newtup)
2160                                 {
2161                                         sz += sizeof(HeapTupleData);
2162                                         newlen = newtup->tuple.t_len;
2163                                         sz += newlen;
2164                                 }
2165
2166                                 /* make sure we have enough space */
2167                                 ReorderBufferSerializeReserve(rb, sz);
2168
2169                                 data = ((char *) rb->outbuf) + sizeof(ReorderBufferDiskChange);
2170                                 /* might have been reallocated above */
2171                                 ondisk = (ReorderBufferDiskChange *) rb->outbuf;
2172
2173                                 if (oldlen)
2174                                 {
2175                                         memcpy(data, &oldtup->tuple, sizeof(HeapTupleData));
2176                                         data += sizeof(HeapTupleData);
2177
2178                                         memcpy(data, oldtup->tuple.t_data, oldlen);
2179                                         data += oldlen;
2180                                 }
2181
2182                                 if (newlen)
2183                                 {
2184                                         memcpy(data, &newtup->tuple, sizeof(HeapTupleData));
2185                                         data += sizeof(HeapTupleData);
2186
2187                                         memcpy(data, newtup->tuple.t_data, newlen);
2188                                         data += newlen;
2189                                 }
2190                                 break;
2191                         }
2192                 case REORDER_BUFFER_CHANGE_MESSAGE:
2193                         {
2194                                 char       *data;
2195                                 Size            prefix_size = strlen(change->data.msg.prefix) + 1;
2196
2197                                 sz += prefix_size + change->data.msg.message_size +
2198                                         sizeof(Size) + sizeof(Size);
2199                                 ReorderBufferSerializeReserve(rb, sz);
2200
2201                                 data = ((char *) rb->outbuf) + sizeof(ReorderBufferDiskChange);
2202
2203                                 /* might have been reallocated above */
2204                                 ondisk = (ReorderBufferDiskChange *) rb->outbuf;
2205
2206                                 /* write the prefix including the size */
2207                                 memcpy(data, &prefix_size, sizeof(Size));
2208                                 data += sizeof(Size);
2209                                 memcpy(data, change->data.msg.prefix,
2210                                            prefix_size);
2211                                 data += prefix_size;
2212
2213                                 /* write the message including the size */
2214                                 memcpy(data, &change->data.msg.message_size, sizeof(Size));
2215                                 data += sizeof(Size);
2216                                 memcpy(data, change->data.msg.message,
2217                                            change->data.msg.message_size);
2218                                 data += change->data.msg.message_size;
2219
2220                                 break;
2221                         }
2222                 case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
2223                         {
2224                                 Snapshot        snap;
2225                                 char       *data;
2226
2227                                 snap = change->data.snapshot;
2228
2229                                 sz += sizeof(SnapshotData) +
2230                                         sizeof(TransactionId) * snap->xcnt +
2231                                         sizeof(TransactionId) * snap->subxcnt
2232                                         ;
2233
2234                                 /* make sure we have enough space */
2235                                 ReorderBufferSerializeReserve(rb, sz);
2236                                 data = ((char *) rb->outbuf) + sizeof(ReorderBufferDiskChange);
2237                                 /* might have been reallocated above */
2238                                 ondisk = (ReorderBufferDiskChange *) rb->outbuf;
2239
2240                                 memcpy(data, snap, sizeof(SnapshotData));
2241                                 data += sizeof(SnapshotData);
2242
2243                                 if (snap->xcnt)
2244                                 {
2245                                         memcpy(data, snap->xip,
2246                                                    sizeof(TransactionId) * snap->xcnt);
2247                                         data += sizeof(TransactionId) * snap->xcnt;
2248                                 }
2249
2250                                 if (snap->subxcnt)
2251                                 {
2252                                         memcpy(data, snap->subxip,
2253                                                    sizeof(TransactionId) * snap->subxcnt);
2254                                         data += sizeof(TransactionId) * snap->subxcnt;
2255                                 }
2256                                 break;
2257                         }
2258                 case REORDER_BUFFER_CHANGE_INTERNAL_SPEC_CONFIRM:
2259                 case REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID:
2260                 case REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID:
2261                         /* ReorderBufferChange contains everything important */
2262                         break;
2263         }
2264
2265         ondisk->size = sz;
2266
2267         pgstat_report_wait_start(WAIT_EVENT_REORDER_BUFFER_WRITE);
2268         if (write(fd, rb->outbuf, ondisk->size) != ondisk->size)
2269         {
2270                 int                     save_errno = errno;
2271
2272                 CloseTransientFile(fd);
2273                 errno = save_errno;
2274                 ereport(ERROR,
2275                                 (errcode_for_file_access(),
2276                                  errmsg("could not write to data file for XID %u: %m",
2277                                                 txn->xid)));
2278         }
2279         pgstat_report_wait_end();
2280
2281         Assert(ondisk->change.action == change->action);
2282 }
2283
2284 /*
2285  * Restore a number of changes spilled to disk back into memory.
2286  */
2287 static Size
2288 ReorderBufferRestoreChanges(ReorderBuffer *rb, ReorderBufferTXN *txn,
2289                                                         int *fd, XLogSegNo *segno)
2290 {
2291         Size            restored = 0;
2292         XLogSegNo       last_segno;
2293         dlist_mutable_iter cleanup_iter;
2294
2295         Assert(txn->first_lsn != InvalidXLogRecPtr);
2296         Assert(txn->final_lsn != InvalidXLogRecPtr);
2297
2298         /* free current entries, so we have memory for more */
2299         dlist_foreach_modify(cleanup_iter, &txn->changes)
2300         {
2301                 ReorderBufferChange *cleanup =
2302                 dlist_container(ReorderBufferChange, node, cleanup_iter.cur);
2303
2304                 dlist_delete(&cleanup->node);
2305                 ReorderBufferReturnChange(rb, cleanup);
2306         }
2307         txn->nentries_mem = 0;
2308         Assert(dlist_is_empty(&txn->changes));
2309
2310         XLByteToSeg(txn->final_lsn, last_segno, wal_segment_size);
2311
2312         while (restored < max_changes_in_memory && *segno <= last_segno)
2313         {
2314                 int                     readBytes;
2315                 ReorderBufferDiskChange *ondisk;
2316
2317                 if (*fd == -1)
2318                 {
2319                         char            path[MAXPGPATH];
2320
2321                         /* first time in */
2322                         if (*segno == 0)
2323                                 XLByteToSeg(txn->first_lsn, *segno, wal_segment_size);
2324
2325                         Assert(*segno != 0 || dlist_is_empty(&txn->changes));
2326
2327                         /*
2328                          * No need to care about TLIs here, only used during a single run,
2329                          * so each LSN only maps to a specific WAL record.
2330                          */
2331                         ReorderBufferSerializedPath(path, MyReplicationSlot, txn->xid,
2332                                                                                 *segno);
2333
2334                         *fd = OpenTransientFile(path, O_RDONLY | PG_BINARY);
2335                         if (*fd < 0 && errno == ENOENT)
2336                         {
2337                                 *fd = -1;
2338                                 (*segno)++;
2339                                 continue;
2340                         }
2341                         else if (*fd < 0)
2342                                 ereport(ERROR,
2343                                                 (errcode_for_file_access(),
2344                                                  errmsg("could not open file \"%s\": %m",
2345                                                                 path)));
2346                 }
2347
2348                 /*
2349                  * Read the statically sized part of a change which has information
2350                  * about the total size. If we couldn't read a record, we're at the
2351                  * end of this file.
2352                  */
2353                 ReorderBufferSerializeReserve(rb, sizeof(ReorderBufferDiskChange));
2354                 pgstat_report_wait_start(WAIT_EVENT_REORDER_BUFFER_READ);
2355                 readBytes = read(*fd, rb->outbuf, sizeof(ReorderBufferDiskChange));
2356                 pgstat_report_wait_end();
2357
2358                 /* eof */
2359                 if (readBytes == 0)
2360                 {
2361                         CloseTransientFile(*fd);
2362                         *fd = -1;
2363                         (*segno)++;
2364                         continue;
2365                 }
2366                 else if (readBytes < 0)
2367                         ereport(ERROR,
2368                                         (errcode_for_file_access(),
2369                                          errmsg("could not read from reorderbuffer spill file: %m")));
2370                 else if (readBytes != sizeof(ReorderBufferDiskChange))
2371                         ereport(ERROR,
2372                                         (errcode_for_file_access(),
2373                                          errmsg("could not read from reorderbuffer spill file: read %d instead of %u bytes",
2374                                                         readBytes,
2375                                                         (uint32) sizeof(ReorderBufferDiskChange))));
2376
2377                 ondisk = (ReorderBufferDiskChange *) rb->outbuf;
2378
2379                 ReorderBufferSerializeReserve(rb,
2380                                                                           sizeof(ReorderBufferDiskChange) + ondisk->size);
2381                 ondisk = (ReorderBufferDiskChange *) rb->outbuf;
2382
2383                 pgstat_report_wait_start(WAIT_EVENT_REORDER_BUFFER_READ);
2384                 readBytes = read(*fd, rb->outbuf + sizeof(ReorderBufferDiskChange),
2385                                                  ondisk->size - sizeof(ReorderBufferDiskChange));
2386                 pgstat_report_wait_end();
2387
2388                 if (readBytes < 0)
2389                         ereport(ERROR,
2390                                         (errcode_for_file_access(),
2391                                          errmsg("could not read from reorderbuffer spill file: %m")));
2392                 else if (readBytes != ondisk->size - sizeof(ReorderBufferDiskChange))
2393                         ereport(ERROR,
2394                                         (errcode_for_file_access(),
2395                                          errmsg("could not read from reorderbuffer spill file: read %d instead of %u bytes",
2396                                                         readBytes,
2397                                                         (uint32) (ondisk->size - sizeof(ReorderBufferDiskChange)))));
2398
2399                 /*
2400                  * ok, read a full change from disk, now restore it into proper
2401                  * in-memory format
2402                  */
2403                 ReorderBufferRestoreChange(rb, txn, rb->outbuf);
2404                 restored++;
2405         }
2406
2407         return restored;
2408 }
2409
2410 /*
2411  * Convert change from its on-disk format to in-memory format and queue it onto
2412  * the TXN's ->changes list.
2413  *
2414  * Note: although "data" is declared char*, at entry it points to a
2415  * maxalign'd buffer, making it safe in most of this function to assume
2416  * that the pointed-to data is suitably aligned for direct access.
2417  */
2418 static void
2419 ReorderBufferRestoreChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
2420                                                    char *data)
2421 {
2422         ReorderBufferDiskChange *ondisk;
2423         ReorderBufferChange *change;
2424
2425         ondisk = (ReorderBufferDiskChange *) data;
2426
2427         change = ReorderBufferGetChange(rb);
2428
2429         /* copy static part */
2430         memcpy(change, &ondisk->change, sizeof(ReorderBufferChange));
2431
2432         data += sizeof(ReorderBufferDiskChange);
2433
2434         /* restore individual stuff */
2435         switch (change->action)
2436         {
2437                         /* fall through these, they're all similar enough */
2438                 case REORDER_BUFFER_CHANGE_INSERT:
2439                 case REORDER_BUFFER_CHANGE_UPDATE:
2440                 case REORDER_BUFFER_CHANGE_DELETE:
2441                 case REORDER_BUFFER_CHANGE_INTERNAL_SPEC_INSERT:
2442                         if (change->data.tp.oldtuple)
2443                         {
2444                                 uint32          tuplelen = ((HeapTuple) data)->t_len;
2445
2446                                 change->data.tp.oldtuple =
2447                                         ReorderBufferGetTupleBuf(rb, tuplelen - SizeofHeapTupleHeader);
2448
2449                                 /* restore ->tuple */
2450                                 memcpy(&change->data.tp.oldtuple->tuple, data,
2451                                            sizeof(HeapTupleData));
2452                                 data += sizeof(HeapTupleData);
2453
2454                                 /* reset t_data pointer into the new tuplebuf */
2455                                 change->data.tp.oldtuple->tuple.t_data =
2456                                         ReorderBufferTupleBufData(change->data.tp.oldtuple);
2457
2458                                 /* restore tuple data itself */
2459                                 memcpy(change->data.tp.oldtuple->tuple.t_data, data, tuplelen);
2460                                 data += tuplelen;
2461                         }
2462
2463                         if (change->data.tp.newtuple)
2464                         {
2465                                 /* here, data might not be suitably aligned! */
2466                                 uint32          tuplelen;
2467
2468                                 memcpy(&tuplelen, data + offsetof(HeapTupleData, t_len),
2469                                            sizeof(uint32));
2470
2471                                 change->data.tp.newtuple =
2472                                         ReorderBufferGetTupleBuf(rb, tuplelen - SizeofHeapTupleHeader);
2473
2474                                 /* restore ->tuple */
2475                                 memcpy(&change->data.tp.newtuple->tuple, data,
2476                                            sizeof(HeapTupleData));
2477                                 data += sizeof(HeapTupleData);
2478
2479                                 /* reset t_data pointer into the new tuplebuf */
2480                                 change->data.tp.newtuple->tuple.t_data =
2481                                         ReorderBufferTupleBufData(change->data.tp.newtuple);
2482
2483                                 /* restore tuple data itself */
2484                                 memcpy(change->data.tp.newtuple->tuple.t_data, data, tuplelen);
2485                                 data += tuplelen;
2486                         }
2487
2488                         break;
2489                 case REORDER_BUFFER_CHANGE_MESSAGE:
2490                         {
2491                                 Size            prefix_size;
2492
2493                                 /* read prefix */
2494                                 memcpy(&prefix_size, data, sizeof(Size));
2495                                 data += sizeof(Size);
2496                                 change->data.msg.prefix = MemoryContextAlloc(rb->context,
2497                                                                                                                          prefix_size);
2498                                 memcpy(change->data.msg.prefix, data, prefix_size);
2499                                 Assert(change->data.msg.prefix[prefix_size - 1] == '\0');
2500                                 data += prefix_size;
2501
2502                                 /* read the message */
2503                                 memcpy(&change->data.msg.message_size, data, sizeof(Size));
2504                                 data += sizeof(Size);
2505                                 change->data.msg.message = MemoryContextAlloc(rb->context,
2506                                                                                                                           change->data.msg.message_size);
2507                                 memcpy(change->data.msg.message, data,
2508                                            change->data.msg.message_size);
2509                                 data += change->data.msg.message_size;
2510
2511                                 break;
2512                         }
2513                 case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
2514                         {
2515                                 Snapshot        oldsnap;
2516                                 Snapshot        newsnap;
2517                                 Size            size;
2518
2519                                 oldsnap = (Snapshot) data;
2520
2521                                 size = sizeof(SnapshotData) +
2522                                         sizeof(TransactionId) * oldsnap->xcnt +
2523                                         sizeof(TransactionId) * (oldsnap->subxcnt + 0);
2524
2525                                 change->data.snapshot = MemoryContextAllocZero(rb->context, size);
2526
2527                                 newsnap = change->data.snapshot;
2528
2529                                 memcpy(newsnap, data, size);
2530                                 newsnap->xip = (TransactionId *)
2531                                         (((char *) newsnap) + sizeof(SnapshotData));
2532                                 newsnap->subxip = newsnap->xip + newsnap->xcnt;
2533                                 newsnap->copied = true;
2534                                 break;
2535                         }
2536                         /* the base struct contains all the data, easy peasy */
2537                 case REORDER_BUFFER_CHANGE_INTERNAL_SPEC_CONFIRM:
2538                 case REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID:
2539                 case REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID:
2540                         break;
2541         }
2542
2543         dlist_push_tail(&txn->changes, &change->node);
2544         txn->nentries_mem++;
2545 }
2546
2547 /*
2548  * Remove all on-disk stored for the passed in transaction.
2549  */
2550 static void
2551 ReorderBufferRestoreCleanup(ReorderBuffer *rb, ReorderBufferTXN *txn)
2552 {
2553         XLogSegNo       first;
2554         XLogSegNo       cur;
2555         XLogSegNo       last;
2556
2557         Assert(txn->first_lsn != InvalidXLogRecPtr);
2558         Assert(txn->final_lsn != InvalidXLogRecPtr);
2559
2560         XLByteToSeg(txn->first_lsn, first, wal_segment_size);
2561         XLByteToSeg(txn->final_lsn, last, wal_segment_size);
2562
2563         /* iterate over all possible filenames, and delete them */
2564         for (cur = first; cur <= last; cur++)
2565         {
2566                 char            path[MAXPGPATH];
2567
2568                 ReorderBufferSerializedPath(path, MyReplicationSlot, txn->xid, cur);
2569                 if (unlink(path) != 0 && errno != ENOENT)
2570                         ereport(ERROR,
2571                                         (errcode_for_file_access(),
2572                                          errmsg("could not remove file \"%s\": %m", path)));
2573         }
2574 }
2575
2576 /*
2577  * Remove any leftover serialized reorder buffers from a slot directory after a
2578  * prior crash or decoding session exit.
2579  */
2580 static void
2581 ReorderBufferCleanupSerializedTXNs(const char *slotname)
2582 {
2583         DIR                *spill_dir;
2584         struct dirent *spill_de;
2585         struct stat statbuf;
2586         char            path[MAXPGPATH * 2 + 12];
2587
2588         sprintf(path, "pg_replslot/%s", slotname);
2589
2590         /* we're only handling directories here, skip if it's not ours */
2591         if (lstat(path, &statbuf) == 0 && !S_ISDIR(statbuf.st_mode))
2592                 return;
2593
2594         spill_dir = AllocateDir(path);
2595         while ((spill_de = ReadDirExtended(spill_dir, path, INFO)) != NULL)
2596         {
2597                 /* only look at names that can be ours */
2598                 if (strncmp(spill_de->d_name, "xid", 3) == 0)
2599                 {
2600                         snprintf(path, sizeof(path),
2601                                          "pg_replslot/%s/%s", slotname,
2602                                          spill_de->d_name);
2603
2604                         if (unlink(path) != 0)
2605                                 ereport(ERROR,
2606                                                 (errcode_for_file_access(),
2607                                                  errmsg("could not remove file \"%s\" during removal of pg_replslot/%s/*.xid: %m",
2608                                                                 path, slotname)));
2609                 }
2610         }
2611         FreeDir(spill_dir);
2612 }
2613
2614 /*
2615  * Given a replication slot, transaction ID and segment number, fill in the
2616  * corresponding spill file into 'path', which is a caller-owned buffer of size
2617  * at least MAXPGPATH.
2618  */
2619 static void
2620 ReorderBufferSerializedPath(char *path, ReplicationSlot *slot, TransactionId xid,
2621                                                         XLogSegNo segno)
2622 {
2623         XLogRecPtr      recptr;
2624
2625         XLogSegNoOffsetToRecPtr(segno, 0, recptr, wal_segment_size);
2626
2627         snprintf(path, MAXPGPATH, "pg_replslot/%s/xid-%u-lsn-%X-%X.snap",
2628                         NameStr(MyReplicationSlot->data.name),
2629                         xid,
2630                         (uint32) (recptr >> 32), (uint32) recptr);
2631 }
2632
2633 /*
2634  * Delete all data spilled to disk after we've restarted/crashed. It will be
2635  * recreated when the respective slots are reused.
2636  */
2637 void
2638 StartupReorderBuffer(void)
2639 {
2640         DIR                *logical_dir;
2641         struct dirent *logical_de;
2642
2643         logical_dir = AllocateDir("pg_replslot");
2644         while ((logical_de = ReadDir(logical_dir, "pg_replslot")) != NULL)
2645         {
2646                 if (strcmp(logical_de->d_name, ".") == 0 ||
2647                         strcmp(logical_de->d_name, "..") == 0)
2648                         continue;
2649
2650                 /* if it cannot be a slot, skip the directory */
2651                 if (!ReplicationSlotValidateName(logical_de->d_name, DEBUG2))
2652                         continue;
2653
2654                 /*
2655                  * ok, has to be a surviving logical slot, iterate and delete
2656                  * everything starting with xid-*
2657                  */
2658                 ReorderBufferCleanupSerializedTXNs(logical_de->d_name);
2659         }
2660         FreeDir(logical_dir);
2661 }
2662
2663 /* ---------------------------------------
2664  * toast reassembly support
2665  * ---------------------------------------
2666  */
2667
2668 /*
2669  * Initialize per tuple toast reconstruction support.
2670  */
2671 static void
2672 ReorderBufferToastInitHash(ReorderBuffer *rb, ReorderBufferTXN *txn)
2673 {
2674         HASHCTL         hash_ctl;
2675
2676         Assert(txn->toast_hash == NULL);
2677
2678         memset(&hash_ctl, 0, sizeof(hash_ctl));
2679         hash_ctl.keysize = sizeof(Oid);
2680         hash_ctl.entrysize = sizeof(ReorderBufferToastEnt);
2681         hash_ctl.hcxt = rb->context;
2682         txn->toast_hash = hash_create("ReorderBufferToastHash", 5, &hash_ctl,
2683                                                                   HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
2684 }
2685
2686 /*
2687  * Per toast-chunk handling for toast reconstruction
2688  *
2689  * Appends a toast chunk so we can reconstruct it when the tuple "owning" the
2690  * toasted Datum comes along.
2691  */
2692 static void
2693 ReorderBufferToastAppendChunk(ReorderBuffer *rb, ReorderBufferTXN *txn,
2694                                                           Relation relation, ReorderBufferChange *change)
2695 {
2696         ReorderBufferToastEnt *ent;
2697         ReorderBufferTupleBuf *newtup;
2698         bool            found;
2699         int32           chunksize;
2700         bool            isnull;
2701         Pointer         chunk;
2702         TupleDesc       desc = RelationGetDescr(relation);
2703         Oid                     chunk_id;
2704         int32           chunk_seq;
2705
2706         if (txn->toast_hash == NULL)
2707                 ReorderBufferToastInitHash(rb, txn);
2708
2709         Assert(IsToastRelation(relation));
2710
2711         newtup = change->data.tp.newtuple;
2712         chunk_id = DatumGetObjectId(fastgetattr(&newtup->tuple, 1, desc, &isnull));
2713         Assert(!isnull);
2714         chunk_seq = DatumGetInt32(fastgetattr(&newtup->tuple, 2, desc, &isnull));
2715         Assert(!isnull);
2716
2717         ent = (ReorderBufferToastEnt *)
2718                 hash_search(txn->toast_hash,
2719                                         (void *) &chunk_id,
2720                                         HASH_ENTER,
2721                                         &found);
2722
2723         if (!found)
2724         {
2725                 Assert(ent->chunk_id == chunk_id);
2726                 ent->num_chunks = 0;
2727                 ent->last_chunk_seq = 0;
2728                 ent->size = 0;
2729                 ent->reconstructed = NULL;
2730                 dlist_init(&ent->chunks);
2731
2732                 if (chunk_seq != 0)
2733                         elog(ERROR, "got sequence entry %d for toast chunk %u instead of seq 0",
2734                                  chunk_seq, chunk_id);
2735         }
2736         else if (found && chunk_seq != ent->last_chunk_seq + 1)
2737                 elog(ERROR, "got sequence entry %d for toast chunk %u instead of seq %d",
2738                          chunk_seq, chunk_id, ent->last_chunk_seq + 1);
2739
2740         chunk = DatumGetPointer(fastgetattr(&newtup->tuple, 3, desc, &isnull));
2741         Assert(!isnull);
2742
2743         /* calculate size so we can allocate the right size at once later */
2744         if (!VARATT_IS_EXTENDED(chunk))
2745                 chunksize = VARSIZE(chunk) - VARHDRSZ;
2746         else if (VARATT_IS_SHORT(chunk))
2747                 /* could happen due to heap_form_tuple doing its thing */
2748                 chunksize = VARSIZE_SHORT(chunk) - VARHDRSZ_SHORT;
2749         else
2750                 elog(ERROR, "unexpected type of toast chunk");
2751
2752         ent->size += chunksize;
2753         ent->last_chunk_seq = chunk_seq;
2754         ent->num_chunks++;
2755         dlist_push_tail(&ent->chunks, &change->node);
2756 }
2757
2758 /*
2759  * Rejigger change->newtuple to point to in-memory toast tuples instead to
2760  * on-disk toast tuples that may not longer exist (think DROP TABLE or VACUUM).
2761  *
2762  * We cannot replace unchanged toast tuples though, so those will still point
2763  * to on-disk toast data.
2764  */
2765 static void
2766 ReorderBufferToastReplace(ReorderBuffer *rb, ReorderBufferTXN *txn,
2767                                                   Relation relation, ReorderBufferChange *change)
2768 {
2769         TupleDesc       desc;
2770         int                     natt;
2771         Datum      *attrs;
2772         bool       *isnull;
2773         bool       *free;
2774         HeapTuple       tmphtup;
2775         Relation        toast_rel;
2776         TupleDesc       toast_desc;
2777         MemoryContext oldcontext;
2778         ReorderBufferTupleBuf *newtup;
2779
2780         /* no toast tuples changed */
2781         if (txn->toast_hash == NULL)
2782                 return;
2783
2784         oldcontext = MemoryContextSwitchTo(rb->context);
2785
2786         /* we should only have toast tuples in an INSERT or UPDATE */
2787         Assert(change->data.tp.newtuple);
2788
2789         desc = RelationGetDescr(relation);
2790
2791         toast_rel = RelationIdGetRelation(relation->rd_rel->reltoastrelid);
2792         toast_desc = RelationGetDescr(toast_rel);
2793
2794         /* should we allocate from stack instead? */
2795         attrs = palloc0(sizeof(Datum) * desc->natts);
2796         isnull = palloc0(sizeof(bool) * desc->natts);
2797         free = palloc0(sizeof(bool) * desc->natts);
2798
2799         newtup = change->data.tp.newtuple;
2800
2801         heap_deform_tuple(&newtup->tuple, desc, attrs, isnull);
2802
2803         for (natt = 0; natt < desc->natts; natt++)
2804         {
2805                 Form_pg_attribute attr = TupleDescAttr(desc, natt);
2806                 ReorderBufferToastEnt *ent;
2807                 struct varlena *varlena;
2808
2809                 /* va_rawsize is the size of the original datum -- including header */
2810                 struct varatt_external toast_pointer;
2811                 struct varatt_indirect redirect_pointer;
2812                 struct varlena *new_datum = NULL;
2813                 struct varlena *reconstructed;
2814                 dlist_iter      it;
2815                 Size            data_done = 0;
2816
2817                 /* system columns aren't toasted */
2818                 if (attr->attnum < 0)
2819                         continue;
2820
2821                 if (attr->attisdropped)
2822                         continue;
2823
2824                 /* not a varlena datatype */
2825                 if (attr->attlen != -1)
2826                         continue;
2827
2828                 /* no data */
2829                 if (isnull[natt])
2830                         continue;
2831
2832                 /* ok, we know we have a toast datum */
2833                 varlena = (struct varlena *) DatumGetPointer(attrs[natt]);
2834
2835                 /* no need to do anything if the tuple isn't external */
2836                 if (!VARATT_IS_EXTERNAL(varlena))
2837                         continue;
2838
2839                 VARATT_EXTERNAL_GET_POINTER(toast_pointer, varlena);
2840
2841                 /*
2842                  * Check whether the toast tuple changed, replace if so.
2843                  */
2844                 ent = (ReorderBufferToastEnt *)
2845                         hash_search(txn->toast_hash,
2846                                                 (void *) &toast_pointer.va_valueid,
2847                                                 HASH_FIND,
2848                                                 NULL);
2849                 if (ent == NULL)
2850                         continue;
2851
2852                 new_datum =
2853                         (struct varlena *) palloc0(INDIRECT_POINTER_SIZE);
2854
2855                 free[natt] = true;
2856
2857                 reconstructed = palloc0(toast_pointer.va_rawsize);
2858
2859                 ent->reconstructed = reconstructed;
2860
2861                 /* stitch toast tuple back together from its parts */
2862                 dlist_foreach(it, &ent->chunks)
2863                 {
2864                         bool            isnull;
2865                         ReorderBufferChange *cchange;
2866                         ReorderBufferTupleBuf *ctup;
2867                         Pointer         chunk;
2868
2869                         cchange = dlist_container(ReorderBufferChange, node, it.cur);
2870                         ctup = cchange->data.tp.newtuple;
2871                         chunk = DatumGetPointer(
2872                                                                         fastgetattr(&ctup->tuple, 3, toast_desc, &isnull));
2873
2874                         Assert(!isnull);
2875                         Assert(!VARATT_IS_EXTERNAL(chunk));
2876                         Assert(!VARATT_IS_SHORT(chunk));
2877
2878                         memcpy(VARDATA(reconstructed) + data_done,
2879                                    VARDATA(chunk),
2880                                    VARSIZE(chunk) - VARHDRSZ);
2881                         data_done += VARSIZE(chunk) - VARHDRSZ;
2882                 }
2883                 Assert(data_done == toast_pointer.va_extsize);
2884
2885                 /* make sure its marked as compressed or not */
2886                 if (VARATT_EXTERNAL_IS_COMPRESSED(toast_pointer))
2887                         SET_VARSIZE_COMPRESSED(reconstructed, data_done + VARHDRSZ);
2888                 else
2889                         SET_VARSIZE(reconstructed, data_done + VARHDRSZ);
2890
2891                 memset(&redirect_pointer, 0, sizeof(redirect_pointer));
2892                 redirect_pointer.pointer = reconstructed;
2893
2894                 SET_VARTAG_EXTERNAL(new_datum, VARTAG_INDIRECT);
2895                 memcpy(VARDATA_EXTERNAL(new_datum), &redirect_pointer,
2896                            sizeof(redirect_pointer));
2897
2898                 attrs[natt] = PointerGetDatum(new_datum);
2899         }
2900
2901         /*
2902          * Build tuple in separate memory & copy tuple back into the tuplebuf
2903          * passed to the output plugin. We can't directly heap_fill_tuple() into
2904          * the tuplebuf because attrs[] will point back into the current content.
2905          */
2906         tmphtup = heap_form_tuple(desc, attrs, isnull);
2907         Assert(newtup->tuple.t_len <= MaxHeapTupleSize);
2908         Assert(ReorderBufferTupleBufData(newtup) == newtup->tuple.t_data);
2909
2910         memcpy(newtup->tuple.t_data, tmphtup->t_data, tmphtup->t_len);
2911         newtup->tuple.t_len = tmphtup->t_len;
2912
2913         /*
2914          * free resources we won't further need, more persistent stuff will be
2915          * free'd in ReorderBufferToastReset().
2916          */
2917         RelationClose(toast_rel);
2918         pfree(tmphtup);
2919         for (natt = 0; natt < desc->natts; natt++)
2920         {
2921                 if (free[natt])
2922                         pfree(DatumGetPointer(attrs[natt]));
2923         }
2924         pfree(attrs);
2925         pfree(free);
2926         pfree(isnull);
2927
2928         MemoryContextSwitchTo(oldcontext);
2929 }
2930
2931 /*
2932  * Free all resources allocated for toast reconstruction.
2933  */
2934 static void
2935 ReorderBufferToastReset(ReorderBuffer *rb, ReorderBufferTXN *txn)
2936 {
2937         HASH_SEQ_STATUS hstat;
2938         ReorderBufferToastEnt *ent;
2939
2940         if (txn->toast_hash == NULL)
2941                 return;
2942
2943         /* sequentially walk over the hash and free everything */
2944         hash_seq_init(&hstat, txn->toast_hash);
2945         while ((ent = (ReorderBufferToastEnt *) hash_seq_search(&hstat)) != NULL)
2946         {
2947                 dlist_mutable_iter it;
2948
2949                 if (ent->reconstructed != NULL)
2950                         pfree(ent->reconstructed);
2951
2952                 dlist_foreach_modify(it, &ent->chunks)
2953                 {
2954                         ReorderBufferChange *change =
2955                         dlist_container(ReorderBufferChange, node, it.cur);
2956
2957                         dlist_delete(&change->node);
2958                         ReorderBufferReturnChange(rb, change);
2959                 }
2960         }
2961
2962         hash_destroy(txn->toast_hash);
2963         txn->toast_hash = NULL;
2964 }
2965
2966
2967 /* ---------------------------------------
2968  * Visibility support for logical decoding
2969  *
2970  *
2971  * Lookup actual cmin/cmax values when using decoding snapshot. We can't
2972  * always rely on stored cmin/cmax values because of two scenarios:
2973  *
2974  * * A tuple got changed multiple times during a single transaction and thus
2975  *       has got a combocid. Combocid's are only valid for the duration of a
2976  *       single transaction.
2977  * * A tuple with a cmin but no cmax (and thus no combocid) got
2978  *       deleted/updated in another transaction than the one which created it
2979  *       which we are looking at right now. As only one of cmin, cmax or combocid
2980  *       is actually stored in the heap we don't have access to the value we
2981  *       need anymore.
2982  *
2983  * To resolve those problems we have a per-transaction hash of (cmin,
2984  * cmax) tuples keyed by (relfilenode, ctid) which contains the actual
2985  * (cmin, cmax) values. That also takes care of combocids by simply
2986  * not caring about them at all. As we have the real cmin/cmax values
2987  * combocids aren't interesting.
2988  *
2989  * As we only care about catalog tuples here the overhead of this
2990  * hashtable should be acceptable.
2991  *
2992  * Heap rewrites complicate this a bit, check rewriteheap.c for
2993  * details.
2994  * -------------------------------------------------------------------------
2995  */
2996
2997 /* struct for qsort()ing mapping files by lsn somewhat efficiently */
2998 typedef struct RewriteMappingFile
2999 {
3000         XLogRecPtr      lsn;
3001         char            fname[MAXPGPATH];
3002 } RewriteMappingFile;
3003
3004 #if NOT_USED
3005 static void
3006 DisplayMapping(HTAB *tuplecid_data)
3007 {
3008         HASH_SEQ_STATUS hstat;
3009         ReorderBufferTupleCidEnt *ent;
3010
3011         hash_seq_init(&hstat, tuplecid_data);
3012         while ((ent = (ReorderBufferTupleCidEnt *) hash_seq_search(&hstat)) != NULL)
3013         {
3014                 elog(DEBUG3, "mapping: node: %u/%u/%u tid: %u/%u cmin: %u, cmax: %u",
3015                          ent->key.relnode.dbNode,
3016                          ent->key.relnode.spcNode,
3017                          ent->key.relnode.relNode,
3018                          ItemPointerGetBlockNumber(&ent->key.tid),
3019                          ItemPointerGetOffsetNumber(&ent->key.tid),
3020                          ent->cmin,
3021                          ent->cmax
3022                         );
3023         }
3024 }
3025 #endif
3026
3027 /*
3028  * Apply a single mapping file to tuplecid_data.
3029  *
3030  * The mapping file has to have been verified to be a) committed b) for our
3031  * transaction c) applied in LSN order.
3032  */
3033 static void
3034 ApplyLogicalMappingFile(HTAB *tuplecid_data, Oid relid, const char *fname)
3035 {
3036         char            path[MAXPGPATH];
3037         int                     fd;
3038         int                     readBytes;
3039         LogicalRewriteMappingData map;
3040
3041         sprintf(path, "pg_logical/mappings/%s", fname);
3042         fd = OpenTransientFile(path, O_RDONLY | PG_BINARY);
3043         if (fd < 0)
3044                 ereport(ERROR,
3045                                 (errcode_for_file_access(),
3046                                  errmsg("could not open file \"%s\": %m", path)));
3047
3048         while (true)
3049         {
3050                 ReorderBufferTupleCidKey key;
3051                 ReorderBufferTupleCidEnt *ent;
3052                 ReorderBufferTupleCidEnt *new_ent;
3053                 bool            found;
3054
3055                 /* be careful about padding */
3056                 memset(&key, 0, sizeof(ReorderBufferTupleCidKey));
3057
3058                 /* read all mappings till the end of the file */
3059                 pgstat_report_wait_start(WAIT_EVENT_REORDER_LOGICAL_MAPPING_READ);
3060                 readBytes = read(fd, &map, sizeof(LogicalRewriteMappingData));
3061                 pgstat_report_wait_end();
3062
3063                 if (readBytes < 0)
3064                         ereport(ERROR,
3065                                         (errcode_for_file_access(),
3066                                          errmsg("could not read file \"%s\": %m",
3067                                                         path)));
3068                 else if (readBytes == 0)        /* EOF */
3069                         break;
3070                 else if (readBytes != sizeof(LogicalRewriteMappingData))
3071                         ereport(ERROR,
3072                                         (errcode_for_file_access(),
3073                                          errmsg("could not read from file \"%s\": read %d instead of %d bytes",
3074                                                         path, readBytes,
3075                                                         (int32) sizeof(LogicalRewriteMappingData))));
3076
3077                 key.relnode = map.old_node;
3078                 ItemPointerCopy(&map.old_tid,
3079                                                 &key.tid);
3080
3081
3082                 ent = (ReorderBufferTupleCidEnt *)
3083                         hash_search(tuplecid_data,
3084                                                 (void *) &key,
3085                                                 HASH_FIND,
3086                                                 NULL);
3087
3088                 /* no existing mapping, no need to update */
3089                 if (!ent)
3090                         continue;
3091
3092                 key.relnode = map.new_node;
3093                 ItemPointerCopy(&map.new_tid,
3094                                                 &key.tid);
3095
3096                 new_ent = (ReorderBufferTupleCidEnt *)
3097                         hash_search(tuplecid_data,
3098                                                 (void *) &key,
3099                                                 HASH_ENTER,
3100                                                 &found);
3101
3102                 if (found)
3103                 {
3104                         /*
3105                          * Make sure the existing mapping makes sense. We sometime update
3106                          * old records that did not yet have a cmax (e.g. pg_class' own
3107                          * entry while rewriting it) during rewrites, so allow that.
3108                          */
3109                         Assert(ent->cmin == InvalidCommandId || ent->cmin == new_ent->cmin);
3110                         Assert(ent->cmax == InvalidCommandId || ent->cmax == new_ent->cmax);
3111                 }
3112                 else
3113                 {
3114                         /* update mapping */
3115                         new_ent->cmin = ent->cmin;
3116                         new_ent->cmax = ent->cmax;
3117                         new_ent->combocid = ent->combocid;
3118                 }
3119         }
3120 }
3121
3122
3123 /*
3124  * Check whether the TransactionOId 'xid' is in the pre-sorted array 'xip'.
3125  */
3126 static bool
3127 TransactionIdInArray(TransactionId xid, TransactionId *xip, Size num)
3128 {
3129         return bsearch(&xid, xip, num,
3130                                    sizeof(TransactionId), xidComparator) != NULL;
3131 }
3132
3133 /*
3134  * qsort() comparator for sorting RewriteMappingFiles in LSN order.
3135  */
3136 static int
3137 file_sort_by_lsn(const void *a_p, const void *b_p)
3138 {
3139         RewriteMappingFile *a = *(RewriteMappingFile **) a_p;
3140         RewriteMappingFile *b = *(RewriteMappingFile **) b_p;
3141
3142         if (a->lsn < b->lsn)
3143                 return -1;
3144         else if (a->lsn > b->lsn)
3145                 return 1;
3146         return 0;
3147 }
3148
3149 /*
3150  * Apply any existing logical remapping files if there are any targeted at our
3151  * transaction for relid.
3152  */
3153 static void
3154 UpdateLogicalMappings(HTAB *tuplecid_data, Oid relid, Snapshot snapshot)
3155 {
3156         DIR                *mapping_dir;
3157         struct dirent *mapping_de;
3158         List       *files = NIL;
3159         ListCell   *file;
3160         RewriteMappingFile **files_a;
3161         size_t          off;
3162         Oid                     dboid = IsSharedRelation(relid) ? InvalidOid : MyDatabaseId;
3163
3164         mapping_dir = AllocateDir("pg_logical/mappings");
3165         while ((mapping_de = ReadDir(mapping_dir, "pg_logical/mappings")) != NULL)
3166         {
3167                 Oid                     f_dboid;
3168                 Oid                     f_relid;
3169                 TransactionId f_mapped_xid;
3170                 TransactionId f_create_xid;
3171                 XLogRecPtr      f_lsn;
3172                 uint32          f_hi,
3173                                         f_lo;
3174                 RewriteMappingFile *f;
3175
3176                 if (strcmp(mapping_de->d_name, ".") == 0 ||
3177                         strcmp(mapping_de->d_name, "..") == 0)
3178                         continue;
3179
3180                 /* Ignore files that aren't ours */
3181                 if (strncmp(mapping_de->d_name, "map-", 4) != 0)
3182                         continue;
3183
3184                 if (sscanf(mapping_de->d_name, LOGICAL_REWRITE_FORMAT,
3185                                    &f_dboid, &f_relid, &f_hi, &f_lo,
3186                                    &f_mapped_xid, &f_create_xid) != 6)
3187                         elog(ERROR, "could not parse filename \"%s\"", mapping_de->d_name);
3188
3189                 f_lsn = ((uint64) f_hi) << 32 | f_lo;
3190
3191                 /* mapping for another database */
3192                 if (f_dboid != dboid)
3193                         continue;
3194
3195                 /* mapping for another relation */
3196                 if (f_relid != relid)
3197                         continue;
3198
3199                 /* did the creating transaction abort? */
3200                 if (!TransactionIdDidCommit(f_create_xid))
3201                         continue;
3202
3203                 /* not for our transaction */
3204                 if (!TransactionIdInArray(f_mapped_xid, snapshot->subxip, snapshot->subxcnt))
3205                         continue;
3206
3207                 /* ok, relevant, queue for apply */
3208                 f = palloc(sizeof(RewriteMappingFile));
3209                 f->lsn = f_lsn;
3210                 strcpy(f->fname, mapping_de->d_name);
3211                 files = lappend(files, f);
3212         }
3213         FreeDir(mapping_dir);
3214
3215         /* build array we can easily sort */
3216         files_a = palloc(list_length(files) * sizeof(RewriteMappingFile *));
3217         off = 0;
3218         foreach(file, files)
3219         {
3220                 files_a[off++] = lfirst(file);
3221         }
3222
3223         /* sort files so we apply them in LSN order */
3224         qsort(files_a, list_length(files), sizeof(RewriteMappingFile *),
3225                   file_sort_by_lsn);
3226
3227         for (off = 0; off < list_length(files); off++)
3228         {
3229                 RewriteMappingFile *f = files_a[off];
3230
3231                 elog(DEBUG1, "applying mapping: \"%s\" in %u", f->fname,
3232                          snapshot->subxip[0]);
3233                 ApplyLogicalMappingFile(tuplecid_data, relid, f->fname);
3234                 pfree(f);
3235         }
3236 }
3237
3238 /*
3239  * Lookup cmin/cmax of a tuple, during logical decoding where we can't rely on
3240  * combocids.
3241  */
3242 bool
3243 ResolveCminCmaxDuringDecoding(HTAB *tuplecid_data,
3244                                                           Snapshot snapshot,
3245                                                           HeapTuple htup, Buffer buffer,
3246                                                           CommandId *cmin, CommandId *cmax)
3247 {
3248         ReorderBufferTupleCidKey key;
3249         ReorderBufferTupleCidEnt *ent;
3250         ForkNumber      forkno;
3251         BlockNumber blockno;
3252         bool            updated_mapping = false;
3253
3254         /* be careful about padding */
3255         memset(&key, 0, sizeof(key));
3256
3257         Assert(!BufferIsLocal(buffer));
3258
3259         /*
3260          * get relfilenode from the buffer, no convenient way to access it other
3261          * than that.
3262          */
3263         BufferGetTag(buffer, &key.relnode, &forkno, &blockno);
3264
3265         /* tuples can only be in the main fork */
3266         Assert(forkno == MAIN_FORKNUM);
3267         Assert(blockno == ItemPointerGetBlockNumber(&htup->t_self));
3268
3269         ItemPointerCopy(&htup->t_self,
3270                                         &key.tid);
3271
3272 restart:
3273         ent = (ReorderBufferTupleCidEnt *)
3274                 hash_search(tuplecid_data,
3275                                         (void *) &key,
3276                                         HASH_FIND,
3277                                         NULL);
3278
3279         /*
3280          * failed to find a mapping, check whether the table was rewritten and
3281          * apply mapping if so, but only do that once - there can be no new
3282          * mappings while we are in here since we have to hold a lock on the
3283          * relation.
3284          */
3285         if (ent == NULL && !updated_mapping)
3286         {
3287                 UpdateLogicalMappings(tuplecid_data, htup->t_tableOid, snapshot);
3288                 /* now check but don't update for a mapping again */
3289                 updated_mapping = true;
3290                 goto restart;
3291         }
3292         else if (ent == NULL)
3293                 return false;
3294
3295         if (cmin)
3296                 *cmin = ent->cmin;
3297         if (cmax)
3298                 *cmax = ent->cmax;
3299         return true;
3300 }