*/
if (!skip)
{
+ /* setup the redirected t_self for the benefit of logical decoding */
+ ItemPointerSet(&(heapTuple->t_self), BufferGetBlockNumber(buffer), offnum);
+
/* If it's visible per the snapshot, we must return it */
valid = HeapTupleSatisfiesVisibility(heapTuple, snapshot, buffer);
CheckForSerializableConflictOut(valid, relation, heapTuple,
buffer, snapshot);
+ /* reset to original, non-redirected, tid */
+ heapTuple->t_self = *tid;
+
if (valid)
{
ItemPointerSetOffsetNumber(tid, offnum);
* decoding.
*/
break;
+ case XLOG_HEAP2_REWRITE:
+ heap_xlog_logical_rewrite(lsn, record);
+ break;
default:
elog(PANIC, "heap2_redo: unknown op code %u", info);
}
*/
#include "postgres.h"
+#include <sys/stat.h>
+#include <unistd.h>
+
+#include "miscadmin.h"
+
#include "access/heapam.h"
#include "access/heapam_xlog.h"
#include "access/rewriteheap.h"
#include "access/transam.h"
#include "access/tuptoaster.h"
+#include "access/xact.h"
+
+#include "catalog/catalog.h"
+
+#include "replication/logical.h"
+
#include "storage/bufmgr.h"
+#include "storage/fd.h"
#include "storage/smgr.h"
#include "utils/memutils.h"
#include "utils/rel.h"
#include "utils/tqual.h"
+#include "lib/ilist.h"
+
+#include "storage/procarray.h"
/*
* State associated with a rewrite operation. This is opaque to the user
*/
typedef struct RewriteStateData
{
+ Relation rs_old_rel; /* source heap */
Relation rs_new_rel; /* destination heap */
Page rs_buffer; /* page currently being built */
BlockNumber rs_blockno; /* block where page will go */
bool rs_buffer_valid; /* T if any tuples in buffer */
bool rs_use_wal; /* must we WAL-log inserts? */
+ bool rs_logical_rewrite; /* do we need to do logical rewriting */
TransactionId rs_oldest_xmin; /* oldest xmin used by caller to
* determine tuple visibility */
TransactionId rs_freeze_xid;/* Xid that will be used as freeze cutoff
* point */
+ TransactionId rs_logical_xmin; /* Xid that will be used as cutoff
+ * point for logical rewrites */
MultiXactId rs_cutoff_multi;/* MultiXactId that will be used as cutoff
* point for multixacts */
MemoryContext rs_cxt; /* for hash tables and entries and tuples in
* them */
+ XLogRecPtr rs_begin_lsn;
HTAB *rs_unresolved_tups; /* unmatched A tuples */
HTAB *rs_old_new_tid_map; /* unmatched B tuples */
+ HTAB *rs_logical_mappings; /* logical remapping files */
+ uint32 rs_num_rewrite_mappings; /* # in memory mappings */
} RewriteStateData;
/*
typedef OldToNewMappingData *OldToNewMapping;
+/*
+ * In-Memory data for a xid that might need logical remapping entries
+ * to be logged.
+ */
+typedef struct RewriteMappingFile
+{
+ TransactionId xid; /* xid that might need to see the row */
+ int vfd; /* fd of mappings file */
+ off_t off; /* how far have we written yet */
+ uint32 num_mappings; /* number of in-memory mappings */
+ dlist_head mappings; /* list of in-memory mappings */
+} RewriteMappingFile;
+
+/*
+ * In-Memeory logical rewrite mapping, linked of
+ * RewriteMappingFile->mappings.
+ */
+typedef struct RewriteMappingDataEntry
+{
+ LogicalRewriteMappingData map;
+ dlist_node node;
+} RewriteMappingDataEntry;
+
/* prototypes for internal functions */
static void raw_heap_insert(RewriteState state, HeapTuple tup);
+/* internal logical remapping prototypes */
+static void logical_begin_heap_rewrite(RewriteState state);
+static void logical_rewrite_heap_tuple(RewriteState state, ItemPointerData old_tid, HeapTuple new_tuple);
+static void logical_end_heap_rewrite(RewriteState state);
+
/*
* Begin a rewrite of a table
*
+ * old_heap old, locked heap relation tuples will be read from
* new_heap new, locked heap relation to insert tuples to
* oldest_xmin xid used by the caller to determine which tuples are dead
* freeze_xid xid before which tuples will be frozen
* to be used in subsequent calls to the other functions.
*/
RewriteState
-begin_heap_rewrite(Relation new_heap, TransactionId oldest_xmin,
+begin_heap_rewrite(Relation old_heap, Relation new_heap, TransactionId oldest_xmin,
TransactionId freeze_xid, MultiXactId cutoff_multi,
bool use_wal)
{
/* Create and fill in the state struct */
state = palloc0(sizeof(RewriteStateData));
+ state->rs_old_rel = old_heap;
state->rs_new_rel = new_heap;
state->rs_buffer = (Page) palloc(BLCKSZ);
/* new_heap needn't be empty, just locked */
MemoryContextSwitchTo(old_cxt);
+ logical_begin_heap_rewrite(state);
+
return state;
}
if (RelationNeedsWAL(state->rs_new_rel))
heap_sync(state->rs_new_rel);
+ logical_end_heap_rewrite(state);
+
/* Deleting the context frees everything */
MemoryContextDelete(state->rs_cxt);
}
raw_heap_insert(state, new_tuple);
new_tid = new_tuple->t_self;
+ logical_rewrite_heap_tuple(state, old_tid, new_tuple);
+
/*
* If the tuple is the updated version of a row, and the prior version
* wouldn't be DEAD yet, then we need to either resolve the prior
if (heaptup != tup)
heap_freetuple(heaptup);
}
+
+/* ------------------------------------------------------------------------
+ * Logical rewrite support
+ *
+ * When doing logical decoding - which relies on using cmin/cmax of catalog
+ * tuples, via xl_heap_new_cid records - heap rewrites have to log enough
+ * information to allow the decoding backend to updates its internal mapping
+ * of (relfilenode,ctid) => (cmin, cmax) to be correct for the rewritten heap.
+ *
+ * For that, everytime we find a tuple that's been modified in a catalog
+ * relation within the xmin horizon of any decoding slot, we log a mapping
+ * from the old to the new location.
+ *
+ * To deal with rewrites that abort the filename of a mapping file contains
+ * the xid of the transaction performing the rewrite which then can be checked
+ * when read in.
+ *
+ * For efficiency we don't immediately spill every single map mapping for a
+ * rrow to disk but only do so in batches when we've collected several of them
+ * in memory or when end_heap_rewrite() has been called.
+ *
+ * Crash-Safety: This module diverts from the usual patterns of doing WAL
+ * since it cannot rely on checkpoint flushing out all buffers and thus
+ * waiting for exlusive locks on buffers. Usually the XLogInsert() covering
+ * buffer modifications is performed while the buffer(s) that are being
+ * modified are exlusively locked guaranteeing that both the WAL record and
+ * the modified heap are on either side of the checkpoint. But since the
+ * mapping files we log aren't in shared_buffers that interlock doesn't work.
+ *
+ * Instead we simply write the mapping files out to disk, *before* the
+ * XLogInsert() is performed. That guarantees that either the XLogInsert() is
+ * inserted after the checkpoint's redo pointer or that the checkpoint (via
+ * LogicalRewriteHeapCheckpoint()) has fsynced the (partial) mapping
+ * file. That leaves the tail end that has not yet been fsync()ed to disk open
+ * to corruption, which is solved by including the current offset in the
+ * xl_heap_rewrite_mapping records and truncating the mapping file to that
+ * position before those records back. Independently from WAL logging,
+ * everytime a rewrite is finished all generated mapping files are directly
+ * fsynced.
+ *
+ * Note that if we were only concerned about crash safety we wouldn't have to
+ * deal with WAL logging at all - the fsyncing at the end of a rewrite would
+ * be sufficient for crash safety. Any mapping that hasn't been safely synced
+ * to disk has to be by an aborted (explicitly or via a crash) transaction and
+ * is ignored by virtue of the xid in it's name being subject to a
+ * TransactionDidCommit() check. But we want to support physical replication
+ * for availability and to support logical decoding on the standbys.
+ * ------------------------------------------------------------------------
+ */
+
+/*
+ * Do preparations for logging logical mappings during a rewrite if
+ * necessary. If we detect that we don't need to log anything we'll prevent
+ * any further action by logical_*rewrite*
+ */
+static void
+logical_begin_heap_rewrite(RewriteState state)
+{
+ HASHCTL hash_ctl;
+ TransactionId logical_xmin;
+
+ state->rs_logical_rewrite =
+ RelationIsAccessibleInLogicalDecoding(state->rs_old_rel);
+
+ if (!state->rs_logical_rewrite)
+ return;
+
+ Assert(LogicalDecodingCtl != NULL);
+ /* we assume reading this is atomic */
+ logical_xmin =
+ ((volatile LogicalDecodingCtlData*) LogicalDecodingCtl)->xmin;
+
+ /*
+ * If there are no logical slots in progress we don't need to do anything,
+ * there cannot be any remappings for relevant rows yet. The relation's
+ * lock protects us against races.
+ */
+ if (logical_xmin == InvalidTransactionId)
+ {
+ state->rs_logical_rewrite = false;
+ return;
+ }
+
+ state->rs_logical_xmin = logical_xmin;
+ state->rs_begin_lsn = GetXLogInsertRecPtr();
+ state->rs_num_rewrite_mappings = 0;
+
+ memset(&hash_ctl, 0, sizeof(hash_ctl));
+ hash_ctl.keysize = sizeof(TransactionId);
+ hash_ctl.entrysize = sizeof(RewriteMappingFile);
+ hash_ctl.hcxt = state->rs_cxt;
+ hash_ctl.hash = tag_hash;
+
+ state->rs_logical_mappings =
+ hash_create("Logical rewrite mapping",
+ 128, /* arbitrary initial size */
+ &hash_ctl,
+ HASH_ELEM | HASH_FUNCTION | HASH_CONTEXT);
+}
+
+/*
+ * Flush all logical in-memory mappings to disk, but don't fsync them yet.
+ */
+static void
+logical_heap_rewrite_flush_mappings(RewriteState state)
+{
+ HASH_SEQ_STATUS seq_status;
+ RewriteMappingFile *src;
+ dlist_mutable_iter iter;
+
+ Assert(state->rs_logical_rewrite);
+
+ /* nothing to do, no need to iterate over mappings */
+ if (state->rs_num_rewrite_mappings == 0)
+ return;
+ elog(DEBUG1, "flushing %u logical rewrite mapping entries",
+ state->rs_num_rewrite_mappings);
+
+ hash_seq_init(&seq_status, state->rs_logical_mappings);
+ while ((src = (RewriteMappingFile *) hash_seq_search(&seq_status)) != NULL)
+ {
+ XLogRecData rdata[2];
+ char *waldata;
+ char *waldata_start;
+ xl_heap_rewrite_mapping xlrec;
+ Oid dboid;
+ uint32 len;
+ int written;
+
+ /* this file hasn't got any new mappings */
+ if (src->num_mappings == 0)
+ continue;
+
+ if (state->rs_old_rel->rd_rel->relisshared)
+ dboid = InvalidOid;
+ else
+ dboid = MyDatabaseId;
+
+ xlrec.num_mappings = src->num_mappings;
+ xlrec.mapped_rel = RelationGetRelid(state->rs_old_rel);
+ xlrec.mapped_xid = src->xid;
+ xlrec.mapped_db = dboid;
+ xlrec.offset = src->off;
+ xlrec.start_lsn = state->rs_begin_lsn;
+
+ rdata[0].data = (char *) (&xlrec);
+ rdata[0].len = sizeof(xlrec);
+ rdata[0].buffer = InvalidBuffer;
+ rdata[0].next = &(rdata[1]);
+
+ /* write all mappings consecutively */
+ len = src->num_mappings * sizeof(LogicalRewriteMappingData);
+ waldata = palloc(len);
+ waldata_start = waldata;
+
+ /*
+ * collect data we need to write out, but don't modify ondisk data yet
+ */
+ dlist_foreach_modify(iter, &src->mappings)
+ {
+ RewriteMappingDataEntry *pmap;
+
+ pmap = dlist_container(RewriteMappingDataEntry, node, iter.cur);
+
+ memcpy(waldata, &pmap->map, sizeof(pmap->map));
+ waldata += sizeof(pmap->map);
+
+ /* remove from the list and free */
+ dlist_delete(&pmap->node);
+ pfree(pmap);
+
+ /* update bookkeeping */
+ state->rs_num_rewrite_mappings--;
+ src->num_mappings--;
+ }
+
+ /*
+ * We intentionally violate the usual WAL coding practices here and
+ * write to the file *first*. This way an eventual checkpoint will
+ * sync any part of the file that's not guaranteed to be recovered by
+ * the XLogInsert(). We deal with the potential corruption in the tail
+ * of the file by truncating it to the last safe point during WAL
+ * replay and by checking whether the xid performing the mapping has
+ * committed.
+ */
+ written = FileWrite(src->vfd, waldata_start, len);
+ if (written < 0)
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("failed to write to logical remapping file: %m")));
+
+ if (written != len)
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("incomplete write to logical remapping file, wrote %d of %u",
+ written, len)));
+
+ src->off += len;
+
+ Assert(src->num_mappings == 0);
+
+ rdata[1].data = waldata_start;
+ rdata[1].len = len;
+ rdata[1].buffer = InvalidBuffer;
+ rdata[1].next = NULL;
+
+ /* write xlog record */
+ XLogInsert(RM_HEAP2_ID, XLOG_HEAP2_REWRITE, rdata);
+
+ }
+ Assert(state->rs_num_rewrite_mappings == 0);
+}
+
+/*
+ * Logical part end_heap_rewrite().
+ */
+static void
+logical_end_heap_rewrite(RewriteState state)
+{
+ HASH_SEQ_STATUS seq_status;
+ RewriteMappingFile *src;
+
+ /* done, no logical rewrite in progress */
+ if (!state->rs_logical_rewrite)
+ return;
+
+ /* writeout remaining in-memory entries */
+ if (state->rs_num_rewrite_mappings > 0 )
+ logical_heap_rewrite_flush_mappings(state);
+
+ /* Iterate over all mappings we have written and fsync the files. */
+ hash_seq_init(&seq_status, state->rs_logical_mappings);
+ while ((src = (RewriteMappingFile *) hash_seq_search(&seq_status)) != NULL)
+ {
+ if(FileSync(src->vfd) != 0)
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not fsync remapping file: %m")));
+ FileClose(src->vfd);
+ }
+ /* memory context cleanup will deal with the rest */
+}
+
+/*
+ * Log a single (old->new) mapping for 'xid'.
+ */
+static void
+logical_rewrite_log_mapping(RewriteState state, TransactionId xid,
+ LogicalRewriteMappingData *map)
+{
+ RewriteMappingFile *src;
+ RewriteMappingDataEntry *pmap;
+ Oid relid;
+ bool found;
+
+ relid = RelationGetRelid(state->rs_old_rel);
+
+ /* look for existing mappings for this 'mapped' xid */
+ src = hash_search(state->rs_logical_mappings, &xid,
+ HASH_ENTER, &found);
+
+ /*
+ * We haven't yet had the need to map anything for this xid, create
+ * per-xid data structures.
+ */
+ if (!found)
+ {
+ char path[MAXPGPATH];
+ Oid dboid;
+
+ if (state->rs_old_rel->rd_rel->relisshared)
+ dboid = InvalidOid;
+ else
+ dboid = MyDatabaseId;
+
+ snprintf(path, MAXPGPATH,
+ "pg_llog/mappings/" LOGICAL_REWRITE_FORMAT,
+ dboid, relid,
+ state->rs_begin_lsn,
+ xid, GetCurrentTransactionId());
+
+ dlist_init(&src->mappings);
+ src->num_mappings = 0;
+ src->off = 0;
+ src->vfd = PathNameOpenFile(path,
+ O_CREAT | O_EXCL | O_WRONLY | PG_BINARY,
+ S_IRUSR | S_IWUSR);
+ if (src->vfd < 0)
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not create remapping file \"%s\": %m",
+ path)));
+ }
+
+ pmap = MemoryContextAlloc(state->rs_cxt,
+ sizeof(RewriteMappingDataEntry));
+ memcpy(&pmap->map, map, sizeof(LogicalRewriteMappingData));
+ dlist_push_tail(&src->mappings, &pmap->node);
+ src->num_mappings++;
+ state->rs_num_rewrite_mappings++;
+
+ /*
+ * Write out buffer everytime we've too many in-memory entries.
+ */
+ if (state->rs_num_rewrite_mappings >= 1000 /* arbitrary number */)
+ logical_heap_rewrite_flush_mappings(state);
+}
+
+/*
+ * Perform logical remapping for a tuple that's mapped from old_tid to
+ * new_tuple->t_self by rewrite_heap_tuple() iff necessary for the tuple.
+ */
+static void
+logical_rewrite_heap_tuple(RewriteState state, ItemPointerData old_tid,
+ HeapTuple new_tuple)
+{
+ ItemPointerData new_tid = new_tuple->t_self;
+ TransactionId cutoff = state->rs_logical_xmin;
+ TransactionId xmin;
+ TransactionId xmax;
+ bool do_log_xmin = false;
+ bool do_log_xmax = false;
+ LogicalRewriteMappingData map;
+
+ /* nothing to do if we're not working on a catalog table */
+ if (!state->rs_logical_rewrite)
+ return;
+
+ xmin = HeapTupleHeaderGetXmin(new_tuple->t_data);
+ /* use *GetUpdateXid to correctly deal with multixacts */
+ xmax = HeapTupleHeaderGetUpdateXid(new_tuple->t_data);
+
+ /*
+ * Log the mapping iff the tuple has been created recently.
+ */
+ if (TransactionIdIsNormal(xmin) && !TransactionIdPrecedes(xmin, cutoff))
+ do_log_xmin = true;
+
+ if (!TransactionIdIsNormal(xmax))
+ {
+ /*
+ * no xmax is set, can't have any permanent ones, so this check is
+ * sufficient
+ */
+ }
+ else if (HEAP_XMAX_IS_LOCKED_ONLY(new_tuple->t_data->t_infomask))
+ {
+ /* only locked, we don't care */
+ }
+ else if (!TransactionIdPrecedes(xmax, cutoff))
+ {
+ /* tuple has been deleted recently, log */
+ do_log_xmax = true;
+ }
+
+ /* if neither needs to be logged, we're done */
+ if (!do_log_xmin && !do_log_xmax)
+ return;
+
+ /* fill out mapping information */
+ map.old_node = state->rs_old_rel->rd_node;
+ map.old_tid = old_tid;
+ map.new_node = state->rs_new_rel->rd_node;
+ map.new_tid = new_tid;
+
+ /* ---
+ * Now persist the mapping for the individual xids that are affected. We
+ * need to log for both xmin and xmax if they aren't the same transaction
+ * since the mapping files are per "affected" xid.
+ * We don't muster all that much effort detecting whether xmin and xmax
+ * are actually the same transaction, we just check whether the xid is the
+ * same disregarding subtransactions. Logging too much is relatively
+ * harmless and we could never do the check fully since subtransaction
+ * data is thrown away during restarts.
+ * ---
+ */
+ if (do_log_xmin)
+ logical_rewrite_log_mapping(state, xmin, &map);
+ /* separately log mapping for xmax unless it'd be redundant */
+ if (do_log_xmax && !TransactionIdEquals(xmin, xmax))
+ logical_rewrite_log_mapping(state, xmax, &map);
+}
+
+/*
+ * Replay XLOG_HEAP2_REWRITE records
+ */
+void
+heap_xlog_logical_rewrite(XLogRecPtr lsn, XLogRecord *r)
+{
+ char path[MAXPGPATH];
+ int fd;
+ xl_heap_rewrite_mapping *xlrec;
+ uint32 len;
+ char *data;
+
+ xlrec = (xl_heap_rewrite_mapping *) XLogRecGetData(r);
+
+ snprintf(path, MAXPGPATH,
+ "pg_llog/mappings/" LOGICAL_REWRITE_FORMAT,
+ xlrec->mapped_db, xlrec->mapped_rel, xlrec->start_lsn,
+ xlrec->mapped_xid, r->xl_xid);
+
+ fd = OpenTransientFile(path,
+ O_CREAT | O_WRONLY | PG_BINARY,
+ S_IRUSR | S_IWUSR);
+ if (fd < 0)
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("Could not create logical remapping file \"%s\": %m",
+ path)));
+ /*
+ * Truncate all data that's not guaranteed to have been safely
+ * fsynced (by previous record or by the last checkpoint).
+ */
+ if (ftruncate(fd, xlrec->offset) != 0)
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("Could not truncate logical remapping file \"%s\" to %u: %m",
+ path, (uint32) xlrec->offset)));
+
+ /* Now seek to the position we want to write our data to */
+ if (lseek(fd, xlrec->offset, SEEK_SET) != xlrec->offset)
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("Could not seek to the end of logical remapping file \"%s\": %m",
+ path)));
+
+ data = XLogRecGetData(r) + sizeof(*xlrec);
+
+ len = xlrec->num_mappings * sizeof(LogicalRewriteMappingData);
+
+ /* write out tail end of mapping file (again) */
+ if (write(fd, data, len) != len)
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("Could not write to logical remapping file \"%s\": %m",
+ path)));
+ /*
+ * Now fsync all previously written data. We could improve things and only
+ * do this for the last write to a file, but the required bookkeeping
+ * doesn't seem worth the trouble.
+ */
+ if (pg_fsync(fd) != 0)
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("Could not fsync logical remapping file \"%s\": %m",
+ path)));
+
+ CloseTransientFile(fd);
+}
+
+/* ---
+ * Perform a checkpoint for logical rewrite mappings
+ *
+ * This serves two tasks:
+ * 1) Remove all mappings not needed anymore based on the logical restart LSN
+ * 2) fsync all remaining mappings so that replay after a checkpoints only has
+ * to deal with the parts of a mapping that have been written out after the
+ * checkpoint started.
+ * ---
+ */
+void
+CheckpointLogicalRewriteHeap(void)
+{
+ XLogRecPtr cutoff;
+ XLogRecPtr redo;
+ DIR *mappings_dir;
+ struct dirent *mapping_de;
+ char path[MAXPGPATH];
+
+ /*
+ * We start of with a minimum of the last redo pointer. No new decoding
+ * slot will start before that, so that's a safe upper bound for removal.
+ */
+ redo = GetRedoRecPtr();
+
+ /* now check for the restart ptrs from existing slots */
+ cutoff = ComputeLogicalRestartLSN();
+
+ /* don't start earlier than the restart lsn */
+ if (cutoff != InvalidXLogRecPtr && redo < cutoff)
+ cutoff = redo;
+
+ mappings_dir = AllocateDir("pg_llog/mappings");
+ while ((mapping_de = ReadDir(mappings_dir, "pg_llog/mappings")) != NULL)
+ {
+ struct stat statbuf;
+ Oid dboid;
+ Oid relid;
+ XLogRecPtr lsn;
+ TransactionId rewrite_xid;
+ TransactionId create_xid;
+
+ if (strcmp(mapping_de->d_name, ".") == 0 ||
+ strcmp(mapping_de->d_name, "..") == 0)
+ continue;
+
+ snprintf(path, MAXPGPATH, "pg_llog/mappings/%s", mapping_de->d_name);
+ if (lstat(path, &statbuf) == 0 && !S_ISREG(statbuf.st_mode))
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("only regular files expected in pg_llog/mappings/ but found \"%s\"",
+ mapping_de->d_name)));
+
+ /* XXX: should we warn about such files? */
+ if (strncmp(mapping_de->d_name, "map-", 4) != 0)
+ continue;
+
+ if (sscanf(mapping_de->d_name, LOGICAL_REWRITE_FORMAT,
+ &dboid, &relid, &lsn, &rewrite_xid, &create_xid) != 5)
+ elog(ERROR, "could not parse logical rewrite filename \"%s\"",
+ mapping_de->d_name);
+
+ if (lsn < cutoff || cutoff == InvalidXLogRecPtr)
+ {
+ elog(DEBUG1, "removing logical rewrite file \"%s\"", path);
+ if (unlink(path) < 0)
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("failed while unlinking logical rewrite file \"%s\": %m",
+ path)));
+ }
+ else
+ {
+ int fd = OpenTransientFile(path, O_RDONLY | PG_BINARY, 0);
+
+ /*
+ * The file cannot vanish due to concurrency since this function
+ * is the only one removing logical mappings and it's run while
+ * CheckpointLock is held exclusively.
+ */
+ if (fd < 0)
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("failure while opening logical remapping file \"%s\": %m",
+ path)));
+ /*
+ * We could try to avoid fsyncing files that either haven't
+ * changed or have only been created since the checkpoint's start,
+ * but it's currently not deemed worth the effort.
+ */
+ else if (pg_fsync(fd) != 0)
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("Could not fsync logical remapping file \"%s\": %m",
+ path)));
+ CloseTransientFile(fd);
+ }
+ }
+ FreeDir(mappings_dir);
+}
xlrec->node.relNode, xlrec->block,
xlrec->latestRemovedXid);
}
+ else if (info == XLOG_HEAP2_REWRITE)
+ {
+ appendStringInfoString(buf, "heap rewrite:");
+ }
else if (info == XLOG_HEAP2_CLEANUP_INFO)
{
xl_heap_cleanup_info *xlrec = (xl_heap_cleanup_info *) rec;
#include "access/clog.h"
#include "access/multixact.h"
+#include "access/rewriteheap.h"
#include "access/subtrans.h"
#include "access/timeline.h"
#include "access/transam.h"
#include "pgstat.h"
#include "postmaster/bgwriter.h"
#include "postmaster/startup.h"
+#include "replication/logical.h"
+#include "replication/snapbuild.h"
#include "replication/walreceiver.h"
#include "replication/walsender.h"
#include "storage/barrier.h"
XLogCtl->ckptXidEpoch = checkPoint.nextXidEpoch;
XLogCtl->ckptXid = checkPoint.nextXid;
+
+ /*
+ * Startup logical state, needs to be setup now so we have proper data
+ * during restore.
+ */
+ StartupLogicalDecoding(checkPoint.redo);
+
/*
* Startup MultiXact. We need to do this early for two reasons: one
* is that we might try to access multixacts when we do tuple freezing,
* StartupSUBTRANS hasn't been called yet.
*/
if (!RecoveryInProgress())
- TruncateSUBTRANS(GetOldestXmin(true, false));
+ TruncateSUBTRANS(GetOldestXmin(true, false, false));
/* Real work is done, but log and update stats before releasing lock. */
LogCheckpointEnd(false);
CheckPointMultiXact();
CheckPointPredicate();
CheckPointRelationMap();
+ CheckPointSnapBuild();
+ CheckpointLogicalRewriteHeap();
CheckPointBuffers(flags); /* performs all required fsyncs */
/* We deliberately delay 2PC checkpointing as long as possible */
CheckPointTwoPhase(checkPointRedo);
* this because StartupSUBTRANS hasn't been called yet.
*/
if (EnableHotStandby)
- TruncateSUBTRANS(GetOldestXmin(true, false));
+ TruncateSUBTRANS(GetOldestXmin(true, false, false));
/* Real work is done, but log and update before releasing lock. */
LogCheckpointEnd(true);
{
snapshot = SnapshotAny;
/* okay to ignore lazy VACUUMs here */
- OldestXmin = GetOldestXmin(heapRelation->rd_rel->relisshared, true);
+ OldestXmin = GetOldestXmin(heapRelation->rd_rel->relisshared, true, false);
}
scan = heap_beginscan_strat(heapRelation, /* relation */
WHERE S.usesysid = U.oid AND
S.pid = W.pid;
+CREATE VIEW pg_stat_logical_decoding AS
+ SELECT
+ L.slot_name,
+ L.plugin,
+ L.datoid,
+ D.datname AS database,
+ L.active,
+ L.xmin,
+ L.restart_decoding_lsn
+ FROM pg_stat_get_logical_decoding_slots() AS L
+ LEFT JOIN pg_database D ON (L.datoid = D.oid);
+
CREATE VIEW pg_stat_database AS
SELECT
D.oid AS datid,
totalblocks = RelationGetNumberOfBlocks(onerel);
/* Need a cutoff xmin for HeapTupleSatisfiesVacuum */
- OldestXmin = GetOldestXmin(onerel->rd_rel->relisshared, true);
+ OldestXmin = GetOldestXmin(onerel->rd_rel->relisshared, true, false);
/* Prepare for sampling block numbers */
BlockSampler_Init(&bs, totalblocks, targrows);
is_system_catalog = IsSystemRelation(OldHeap);
/* Initialize the rewrite operation */
- rwstate = begin_heap_rewrite(NewHeap, OldestXmin, FreezeXid,
+ rwstate = begin_heap_rewrite(OldHeap, NewHeap, OldestXmin, FreezeXid,
MultiXactCutoff, use_wal);
/*
#include "miscadmin.h"
#include "pgstat.h"
#include "postmaster/bgwriter.h"
+#include "replication/logical.h"
#include "storage/copydir.h"
#include "storage/fd.h"
#include "storage/lmgr.h"
HeapTuple tup;
int notherbackends;
int npreparedxacts;
+ int nslots, nslots_active;
/*
* Look up the target database's OID, and get exclusive lock on it. We
(errcode(ERRCODE_OBJECT_IN_USE),
errmsg("cannot drop the currently open database")));
+ /*
+ * Check whether there are, possibly unconnected, logical
+ * slots that refer to the to-be-dropped database. The database
+ * lock we are holding prevents the creation of new slots using
+ * the database.
+ */
+ if (LogicalDecodingCountDBSlots(db_id, &nslots, &nslots_active))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_IN_USE),
+ errmsg("database \"%s\" is used in a logical decoding slot",
+ dbname),
+ errdetail("There are %d slot(s), %d of them active",
+ nslots, nslots_active)));
+
/*
* Check for other backends in the target database. (Because we hold the
* database lock, no new ones can start after this.)
* working on a particular table at any time, and that each vacuum is
* always an independent transaction.
*/
- *oldestXmin = GetOldestXmin(sharedRel, true);
+ *oldestXmin = GetOldestXmin(sharedRel, true, false);
Assert(TransactionIdIsNormal(*oldestXmin));
* committed pg_class entries for new tables; see AddNewRelationTuple().
* So we cannot produce a wrong minimum by starting with this.
*/
- newFrozenXid = GetOldestXmin(true, true);
+ newFrozenXid = GetOldestXmin(true, true, false);
/*
* Similarly, initialize the MultiXact "min" with the value that would be
OBJS = walsender.o walreceiverfuncs.o walreceiver.o basebackup.o \
repl_gram.o syncrep.o
+SUBDIRS = logical
+
include $(top_srcdir)/src/backend/common.mk
# repl_scanner is compiled as part of repl_gram
--- /dev/null
+#-------------------------------------------------------------------------
+#
+# Makefile--
+# Makefile for src/backend/replication/logical
+#
+# IDENTIFICATION
+# src/backend/replication/logical/Makefile
+#
+#-------------------------------------------------------------------------
+
+subdir = src/backend/replication/logical
+top_builddir = ../../../..
+include $(top_builddir)/src/Makefile.global
+
+override CPPFLAGS := -I$(srcdir) $(CPPFLAGS)
+
+OBJS = decode.o logical.o logicalfuncs.o reorderbuffer.o snapbuild.o
+
+include $(top_srcdir)/src/backend/common.mk
--- /dev/null
+/* -------------------------------------------------------------------------
+ *
+ * decode.c
+ * Decodes WAL records fed from xlogreader.h read into an reorderbuffer
+ * while simultaneously letting snapbuild.c build an appropriate
+ * snapshots to decode those.
+ *
+ * NOTE:
+ * This basically tries to handle all low level xlog stuff for
+ * reorderbuffer.c and snapbuild.c. There's some minor leakage where a
+ * specific record's struct is used to pass data along, but that's just
+ * because those are convenient and uncomplicated to read.
+ *
+ * Portions Copyright (c) 1996-2013, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/replication/logical/decode.c
+ *
+ * -------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "miscadmin.h"
+
+#include "access/heapam.h"
+#include "access/heapam_xlog.h"
+#include "access/transam.h"
+#include "access/xact.h"
+#include "access/xlog_internal.h"
+#include "access/xlogreader.h"
+
+#include "catalog/pg_control.h"
+
+#include "replication/decode.h"
+#include "replication/logical.h"
+#include "replication/reorderbuffer.h"
+#include "replication/snapbuild.h"
+
+#include "storage/standby.h"
+
+/* RMGR Handlers */
+static void DecodeXLogOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
+static void DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
+static void DecodeHeap2Op(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
+static void DecodeXactOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
+static void DecodeStandbyOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
+
+/* individual record(group)'s handlers */
+static void DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
+static void DecodeUpdate(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
+static void DecodeDelete(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
+static void DecodeMultiInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
+static void DecodeCommit(LogicalDecodingContext *ctx, XLogRecordBuffer *buf,
+ TransactionId xid, TimestampTz commit_time,
+ int nsubxacts, TransactionId *sub_xids,
+ int ninval_msgs, SharedInvalidationMessage *msg);
+static void DecodeAbort(LogicalDecodingContext *ctx, XLogRecPtr lsn,
+ TransactionId xid, TransactionId *sub_xids, int nsubxacts);
+
+/* common function to decode tuples */
+static void DecodeXLogTuple(char *data, Size len, ReorderBufferTupleBuf *tup);
+
+/*
+ * Take every XLogReadRecord()ed record and perform the actions required to
+ * decode it using the output plugin already setup in the logical decoding
+ * context.
+ */
+void
+DecodeRecordIntoReorderBuffer(LogicalDecodingContext *ctx,
+ XLogRecordBuffer *buf)
+{
+ /* cast so we get a warning when new rmgrs are added */
+ switch ((RmgrIds) buf->record.xl_rmid)
+ {
+ /*
+ * Rmgrs we care about for logical decoding. Add new rmgrs in
+ * rmgrlist.h's order.
+ */
+ case RM_XLOG_ID:
+ DecodeXLogOp(ctx, buf);
+ break;
+
+ case RM_XACT_ID:
+ DecodeXactOp(ctx, buf);
+ break;
+
+ case RM_STANDBY_ID:
+ DecodeStandbyOp(ctx, buf);
+ break;
+
+ case RM_HEAP2_ID:
+ DecodeHeap2Op(ctx, buf);
+ break;
+
+ case RM_HEAP_ID:
+ DecodeHeapOp(ctx, buf);
+ break;
+
+ /*
+ * Rmgrs irrelevant for changeset extraction, they describe stuff not
+ * represented in logical decoding. Add new rmgrs in rmgrlist.h's
+ * order.
+ */
+ case RM_SMGR_ID:
+ case RM_CLOG_ID:
+ case RM_DBASE_ID:
+ case RM_TBLSPC_ID:
+ case RM_MULTIXACT_ID:
+ case RM_RELMAP_ID:
+ case RM_BTREE_ID:
+ case RM_HASH_ID:
+ case RM_GIN_ID:
+ case RM_GIST_ID:
+ case RM_SEQ_ID:
+ case RM_SPGIST_ID:
+ break;
+ case RM_NEXT_ID:
+ default:
+ elog(ERROR, "unexpected RM_NEXT_ID rmgr_id");
+ }
+}
+
+/*
+ * Handle rmgr XLOG_ID records for DecodeRecordIntoReorderBuffer().
+ */
+static void
+DecodeXLogOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
+{
+ SnapBuild *builder = ctx->snapshot_builder;
+ uint8 info = buf->record.xl_info & ~XLR_INFO_MASK;
+
+ switch (info)
+ {
+ /* this is also used in END_OF_RECOVERY checkpoints */
+ case XLOG_CHECKPOINT_SHUTDOWN:
+ case XLOG_END_OF_RECOVERY:
+ SnapBuildSerializationPoint(builder, buf->origptr);
+
+ break;
+ case XLOG_CHECKPOINT_ONLINE:
+ /*
+ * a RUNNING_XACTS record will have been logged near to this, we
+ * can restart from there.
+ */
+ break;
+ case XLOG_NOOP:
+ case XLOG_NEXTOID:
+ case XLOG_SWITCH:
+ case XLOG_BACKUP_END:
+ case XLOG_PARAMETER_CHANGE:
+ case XLOG_RESTORE_POINT:
+ case XLOG_FPW_CHANGE:
+ case XLOG_FPI:
+ break;
+ default:
+ elog(ERROR, "unexpected RM_XLOG_ID record type: %u", info);
+ }
+}
+
+/*
+ * Handle rmgr XACT_ID records for DecodeRecordIntoReorderBuffer().
+ */
+static void
+DecodeXactOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
+{
+ SnapBuild *builder = ctx->snapshot_builder;
+ ReorderBuffer *reorder = ctx->reorder;
+ XLogRecord *r = &buf->record;
+ uint8 info = r->xl_info & ~XLR_INFO_MASK;
+
+ /* no point in doing anything yet */
+ if (SnapBuildCurrentState(builder) < SNAPBUILD_FULL_SNAPSHOT)
+ return;
+
+ switch (info)
+ {
+ case XLOG_XACT_COMMIT:
+ {
+ xl_xact_commit *xlrec;
+ TransactionId *subxacts = NULL;
+ SharedInvalidationMessage *invals = NULL;
+
+ xlrec = (xl_xact_commit *) buf->record_data;
+
+ subxacts = (TransactionId *) &(xlrec->xnodes[xlrec->nrels]);
+ invals = (SharedInvalidationMessage *) &(subxacts[xlrec->nsubxacts]);
+
+ /* FIXME: skip if wrong db? */
+
+ DecodeCommit(ctx, buf, r->xl_xid, xlrec->xact_time,
+ xlrec->nsubxacts, subxacts,
+ xlrec->nmsgs, invals);
+
+ break;
+ }
+ case XLOG_XACT_COMMIT_PREPARED:
+ {
+ xl_xact_commit_prepared *prec;
+ xl_xact_commit *xlrec;
+ TransactionId *subxacts;
+ SharedInvalidationMessage *invals = NULL;
+
+ /* Prepared commits contain a normal commit record... */
+ prec = (xl_xact_commit_prepared *) buf->record_data;
+ xlrec = &prec->crec;
+
+ subxacts = (TransactionId *) &(xlrec->xnodes[xlrec->nrels]);
+ invals = (SharedInvalidationMessage *) &(subxacts[xlrec->nsubxacts]);
+
+ /* FIXME: skip if wrong db? */
+
+ DecodeCommit(ctx, buf, r->xl_xid,
+ xlrec->xact_time,
+ xlrec->nsubxacts, subxacts,
+ xlrec->nmsgs, invals);
+
+ break;
+ }
+ case XLOG_XACT_COMMIT_COMPACT:
+ {
+ xl_xact_commit_compact *xlrec;
+
+ xlrec = (xl_xact_commit_compact *) buf->record_data;
+
+ /* FIXME: skip if wrong db? */
+
+ DecodeCommit(ctx, buf, r->xl_xid, xlrec->xact_time,
+ xlrec->nsubxacts, xlrec->subxacts,
+ 0, NULL);
+ break;
+ }
+ case XLOG_XACT_ABORT:
+ {
+ xl_xact_abort *xlrec;
+ TransactionId *sub_xids;
+
+ xlrec = (xl_xact_abort *) buf->record_data;
+
+ sub_xids = (TransactionId *) &(xlrec->xnodes[xlrec->nrels]);
+
+ DecodeAbort(ctx, buf->origptr, r->xl_xid,
+ sub_xids, xlrec->nsubxacts);
+ break;
+ }
+ case XLOG_XACT_ABORT_PREPARED:
+ {
+ xl_xact_abort_prepared *prec;
+ xl_xact_abort *xlrec;
+ TransactionId *sub_xids;
+
+ /* prepared abort contain a normal commit abort... */
+ prec = (xl_xact_abort_prepared *) buf->record_data;
+ xlrec = &prec->arec;
+
+ sub_xids = (TransactionId *) &(xlrec->xnodes[xlrec->nrels]);
+
+ /* r->xl_xid is committed in a separate record */
+ DecodeAbort(ctx, buf->origptr, prec->xid,
+ sub_xids, xlrec->nsubxacts);
+ break;
+ }
+
+ case XLOG_XACT_ASSIGNMENT:
+ {
+ xl_xact_assignment *xlrec;
+ int i;
+ TransactionId *sub_xid;
+
+ xlrec = (xl_xact_assignment *) buf->record_data;
+
+ /* FIXME: skip based on database */
+
+ sub_xid = &xlrec->xsub[0];
+
+ for (i = 0; i < xlrec->nsubxacts; i++)
+ {
+ ReorderBufferAssignChild(reorder, xlrec->xtop,
+ *(sub_xid++), buf->origptr);
+ }
+ break;
+ }
+ case XLOG_XACT_PREPARE:
+
+ /* XXX: we could replay the transaction and prepare it as well. */
+ break;
+ default:
+ elog(ERROR, "unexpected RM_XACT_ID record type: %u", info);
+ }
+}
+
+/*
+ * Handle rmgr STANDBY_ID records for DecodeRecordIntoReorderBuffer().
+ */
+static void
+DecodeStandbyOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
+{
+ SnapBuild *builder = ctx->snapshot_builder;
+ XLogRecord *r = &buf->record;
+ uint8 info = r->xl_info & ~XLR_INFO_MASK;
+
+ switch (info)
+ {
+ case XLOG_RUNNING_XACTS:
+ {
+ xl_running_xacts *running = (xl_running_xacts *) buf->record_data;
+ SnapBuildProcessRunningXacts(builder, buf->origptr, running);
+ /*
+ * Abort all transactions that we keep track of that are older
+ * than ->oldestRunningXid. This is the most convenient spot
+ * for doing so since, in contrast to shutdown or end of
+ * recover checkpoints, we have sufficient knowledge to deal
+ * with prepared transactions here.
+ */
+ ReorderBufferAbortOld(ctx->reorder, running->oldestRunningXid);
+ }
+ break;
+ case XLOG_STANDBY_LOCK:
+ break;
+ default:
+ elog(ERROR, "unexpected RM_STANDBY_ID record type: %u", info);
+ }
+}
+
+/*
+ * Handle rmgr HEAP2_ID records for DecodeRecordIntoReorderBuffer().
+ */
+static void
+DecodeHeap2Op(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
+{
+ uint8 info = buf->record.xl_info & XLOG_HEAP_OPMASK;
+ TransactionId xid = buf->record.xl_xid;
+ SnapBuild *builder = ctx->snapshot_builder;
+
+ /* no point in doing anything yet */
+ if (SnapBuildCurrentState(builder) < SNAPBUILD_FULL_SNAPSHOT)
+ return;
+
+ switch (info)
+ {
+ case XLOG_HEAP2_MULTI_INSERT:
+ if (SnapBuildProcessChange(builder, xid, buf->origptr))
+ DecodeMultiInsert(ctx, buf);
+ break;
+ case XLOG_HEAP2_NEW_CID:
+ {
+ xl_heap_new_cid *xlrec;
+ xlrec = (xl_heap_new_cid *) buf->record_data;
+ SnapBuildProcessNewCid(builder, xid, buf->origptr, xlrec);
+
+ break;
+ }
+ case XLOG_HEAP2_REWRITE:
+ /* only crash recovery/replication needs to care */
+ break;
+
+ /*
+ * Everything else here is just low level physical stuff we're
+ * not interested in.
+ */
+ case XLOG_HEAP2_FREEZE:
+ case XLOG_HEAP2_CLEAN:
+ case XLOG_HEAP2_CLEANUP_INFO:
+ case XLOG_HEAP2_VISIBLE:
+ case XLOG_HEAP2_LOCK_UPDATED:
+ break;
+ default:
+ elog(ERROR, "unexpected RM_HEAP2_ID record type: %u", info);
+ }
+}
+
+/*
+ * Handle rmgr HEAP_ID records for DecodeRecordIntoReorderBuffer().
+ */
+static void
+DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
+{
+ uint8 info = buf->record.xl_info & XLOG_HEAP_OPMASK;
+ TransactionId xid = buf->record.xl_xid;
+ SnapBuild *builder = ctx->snapshot_builder;
+
+ /* no point in doing anything yet */
+ if (SnapBuildCurrentState(builder) < SNAPBUILD_FULL_SNAPSHOT)
+ return;
+
+ switch (info)
+ {
+ case XLOG_HEAP_INSERT:
+ if (SnapBuildProcessChange(builder, xid, buf->origptr))
+ DecodeInsert(ctx, buf);
+ break;
+
+ /*
+ * Treat HOT update as normal updates, there is no useful
+ * information in the fact that we could make it a HOT update
+ * locally and the WAL layout is compatible.
+ */
+ case XLOG_HEAP_HOT_UPDATE:
+ case XLOG_HEAP_UPDATE:
+ if (SnapBuildProcessChange(builder, xid, buf->origptr))
+ DecodeUpdate(ctx, buf);
+ break;
+
+ case XLOG_HEAP_DELETE:
+ if (SnapBuildProcessChange(builder, xid, buf->origptr))
+ DecodeDelete(ctx, buf);
+ break;
+
+ case XLOG_HEAP_NEWPAGE:
+ /*
+ * XXX: There doesn't seem to be a usecase for decoding
+ * HEAP_NEWPAGE's. Its only used in various indexam's and CLUSTER,
+ * neither of which should be relevant for the logical
+ * changestream.
+ */
+ break;
+
+ case XLOG_HEAP_INPLACE:
+ /*
+ * Cannot be important for our purposes, not part of transactions.
+ */
+ if (!TransactionIdIsValid(xid))
+ break;
+
+ SnapBuildProcessChange(builder, xid, buf->origptr);
+ /* heap_inplace is only done in catalog modifying txns */
+ ReorderBufferXidSetCatalogChanges(ctx->reorder, xid, buf->origptr);
+ break;
+
+ /* we don't care about row level locks for now */
+ case XLOG_HEAP_LOCK:
+ break;
+
+ default:
+ elog(ERROR, "unexpected RM_HEAP_ID record type: %u", info);
+ break;
+ }
+}
+
+/*
+ * Get the data from the various forms of commit records and pass it
+ * on to snapbuild.c and reorderbuffer.c
+ */
+static void
+DecodeCommit(LogicalDecodingContext *ctx, XLogRecordBuffer *buf,
+ TransactionId xid, TimestampTz commit_time,
+ int nsubxacts, TransactionId *sub_xids,
+ int ninval_msgs, SharedInvalidationMessage *msgs)
+{
+ int i;
+
+ /* always need the invalidation messages */
+ if (ninval_msgs > 0)
+ {
+ ReorderBufferAddInvalidations(ctx->reorder, xid, buf->origptr,
+ ninval_msgs, msgs);
+ ReorderBufferXidSetCatalogChanges(ctx->reorder, xid, buf->origptr);
+ }
+
+ SnapBuildCommitTxn(ctx->snapshot_builder, buf->origptr, xid,
+ nsubxacts, sub_xids);
+
+ /*
+ * We might not be interested in decoding transactions up to this
+ * LSN. This can happen because we previously decoded it and now just are
+ * restarting or we haven't assembled a consistent snapshot.
+ *
+ * If we're not interested just tell ReorderBuffer it's an abort and make
+ * it throw away the data.
+ */
+ if (SnapBuildXactNeedsSkip(ctx->snapshot_builder, buf->origptr))
+ {
+ /*
+ * XXX: At some point we might want to execute the transaction's
+ * invalidations here. Currently skips will only happen after we've
+ * invalidated the entire cache after initially starting logically
+ * decoding, but that might change at some point when we find it
+ * necessary to use a more fine-grained mechanism.
+ */
+ for (i = 0; i < nsubxacts; i++)
+ {
+ ReorderBufferAbort(ctx->reorder, *sub_xids, buf->origptr);
+ sub_xids++;
+ }
+ ReorderBufferAbort(ctx->reorder, xid, buf->origptr);
+
+ return;
+ }
+
+ for (i = 0; i < nsubxacts; i++)
+ {
+ ReorderBufferCommitChild(ctx->reorder, xid, *sub_xids,
+ buf->origptr, buf->endptr);
+ sub_xids++;
+ }
+
+ /* replay actions of all transaction + subtransactions in order */
+ ReorderBufferCommit(ctx->reorder, xid, buf->origptr, buf->endptr,
+ commit_time);
+}
+
+/*
+ * Get the data from the various forms of abort records and pass it on to
+ * snapbuild.c and reorderbuffer.c
+ */
+static void
+DecodeAbort(LogicalDecodingContext *ctx, XLogRecPtr lsn, TransactionId xid,
+ TransactionId *sub_xids, int nsubxacts)
+{
+ int i;
+
+ SnapBuildAbortTxn(ctx->snapshot_builder, lsn, xid, nsubxacts, sub_xids);
+
+ for (i = 0; i < nsubxacts; i++)
+ {
+ ReorderBufferAbort(ctx->reorder, *sub_xids, lsn);
+ sub_xids++;
+ }
+
+ ReorderBufferAbort(ctx->reorder, xid, lsn);
+}
+
+/*
+ * Parse XLOG_HEAP_INSERT (not MULTI_INSERT!) records into tuplebufs.
+ *
+ * Deletes can contain the new tuple.
+ */
+static void
+DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
+{
+ XLogRecord *r = &buf->record;
+ xl_heap_insert *xlrec;
+ ReorderBufferChange *change;
+
+ xlrec = (xl_heap_insert *) buf->record_data;
+
+ /* only interested in our database */
+ if (xlrec->target.node.dbNode != ctx->slot->database)
+ return;
+
+ change = ReorderBufferGetChange(ctx->reorder);
+ change->action = REORDER_BUFFER_CHANGE_INSERT;
+ memcpy(&change->tp.relnode, &xlrec->target.node, sizeof(RelFileNode));
+
+ if (xlrec->flags & XLOG_HEAP_CONTAINS_NEW_TUPLE)
+ {
+ Assert(r->xl_len > (SizeOfHeapInsert + SizeOfHeapHeader));
+
+ change->tp.newtuple = ReorderBufferGetTupleBuf(ctx->reorder);
+
+ DecodeXLogTuple((char *) xlrec + SizeOfHeapInsert,
+ r->xl_len - SizeOfHeapInsert,
+ change->tp.newtuple);
+ }
+
+ ReorderBufferQueueChange(ctx->reorder, r->xl_xid, buf->origptr, change);
+}
+
+/*
+ * Parse XLOG_HEAP_UPDATE and XLOG_HEAP_HOT_UPDATE, which have the same layout
+ * in the record, from wal into proper tuplebufs.
+ *
+ * Updates can possibly contain a new tuple and the old primary key.
+ */
+static void
+DecodeUpdate(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
+{
+ XLogRecord *r = &buf->record;
+ xl_heap_update *xlrec;
+ xl_heap_header_len *xlhdr;
+ ReorderBufferChange *change;
+ char *data;
+
+ xlrec = (xl_heap_update *) buf->record_data;
+ xlhdr = (xl_heap_header_len *) (buf->record_data + SizeOfHeapUpdate);
+
+ /* only interested in our database */
+ if (xlrec->target.node.dbNode != ctx->slot->database)
+ return;
+
+ change = ReorderBufferGetChange(ctx->reorder);
+ change->action = REORDER_BUFFER_CHANGE_UPDATE;
+ memcpy(&change->tp.relnode, &xlrec->target.node, sizeof(RelFileNode));
+
+ data = (char *) &xlhdr->header;
+
+ if (xlrec->flags & XLOG_HEAP_CONTAINS_NEW_TUPLE)
+ {
+ Assert(r->xl_len > (SizeOfHeapUpdate + SizeOfHeapHeaderLen));
+
+ change->tp.newtuple = ReorderBufferGetTupleBuf(ctx->reorder);
+
+ DecodeXLogTuple(data,
+ xlhdr->t_len + SizeOfHeapHeader,
+ change->tp.newtuple);
+ /* skip over the rest of the tuple header */
+ data += SizeOfHeapHeader;
+ /* skip over the tuple data */
+ data += xlhdr->t_len;
+ }
+
+ if (xlrec->flags & XLOG_HEAP_CONTAINS_OLD)
+ {
+ xlhdr = (xl_heap_header_len *) data;
+ change->tp.oldtuple = ReorderBufferGetTupleBuf(ctx->reorder);
+ DecodeXLogTuple((char *) &xlhdr->header,
+ xlhdr->t_len + SizeOfHeapHeader,
+ change->tp.oldtuple);
+ data = (char *) &xlhdr->header;
+ data += SizeOfHeapHeader;
+ data += xlhdr->t_len;
+ }
+
+ ReorderBufferQueueChange(ctx->reorder, r->xl_xid, buf->origptr, change);
+}
+
+/*
+ * Parse XLOG_HEAP_DELETE from wal into proper tuplebufs.
+ *
+ * Deletes can possibly contain the old primary key.
+ */
+static void
+DecodeDelete(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
+{
+ XLogRecord *r = &buf->record;
+ xl_heap_delete *xlrec;
+ ReorderBufferChange *change;
+
+ xlrec = (xl_heap_delete *) buf->record_data;
+
+ /* only interested in our database */
+ if (xlrec->target.node.dbNode != ctx->slot->database)
+ return;
+
+ change = ReorderBufferGetChange(ctx->reorder);
+ change->action = REORDER_BUFFER_CHANGE_DELETE;
+
+ memcpy(&change->tp.relnode, &xlrec->target.node, sizeof(RelFileNode));
+
+ /* old primary key stored */
+ if (xlrec->flags & XLOG_HEAP_CONTAINS_OLD)
+ {
+ Assert(r->xl_len > (SizeOfHeapDelete + SizeOfHeapHeader));
+
+ change->tp.oldtuple = ReorderBufferGetTupleBuf(ctx->reorder);
+
+ DecodeXLogTuple((char *) xlrec + SizeOfHeapDelete,
+ r->xl_len - SizeOfHeapDelete,
+ change->tp.oldtuple);
+ }
+ ReorderBufferQueueChange(ctx->reorder, r->xl_xid, buf->origptr, change);
+}
+
+/*
+ * Decode XLOG_HEAP2_MULTI_INSERT_insert record into multiple tuplebufs.
+ *
+ * Currently MULTI_INSERT will always contain the full tuples.
+ */
+static void
+DecodeMultiInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
+{
+ XLogRecord *r = &buf->record;
+ xl_heap_multi_insert *xlrec;
+ int i;
+ char *data;
+ bool isinit = (r->xl_info & XLOG_HEAP_INIT_PAGE) != 0;
+
+ xlrec = (xl_heap_multi_insert *) buf->record_data;
+
+ /* only interested in our database */
+ if (xlrec->node.dbNode != ctx->slot->database)
+ return;
+
+ data = buf->record_data + SizeOfHeapMultiInsert;
+
+ /*
+ * OffsetNumbers (which are not of interest to us) are stored when
+ * XLOG_HEAP_INIT_PAGE is not set -- skip over them.
+ */
+ if (!isinit)
+ data += sizeof(OffsetNumber) * xlrec->ntuples;
+
+ for (i = 0; i < xlrec->ntuples; i++)
+ {
+ ReorderBufferChange *change;
+ xl_multi_insert_tuple *xlhdr;
+ int datalen;
+ ReorderBufferTupleBuf *tuple;
+
+ change = ReorderBufferGetChange(ctx->reorder);
+ change->action = REORDER_BUFFER_CHANGE_INSERT;
+ memcpy(&change->tp.relnode, &xlrec->node, sizeof(RelFileNode));
+
+ /*
+ * CONTAINS_NEW_TUPLE will always be set currently as multi_insert
+ * isn't used for catalogs, but better be future proof.
+ *
+ * We decode the tuple in pretty much the same way as DecodeXLogTuple,
+ * but since the layout is slightly different, we can't use it here.
+ */
+ if (xlrec->flags & XLOG_HEAP_CONTAINS_NEW_TUPLE)
+ {
+ change->tp.newtuple = ReorderBufferGetTupleBuf(ctx->reorder);
+
+ tuple = change->tp.newtuple;
+
+ /* not a disk based tuple */
+ ItemPointerSetInvalid(&tuple->tuple.t_self);
+
+ xlhdr = (xl_multi_insert_tuple *) SHORTALIGN(data);
+ data = ((char *) xlhdr) + SizeOfMultiInsertTuple;
+ datalen = xlhdr->datalen;
+
+ /*
+ * We can only figure this out after reassembling the
+ * transactions.
+ */
+ tuple->tuple.t_tableOid = InvalidOid;
+ tuple->tuple.t_data = &tuple->header;
+ tuple->tuple.t_len = datalen
+ + offsetof(HeapTupleHeaderData, t_bits);
+
+ memset(&tuple->header, 0, sizeof(HeapTupleHeaderData));
+
+ memcpy((char *) &tuple->header
+ + offsetof(HeapTupleHeaderData, t_bits),
+ (char *) data,
+ datalen);
+ data += datalen;
+
+ tuple->header.t_infomask = xlhdr->t_infomask;
+ tuple->header.t_infomask2 = xlhdr->t_infomask2;
+ tuple->header.t_hoff = xlhdr->t_hoff;
+ }
+
+ ReorderBufferQueueChange(ctx->reorder, r->xl_xid,
+ buf->origptr, change);
+ }
+}
+
+/*
+ * Read a HeapTuple as WAL logged by heap_insert, heap_update and
+ * heap_delete, but not by heap_multi_insert into a tuplebuf.
+ *
+ * The size 'len' and the pointer 'data' in the record need to be
+ * computed outside as they are record specific.
+ */
+static void
+DecodeXLogTuple(char *data, Size len, ReorderBufferTupleBuf *tuple)
+{
+ xl_heap_header xlhdr;
+ int datalen = len - SizeOfHeapHeader;
+
+ Assert(datalen >= 0);
+ Assert(datalen <= MaxHeapTupleSize);
+
+ tuple->tuple.t_len = datalen + offsetof(HeapTupleHeaderData, t_bits);
+
+ /* not a disk based tuple */
+ ItemPointerSetInvalid(&tuple->tuple.t_self);
+
+ /* we can only figure this out after reassembling the transactions */
+ tuple->tuple.t_tableOid = InvalidOid;
+ tuple->tuple.t_data = &tuple->header;
+
+ /* data is not stored aligned, copy to aligned storage */
+ memcpy((char *) &xlhdr,
+ data,
+ SizeOfHeapHeader);
+
+ memset(&tuple->header, 0, sizeof(HeapTupleHeaderData));
+
+ memcpy((char *) &tuple->header + offsetof(HeapTupleHeaderData, t_bits),
+ data + SizeOfHeapHeader,
+ datalen);
+
+ tuple->header.t_infomask = xlhdr.t_infomask;
+ tuple->header.t_infomask2 = xlhdr.t_infomask2;
+ tuple->header.t_hoff = xlhdr.t_hoff;
+}
--- /dev/null
+/*-------------------------------------------------------------------------
+ *
+ * logical.c
+ *
+ * Logical decoding shared memory management
+ *
+ *
+ * Copyright (c) 2012-2013, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ * src/backend/replication/logical/logical.c
+ *
+ */
+
+#include "postgres.h"
+
+#include <unistd.h>
+#include <sys/stat.h>
+
+#include "access/transam.h"
+
+#include "fmgr.h"
+#include "miscadmin.h"
+
+#include "replication/logical.h"
+#include "replication/reorderbuffer.h"
+#include "replication/snapbuild.h"
+#include "storage/ipc.h"
+#include "storage/procarray.h"
+#include "storage/fd.h"
+
+#include "utils/memutils.h"
+#include "utils/syscache.h"
+
+/*
+ * logical decoding on-disk data structures.
+ */
+typedef struct LogicalDecodingSlotOnDisk
+{
+ /* first part of this struct needs to be version independent */
+
+ /* data not covered by checksum */
+ uint32 magic;
+ pg_crc32 checksum;
+
+ /* data covered by checksum */
+ uint32 version;
+ uint32 length;
+
+ /* data with potentially evolving format */
+ LogicalDecodingSlot slot;
+} LogicalDecodingSlotOnDisk;
+
+/* size of the part of the slot that is version independent */
+#define LogicalDecodingSlotOnDiskConstantSize \
+ offsetof(LogicalDecodingSlotOnDisk, slot)
+/* size of the slots that is not version indepenent */
+#define LogicalDecodingSlotOnDiskDynamicSize \
+ sizeof(LogicalDecodingSlotOnDisk) - LogicalDecodingSlotOnDiskConstantSize
+
+#define LOGICAL_MAGIC 0x1051CA1 /* format identifier */
+#define LOGICAL_VERSION 1 /* version for new files */
+
+/* Control array for logical decoding */
+LogicalDecodingCtlData *LogicalDecodingCtl = NULL;
+
+/* My slot for logical rep in the shared memory array */
+LogicalDecodingSlot *MyLogicalDecodingSlot = NULL;
+
+/* user settable parameters */
+int max_logical_slots = 0; /* the maximum number of logical slots */
+
+static void LogicalSlotKill(int code, Datum arg);
+
+/* persistency functions */
+static void RestoreLogicalSlot(const char *name);
+static void CreateLogicalSlot(LogicalDecodingSlot *slot);
+static void SaveLogicalSlot(LogicalDecodingSlot *slot);
+static void SaveLogicalSlotInternal(LogicalDecodingSlot *slot, const char *path);
+static void DeleteLogicalSlot(LogicalDecodingSlot *slot);
+
+
+/* Report shared-memory space needed by LogicalDecodingShmemInit */
+Size
+LogicalDecodingShmemSize(void)
+{
+ Size size = 0;
+
+ if (max_logical_slots == 0)
+ return size;
+
+ size = offsetof(LogicalDecodingCtlData, logical_slots);
+ size = add_size(size,
+ mul_size(max_logical_slots, sizeof(LogicalDecodingSlot)));
+
+ return size;
+}
+
+/* Allocate and initialize walsender-related shared memory */
+void
+LogicalDecodingShmemInit(void)
+{
+ bool found;
+
+ if (max_logical_slots == 0)
+ return;
+
+ LogicalDecodingCtl = (LogicalDecodingCtlData *)
+ ShmemInitStruct("Logical Decoding Ctl", LogicalDecodingShmemSize(),
+ &found);
+
+ if (!found)
+ {
+ int i;
+
+ /* First time through, so initialize */
+ MemSet(LogicalDecodingCtl, 0, LogicalDecodingShmemSize());
+
+ LogicalDecodingCtl->xmin = InvalidTransactionId;
+
+ for (i = 0; i < max_logical_slots; i++)
+ {
+ LogicalDecodingSlot *slot =
+ &LogicalDecodingCtl->logical_slots[i];
+
+ slot->xmin = InvalidTransactionId;
+ slot->effective_xmin = InvalidTransactionId;
+ SpinLockInit(&slot->mutex);
+ }
+ }
+}
+
+/* mark the currently used slot as unused */
+static void
+LogicalSlotKill(int code, Datum arg)
+{
+ /* LOCK? */
+ if (MyLogicalDecodingSlot && MyLogicalDecodingSlot->active)
+ {
+ MyLogicalDecodingSlot->active = false;
+ }
+ MyLogicalDecodingSlot = NULL;
+}
+
+/*
+ * Set the xmin required for decoding snapshots for the specific decoding
+ * slot.
+ */
+void
+IncreaseLogicalXminForSlot(XLogRecPtr lsn, TransactionId xmin)
+{
+ bool updated_xmin = false;
+
+ Assert(MyLogicalDecodingSlot != NULL);
+
+ SpinLockAcquire(&MyLogicalDecodingSlot->mutex);
+
+ /*
+ * don't overwrite if we already have a newer xmin. This can
+ * happen if we restart decoding in a slot.
+ */
+ if (TransactionIdPrecedesOrEquals(xmin, MyLogicalDecodingSlot->xmin))
+ {
+ }
+ /*
+ * If the client has already confirmed up to this lsn, we directly
+ * can mark this as accepted. This can happen if we restart
+ * decoding in a slot.
+ */
+ else if (lsn <= MyLogicalDecodingSlot->confirmed_flush)
+ {
+ MyLogicalDecodingSlot->candidate_xmin = xmin;
+ MyLogicalDecodingSlot->candidate_xmin_lsn = lsn;
+
+ /* our candidate can directly be used */
+ updated_xmin = true;
+ }
+ /*
+ * Only increase if the previous values have been applied, otherwise we
+ * might never end up updating if the receiver acks too slowly.
+ */
+ else if (MyLogicalDecodingSlot->candidate_xmin_lsn == InvalidXLogRecPtr)
+ {
+ MyLogicalDecodingSlot->candidate_xmin = xmin;
+ MyLogicalDecodingSlot->candidate_xmin_lsn = lsn;
+ elog(DEBUG1, "got new xmin %u at %X/%X", xmin,
+ (uint32) (lsn >> 32), (uint32) lsn);
+ }
+ SpinLockRelease(&MyLogicalDecodingSlot->mutex);
+
+ /* candidate already valid with the current flush position, apply */
+ if (updated_xmin)
+ LogicalConfirmReceivedLocation(MyLogicalDecodingSlot->confirmed_flush);
+}
+
+/*
+ * Mark the minimal LSN (restart_lsn) we need to read to replay all
+ * transactions that have not yet committed at current_lsn. Only takes
+ * effect when the client has confirmed to have received current_lsn.
+ */
+void
+IncreaseRestartDecodingForSlot(XLogRecPtr current_lsn, XLogRecPtr restart_lsn)
+{
+ bool updated_lsn = false;
+
+ Assert(MyLogicalDecodingSlot != NULL);
+ Assert(restart_lsn != InvalidXLogRecPtr);
+ Assert(current_lsn != InvalidXLogRecPtr);
+
+ SpinLockAcquire(&MyLogicalDecodingSlot->mutex);
+
+ /* don't overwrite if have a newer restart lsn*/
+ if (restart_lsn <= MyLogicalDecodingSlot->restart_decoding)
+ {
+ }
+ /*
+ * We might have already flushed far enough to directly accept this lsn, in
+ * this case there is no need to check for existing candidate LSNs
+ */
+ else if (current_lsn <= MyLogicalDecodingSlot->confirmed_flush)
+ {
+ MyLogicalDecodingSlot->candidate_restart_valid = current_lsn;
+ MyLogicalDecodingSlot->candidate_restart_decoding = restart_lsn;
+
+ /* our candidate can directly be used */
+ updated_lsn = true;
+ }
+ /*
+ * Only increase if the previous values have been applied, otherwise we
+ * might never end up updating if the receiver acks too slowly. A missed
+ * value here will just cause some extra effort after reconnecting.
+ */
+ if (MyLogicalDecodingSlot->candidate_restart_valid == InvalidXLogRecPtr)
+ {
+ MyLogicalDecodingSlot->candidate_restart_valid = current_lsn;
+ MyLogicalDecodingSlot->candidate_restart_decoding = restart_lsn;
+
+ elog(DEBUG1, "got new restart lsn %X/%X at %X/%X",
+ (uint32) (restart_lsn >> 32), (uint32) restart_lsn,
+ (uint32) (current_lsn >> 32), (uint32) current_lsn);
+ }
+ else
+ {
+ elog(DEBUG1, "failed to increase restart lsn: proposed %X/%X, after %X/%X, current candidate %X/%X, current after %X/%X, flushed up to %X/%X",
+ (uint32) (restart_lsn >> 32), (uint32) restart_lsn,
+ (uint32) (current_lsn >> 32), (uint32) current_lsn,
+ (uint32) (MyLogicalDecodingSlot->candidate_restart_decoding >> 32),
+ (uint32) MyLogicalDecodingSlot->candidate_restart_decoding,
+ (uint32) (MyLogicalDecodingSlot->candidate_restart_valid >> 32),
+ (uint32) MyLogicalDecodingSlot->candidate_restart_valid,
+ (uint32) (MyLogicalDecodingSlot->confirmed_flush >> 32),
+ (uint32) MyLogicalDecodingSlot->confirmed_flush
+ );
+ }
+ SpinLockRelease(&MyLogicalDecodingSlot->mutex);
+
+ /* candidates are already valid with the current flush position, apply */
+ if (updated_lsn)
+ LogicalConfirmReceivedLocation(MyLogicalDecodingSlot->confirmed_flush);
+}
+
+void
+LogicalConfirmReceivedLocation(XLogRecPtr lsn)
+{
+ Assert(lsn != InvalidXLogRecPtr);
+
+ /* Do an unlocked check for candidate_lsn first. */
+ if (MyLogicalDecodingSlot->candidate_xmin_lsn != InvalidXLogRecPtr ||
+ MyLogicalDecodingSlot->candidate_restart_valid != InvalidXLogRecPtr)
+ {
+ bool updated_xmin = false;
+ bool updated_restart = false;
+
+ /* use volatile pointer to prevent code rearrangement */
+ volatile LogicalDecodingSlot *slot = MyLogicalDecodingSlot;
+
+ SpinLockAcquire(&slot->mutex);
+
+ slot->confirmed_flush = lsn;
+
+ /* if were past the location required for bumping xmin, do so */
+ if (slot->candidate_xmin_lsn != InvalidXLogRecPtr &&
+ slot->candidate_xmin_lsn <= lsn)
+ {
+ /*
+ * We have to write the changed xmin to disk *before* we change
+ * the in-memory value, otherwise after a crash we wouldn't know
+ * that some catalog tuples might have been removed already.
+ *
+ * Ensure that by first writing to ->xmin and only update
+ * ->effective_xmin once the new state is fsynced to disk. After a
+ * crash ->effective_xmin is set to ->xmin.
+ */
+ if (TransactionIdIsValid(slot->candidate_xmin) &&
+ slot->xmin != slot->candidate_xmin)
+ {
+ slot->xmin = slot->candidate_xmin;
+ slot->candidate_xmin = InvalidTransactionId;
+ slot->candidate_xmin_lsn = InvalidXLogRecPtr;
+ updated_xmin = true;
+ }
+ }
+
+ if (slot->candidate_restart_valid != InvalidXLogRecPtr &&
+ slot->candidate_restart_valid <= lsn)
+ {
+ Assert(slot->candidate_restart_decoding != InvalidXLogRecPtr);
+
+ slot->restart_decoding = slot->candidate_restart_decoding;
+ slot->candidate_restart_decoding = InvalidXLogRecPtr;
+ slot->candidate_restart_valid = InvalidXLogRecPtr;
+ updated_restart = true;
+ }
+
+ SpinLockRelease(&slot->mutex);
+
+ /* first write new xmin to disk, so we know whats up after a crash */
+ if (updated_xmin || updated_restart)
+ {
+ /* cast away volatile, thats ok. */
+ SaveLogicalSlot((LogicalDecodingSlot *) slot);
+ elog(DEBUG1, "updated xmin: %u restart: %u", updated_xmin, updated_restart);
+ }
+ /*
+ * now the new xmin is safely on disk, we can let the global value
+ * advance
+ */
+ if (updated_xmin)
+ {
+ SpinLockAcquire(&slot->mutex);
+ slot->effective_xmin = slot->xmin;
+ SpinLockRelease(&slot->mutex);
+
+ ComputeLogicalXmin();
+ }
+ }
+ else
+ {
+ volatile LogicalDecodingSlot *slot = MyLogicalDecodingSlot;
+
+ SpinLockAcquire(&slot->mutex);
+ slot->confirmed_flush = lsn;
+ SpinLockRelease(&slot->mutex);
+ }
+}
+
+/*
+ * Compute the xmin between all of the decoding slots and store it in
+ * WalSndCtlData.
+ */
+void
+ComputeLogicalXmin(void)
+{
+ int i;
+ TransactionId xmin = InvalidTransactionId;
+ LogicalDecodingSlot *slot;
+
+ Assert(LogicalDecodingCtl);
+
+ LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+ for (i = 0; i < max_logical_slots; i++)
+ {
+ slot = &LogicalDecodingCtl->logical_slots[i];
+
+ SpinLockAcquire(&slot->mutex);
+ if (slot->in_use &&
+ TransactionIdIsValid(slot->effective_xmin) && (
+ !TransactionIdIsValid(xmin) ||
+ TransactionIdPrecedes(slot->effective_xmin, xmin))
+ )
+ {
+ xmin = slot->effective_xmin;
+ }
+ SpinLockRelease(&slot->mutex);
+ }
+ LogicalDecodingCtl->xmin = xmin;
+ LWLockRelease(ProcArrayLock);
+
+ elog(DEBUG1, "computed new global xmin for decoding: %u", xmin);
+}
+
+/*
+ * Compute the oldest WAL LSN we need to be able to read to be able to continue
+ * decoding all slots.
+ *
+ * Returns InvalidXLogRecPtr if logical decoding is disabled or there are no
+ * active slots including the case where wal_level < logical.
+ */
+XLogRecPtr ComputeLogicalRestartLSN(void)
+{
+ XLogRecPtr result = InvalidXLogRecPtr;
+ int i;
+
+ if (max_logical_slots <= 0)
+ return InvalidXLogRecPtr;
+
+ LWLockAcquire(LogicalDecodingSlotCtlLock, LW_SHARED);
+
+ for (i = 0; i < max_logical_slots; i++)
+ {
+ volatile LogicalDecodingSlot *s = &LogicalDecodingCtl->logical_slots[i];
+ XLogRecPtr restart_decoding;
+
+ /* cannot change while LogicalDecodingSlotCtlLock is held */
+ if (!s->in_use)
+ continue;
+
+ /* read once, it's ok if it increases while we're checking */
+ SpinLockAcquire(&s->mutex);
+ restart_decoding = s->restart_decoding;
+ SpinLockRelease(&s->mutex);
+
+ if (result == InvalidXLogRecPtr ||
+ restart_decoding < result)
+ result = restart_decoding;
+ }
+
+ LWLockRelease(LogicalDecodingSlotCtlLock);
+
+ return result;
+}
+
+/*
+ * Make sure the current settings & environment are capable of doing logical
+ * changeset extraction.
+ */
+void
+CheckLogicalDecodingRequirements(void)
+{
+ if (wal_level < WAL_LEVEL_LOGICAL)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("logical decoding requires wal_level=logical")));
+
+ if (MyDatabaseId == InvalidOid)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("logical decoding requires to be connected to a database")));
+
+ if (max_logical_slots == 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ (errmsg("logical decoding requires needs max_logical_slots > 0"))));
+}
+
+/*
+ * Search for a free slot, mark it as used and acquire a valid xmin horizon
+ * value.
+ */
+void
+LogicalDecodingAcquireFreeSlot(const char *name, const char *plugin)
+{
+ LogicalDecodingSlot *slot;
+ bool name_in_use;
+ int i;
+
+ Assert(!MyLogicalDecodingSlot);
+
+ CheckLogicalDecodingRequirements();
+
+ LWLockAcquire(LogicalDecodingSlotCtlLock, LW_EXCLUSIVE);
+
+ /* First, make sure the requested name is not in use. */
+
+ name_in_use = false;
+ for (i = 0; i < max_logical_slots && !name_in_use; i++)
+ {
+ LogicalDecodingSlot *s = &LogicalDecodingCtl->logical_slots[i];
+
+ SpinLockAcquire(&s->mutex);
+ if (s->in_use && strcmp(name, NameStr(s->name)) == 0)
+ name_in_use = true;
+ SpinLockRelease(&s->mutex);
+ }
+
+ if (name_in_use)
+ ereport(ERROR,
+ (errcode(ERRCODE_DUPLICATE_OBJECT),
+ errmsg("There already is a logical slot named \"%s\"", name)));
+
+ /* Find the first available (not in_use (=> not active)) slot. */
+
+ slot = NULL;
+ for (i = 0; i < max_logical_slots; i++)
+ {
+ LogicalDecodingSlot *s = &LogicalDecodingCtl->logical_slots[i];
+
+ SpinLockAcquire(&s->mutex);
+ if (!s->in_use)
+ {
+ Assert(!s->active);
+ /* NOT releasing the lock yet */
+ slot = s;
+ break;
+ }
+ SpinLockRelease(&s->mutex);
+ }
+
+ LWLockRelease(LogicalDecodingSlotCtlLock);
+
+ if (!slot)
+ ereport(ERROR,
+ (errcode(ERRCODE_CONFIGURATION_LIMIT_EXCEEDED),
+ errmsg("couldn't find free logical slot. free one or increase max_logical_slots")));
+
+ MyLogicalDecodingSlot = slot;
+
+ /* Lets start with enough information if we can */
+ if (!RecoveryInProgress())
+ {
+ XLogRecPtr flushptr;
+
+ /* start at current insert position*/
+ slot->restart_decoding = GetXLogInsertRecPtr();
+
+ /* make sure we have enough information to start */
+ flushptr = LogStandbySnapshot();
+
+ /* and make sure it's fsynced to disk */
+ XLogFlush(flushptr);
+ }
+ else
+ slot->restart_decoding = GetRedoRecPtr();
+
+ slot->in_use = true;
+ slot->active = true;
+ slot->database = MyDatabaseId;
+ /* XXX: do we want to use truncate identifier instead? */
+ strncpy(NameStr(slot->plugin), plugin, NAMEDATALEN);
+ NameStr(slot->plugin)[NAMEDATALEN - 1] = '\0';
+ strncpy(NameStr(slot->name), name, NAMEDATALEN);
+ NameStr(slot->name)[NAMEDATALEN - 1] = '\0';
+
+ /* Arrange to clean up at exit/error */
+ on_shmem_exit(LogicalSlotKill, 0);
+
+ /* release slot so it can be examined by others */
+ SpinLockRelease(&slot->mutex);
+
+ /* XXX: verify that the specified plugin is valid */
+
+ /*
+ * Acquire the current global xmin value and directly set the logical xmin
+ * before releasing the lock if necessary. We do this so wal decoding is
+ * guaranteed to have all catalog rows produced by xacts with an xid >
+ * walsnd->xmin available.
+ *
+ * We can't use ComputeLogicalXmin here as that acquires ProcArrayLock
+ * separately which would open a short window for the global xmin to
+ * advance above walsnd->xmin.
+ */
+ LWLockAcquire(ProcArrayLock, LW_SHARED);
+ slot->effective_xmin = GetOldestXmin(true, true, true);
+ slot->xmin = slot->effective_xmin;
+
+ if (!TransactionIdIsValid(LogicalDecodingCtl->xmin) ||
+ NormalTransactionIdPrecedes(slot->effective_xmin, LogicalDecodingCtl->xmin))
+ LogicalDecodingCtl->xmin = slot->effective_xmin;
+ LWLockRelease(ProcArrayLock);
+
+ Assert(slot->effective_xmin <= GetOldestXmin(true, true, false));
+
+ LWLockAcquire(LogicalDecodingSlotCtlLock, LW_EXCLUSIVE);
+ CreateLogicalSlot(slot);
+ LWLockRelease(LogicalDecodingSlotCtlLock);
+}
+
+/*
+ * Find an previously initiated slot and mark it as used again.
+ */
+void
+LogicalDecodingReAcquireSlot(const char *name)
+{
+ LogicalDecodingSlot *slot;
+ int i;
+
+ CheckLogicalDecodingRequirements();
+
+ Assert(!MyLogicalDecodingSlot);
+
+ for (i = 0; i < max_logical_slots; i++)
+ {
+ slot = &LogicalDecodingCtl->logical_slots[i];
+
+ SpinLockAcquire(&slot->mutex);
+ if (slot->in_use && strcmp(name, NameStr(slot->name)) == 0)
+ {
+ MyLogicalDecodingSlot = slot;
+ /* NOT releasing the lock yet */
+ break;
+ }
+ SpinLockRelease(&slot->mutex);
+ }
+
+ if (!MyLogicalDecodingSlot)
+ ereport(ERROR,
+ (errcode(ERRCODE_UNDEFINED_OBJECT),
+ errmsg("couldn't find logical slot \"%s\"", name)));
+
+ slot = MyLogicalDecodingSlot;
+
+ if (slot->active)
+ {
+ SpinLockRelease(&slot->mutex);
+ MyLogicalDecodingSlot = NULL;
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_IN_USE),
+ errmsg("slot already active")));
+ }
+
+ slot->active = true;
+ /* now that we've marked it as active, we release our lock */
+ SpinLockRelease(&slot->mutex);
+
+ /* Don't let the user switch the database... */
+ if (slot->database != MyDatabaseId)
+ {
+ SpinLockAcquire(&slot->mutex);
+ slot->active = false;
+ MyLogicalDecodingSlot = NULL;
+ SpinLockRelease(&slot->mutex);
+
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ (errmsg("START_LOGICAL_REPLICATION needs to be run in the same database as INIT_LOGICAL_REPLICATION"))));
+ }
+
+ /* Arrange to clean up at exit */
+ on_shmem_exit(LogicalSlotKill, 0);
+
+ SaveLogicalSlot(slot);
+}
+
+/*
+ * Temporarily remove a logical decoding slot, this or another backend can
+ * reacquire it later.
+ */
+void
+LogicalDecodingReleaseSlot(void)
+{
+ LogicalDecodingSlot *slot;
+
+ CheckLogicalDecodingRequirements();
+
+ slot = MyLogicalDecodingSlot;
+
+ Assert(slot != NULL && slot->active);
+
+ SpinLockAcquire(&slot->mutex);
+ slot->active = false;
+ SpinLockRelease(&slot->mutex);
+
+ MyLogicalDecodingSlot = NULL;
+
+ SaveLogicalSlot(slot);
+
+ cancel_shmem_exit(LogicalSlotKill, 0);
+}
+
+/*
+ * Permanently remove a logical decoding slot.
+ */
+void
+LogicalDecodingFreeSlot(const char *name)
+{
+ LogicalDecodingSlot *slot = NULL;
+ int i;
+
+ CheckLogicalDecodingRequirements();
+
+ for (i = 0; i < max_logical_slots; i++)
+ {
+ slot = &LogicalDecodingCtl->logical_slots[i];
+
+ SpinLockAcquire(&slot->mutex);
+ if (slot->in_use && strcmp(name, NameStr(slot->name)) == 0)
+ {
+ /* NOT releasing the lock yet */
+ break;
+ }
+ SpinLockRelease(&slot->mutex);
+ slot = NULL;
+ }
+
+ if (!slot)
+ ereport(ERROR,
+ (errcode(ERRCODE_UNDEFINED_OBJECT),
+ errmsg("couldn't find logical slot \"%s\"", name)));
+
+ if (slot->active)
+ {
+ SpinLockRelease(&slot->mutex);
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_IN_USE),
+ errmsg("cannot free active logical slot \"%s\"", name)));
+ }
+
+ /*
+ * Mark it as as active, so nobody can claim this slot while we are
+ * working on it. We don't want to hold the spinlock while doing stuff
+ * like fsyncing the state file to disk.
+ */
+ slot->active = true;
+
+ SpinLockRelease(&slot->mutex);
+
+ /*
+ * Start critical section, we can't to be interrupted while on-disk/memory
+ * state aren't coherent.
+ */
+ START_CRIT_SECTION();
+
+ DeleteLogicalSlot(slot);
+
+ /* ok, everything gone, after a crash we now wouldn't restore this slot */
+ SpinLockAcquire(&slot->mutex);
+ slot->active = false;
+ slot->in_use = false;
+ SpinLockRelease(&slot->mutex);
+
+ END_CRIT_SECTION();
+
+ /* slot is dead and doesn't nail the xmin anymore */
+ ComputeLogicalXmin();
+}
+
+/*
+ * LogicalDecodingCountDBSlots -- count number of slots referring to the given DB
+ *
+ * Returns true if there are any slots referencing the
+ * database. *nslots will be set to the absolute number of slots in
+ * the database, *nactive to ones currently active.
+ */
+bool
+LogicalDecodingCountDBSlots(Oid dboid, int *nslots, int *nactive)
+{
+ int i;
+
+ *nslots = *nactive = 0;
+
+ if (max_logical_slots <= 0)
+ return false;
+
+ LWLockAcquire(LogicalDecodingSlotCtlLock, LW_SHARED);
+ for (i = 0; i < max_logical_slots; i++)
+ {
+ volatile LogicalDecodingSlot *s = &LogicalDecodingCtl->logical_slots[i];
+
+ /* cannot change while LogicalDecodingSlotCtlLock is held */
+ if (!s->in_use)
+ continue;
+
+ /* not our database, don't count */
+ if (s->database != dboid)
+ continue;
+
+ /* count slots with spinlock held */
+ SpinLockAcquire(&s->mutex);
+ (*nslots)++;
+ if (s->active)
+ (*nactive)++;
+ SpinLockRelease(&s->mutex);
+ }
+ LWLockRelease(LogicalDecodingSlotCtlLock);
+
+ if (*nslots > 0)
+ return true;
+ return false;
+}
+
+/*
+ * Load logical slots from disk into memory at server startup. This needs to be
+ * run before we start crash recovery.
+ */
+void
+StartupLogicalDecoding(XLogRecPtr checkPointRedo)
+{
+ DIR *logical_dir;
+ struct dirent *logical_de;
+
+ ereport(DEBUG1,
+ (errmsg("starting up logical decoding from %X/%X",
+ (uint32) (checkPointRedo >> 32), (uint32) checkPointRedo)));
+
+ /* restore all slots */
+ logical_dir = AllocateDir("pg_llog");
+ while ((logical_de = ReadDir(logical_dir, "pg_llog")) != NULL)
+ {
+ if (strcmp(logical_de->d_name, ".") == 0 ||
+ strcmp(logical_de->d_name, "..") == 0)
+ continue;
+
+ /* one of our own directories */
+ if (strcmp(logical_de->d_name, "snapshots") == 0 ||
+ strcmp(logical_de->d_name, "mappings") == 0)
+ continue;
+
+ /* we crashed while a slot was being setup or deleted, clean up */
+ if (strcmp(logical_de->d_name, "new") == 0 ||
+ strcmp(logical_de->d_name, "old") == 0)
+ {
+ char path[MAXPGPATH];
+
+ sprintf(path, "pg_llog/%s", logical_de->d_name);
+
+ if (!rmtree(path, true))
+ {
+ FreeDir(logical_dir);
+ ereport(PANIC,
+ (errcode_for_file_access(),
+ errmsg("could not remove directory \"%s\": %m",
+ path)));
+ }
+ continue;
+ }
+ /* XXX: check for a filename pattern and skip? */
+ RestoreLogicalSlot(logical_de->d_name);
+ }
+ FreeDir(logical_dir);
+
+ /* currently no slots exist , we're done. */
+ if (max_logical_slots <= 0)
+ return;
+
+ /* Now that we have recovered all the data, compute logical xmin */
+ ComputeLogicalXmin();
+
+ ReorderBufferStartup();
+}
+
+/* ----
+ * Manipulation of ondisk state of logical slots
+ * ----
+ */
+static void
+CreateLogicalSlot(LogicalDecodingSlot *slot)
+{
+ char tmppath[MAXPGPATH];
+ char path[MAXPGPATH];
+
+ START_CRIT_SECTION();
+
+ sprintf(tmppath, "pg_llog/new");
+ sprintf(path, "pg_llog/%s", NameStr(slot->name));
+
+ if (mkdir(tmppath, S_IRWXU) < 0)
+ ereport(PANIC,
+ (errcode_for_file_access(),
+ errmsg("could not create directory \"%s\": %m",
+ tmppath)));
+
+ fsync_fname(tmppath, true);
+
+ SaveLogicalSlotInternal(slot, tmppath);
+
+ if (rename(tmppath, path) != 0)
+ {
+ ereport(PANIC,
+ (errcode_for_file_access(),
+ errmsg("could not rename logical checkpoint from \"%s\" to \"%s\": %m",
+ tmppath, path)));
+ }
+
+ fsync_fname(path, true);
+
+ END_CRIT_SECTION();
+}
+
+static void
+SaveLogicalSlot(LogicalDecodingSlot *slot)
+{
+ char path[MAXPGPATH];
+
+ sprintf(path, "pg_llog/%s", NameStr(slot->name));
+ SaveLogicalSlotInternal(slot, path);
+}
+
+/*
+ * Shared functionality between saving and creating a logical slot.
+ */
+static void
+SaveLogicalSlotInternal(LogicalDecodingSlot *slot, const char *dir)
+{
+ char tmppath[MAXPGPATH];
+ char path[MAXPGPATH];
+ int fd;
+ LogicalDecodingSlotOnDisk cp;
+
+ /* silence valgrind :( */
+ memset(&cp, 0, sizeof(LogicalDecodingSlotOnDisk));
+
+ sprintf(tmppath, "%s/state.tmp", dir);
+ sprintf(path, "%s/state", dir);
+
+ START_CRIT_SECTION();
+
+ fd = OpenTransientFile(tmppath,
+ O_CREAT | O_EXCL | O_WRONLY | PG_BINARY,
+ S_IRUSR | S_IWUSR);
+ if (fd < 0)
+ ereport(PANIC,
+ (errcode_for_file_access(),
+ errmsg("could not create logical checkpoint file \"%s\": %m",
+ tmppath)));
+
+ cp.magic = LOGICAL_MAGIC;
+ INIT_CRC32(cp.checksum);
+ cp.version = 1;
+ cp.length = LogicalDecodingSlotOnDiskDynamicSize;
+
+ SpinLockAcquire(&slot->mutex);
+
+ cp.slot.xmin = slot->xmin;
+ cp.slot.effective_xmin = slot->effective_xmin;
+
+ strcpy(NameStr(cp.slot.name), NameStr(slot->name));
+ strcpy(NameStr(cp.slot.plugin), NameStr(slot->plugin));
+
+ cp.slot.database = slot->database;
+ cp.slot.confirmed_flush = slot->confirmed_flush;
+ cp.slot.restart_decoding = slot->restart_decoding;
+ cp.slot.candidate_xmin = InvalidTransactionId;
+ cp.slot.candidate_xmin_lsn = InvalidXLogRecPtr;
+ cp.slot.candidate_restart_decoding = InvalidXLogRecPtr;
+ cp.slot.candidate_restart_valid = InvalidXLogRecPtr;
+ cp.slot.in_use = slot->in_use;
+ cp.slot.active = false;
+
+ SpinLockRelease(&slot->mutex);
+
+ COMP_CRC32(cp.checksum,
+ (char *)(&cp) + LogicalDecodingSlotOnDiskConstantSize,
+ LogicalDecodingSlotOnDiskDynamicSize);
+
+ if ((write(fd, &cp, sizeof(cp))) != sizeof(cp))
+ {
+ CloseTransientFile(fd);
+ ereport(PANIC,
+ (errcode_for_file_access(),
+ errmsg("could not write logical checkpoint file \"%s\": %m",
+ tmppath)));
+ }
+
+ /* fsync the file */
+ if (pg_fsync(fd) != 0)
+ {
+ CloseTransientFile(fd);
+ ereport(PANIC,
+ (errcode_for_file_access(),
+ errmsg("could not fsync logical checkpoint \"%s\": %m",
+ tmppath)));
+ }
+
+ CloseTransientFile(fd);
+
+ /* rename to permanent file, fsync file and directory */
+ if (rename(tmppath, path) != 0)
+ {
+ ereport(PANIC,
+ (errcode_for_file_access(),
+ errmsg("could not rename logical checkpoint from \"%s\" to \"%s\": %m",
+ tmppath, path)));
+ }
+
+ fsync_fname((char *) dir, true);
+ fsync_fname(path, false);
+
+ END_CRIT_SECTION();
+}
+
+
+static void
+DeleteLogicalSlot(LogicalDecodingSlot *slot)
+{
+ char path[MAXPGPATH];
+ char tmppath[] = "pg_llog/old";
+
+ START_CRIT_SECTION();
+
+ sprintf(path, "pg_llog/%s", NameStr(slot->name));
+
+ if (rename(path, tmppath) != 0)
+ {
+ ereport(PANIC,
+ (errcode_for_file_access(),
+ errmsg("could not rename logical checkpoint from \"%s\" to \"%s\": %m",
+ path, tmppath)));
+ }
+
+ /* make sure no partial state is visible after a crash */
+ fsync_fname(tmppath, true);
+ fsync_fname("pg_llog", true);
+
+ if (!rmtree(tmppath, true))
+ {
+ ereport(PANIC,
+ (errcode_for_file_access(),
+ errmsg("could not remove directory \"%s\": %m",
+ tmppath)));
+ }
+
+ END_CRIT_SECTION();
+}
+
+/*
+ * Load a single ondisk slot into memory.
+ */
+static void
+RestoreLogicalSlot(const char *name)
+{
+ LogicalDecodingSlotOnDisk cp;
+ int i;
+ char path[MAXPGPATH];
+ int fd;
+ bool restored = false;
+ int readBytes;
+ pg_crc32 checksum;
+
+ START_CRIT_SECTION();
+
+ /* delete temp file if it exists */
+ sprintf(path, "pg_llog/%s/state.tmp", name);
+ if (unlink(path) < 0 && errno != ENOENT)
+ ereport(PANIC,
+ (errcode_for_file_access(),
+ errmsg("failed while unlinking %s: %m", path)));
+
+ sprintf(path, "pg_llog/%s/state", name);
+
+ elog(DEBUG1, "restoring logical slot from %s", path);
+
+ fd = OpenTransientFile(path, O_RDONLY | PG_BINARY, 0);
+
+ /*
+ * We do not need to handle this as we are rename()ing the directory into
+ * place only after we fsync()ed the state file.
+ */
+ if (fd < 0)
+ ereport(PANIC,
+ (errcode_for_file_access(),
+ errmsg("could not open state file %s", path)));
+
+ /* read part of statefile that's guaranteed to be version independent */
+ readBytes = read(fd, &cp, LogicalDecodingSlotOnDiskConstantSize);
+ if (readBytes != LogicalDecodingSlotOnDiskConstantSize)
+ {
+ int saved_errno = errno;
+
+ CloseTransientFile(fd);
+ errno = saved_errno;
+ ereport(PANIC,
+ (errcode_for_file_access(),
+ errmsg("could not read logical checkpoint file \"%s\":read %d of %u",
+ path, readBytes,
+ (uint32) LogicalDecodingSlotOnDiskConstantSize)));
+ }
+
+ /* verify magic */
+ if (cp.magic != LOGICAL_MAGIC)
+ ereport(PANIC,
+ (errcode_for_file_access(),
+ errmsg("Logical checkpoint has wrong magic %u instead of %u",
+ cp.magic, LOGICAL_MAGIC)));
+ /* verify version */
+ if (cp.version != LOGICAL_VERSION)
+ ereport(PANIC,
+ (errcode_for_file_access(),
+ errmsg("Logical checkpoint has unsupported version %u",
+ cp.version)));
+
+ /* boundary check on length */
+ if (cp.length != LogicalDecodingSlotOnDiskDynamicSize)
+ ereport(PANIC,
+ (errcode_for_file_access(),
+ errmsg("Logical checkpoint has corrupted length %u",
+ cp.length)));
+
+ /* Now that we know the size, read the entire file */
+ readBytes = read(fd,
+ (char *)&cp + LogicalDecodingSlotOnDiskConstantSize,
+ cp.length);
+ if (readBytes != cp.length)
+ {
+ int saved_errno = errno;
+
+ CloseTransientFile(fd);
+ errno = saved_errno;
+ ereport(PANIC,
+ (errcode_for_file_access(),
+ errmsg("could not read logical checkpoint file \"%s\":read %d of %u",
+ path, readBytes, cp.length)));
+ }
+
+ CloseTransientFile(fd);
+
+ /* now verify the CRC32 */
+ INIT_CRC32(checksum);
+ COMP_CRC32(checksum,
+ (char *)&cp + LogicalDecodingSlotOnDiskConstantSize,
+ LogicalDecodingSlotOnDiskDynamicSize);
+
+ if (!EQ_CRC32(checksum, cp.checksum))
+ ereport(PANIC,
+ (errmsg("logical checksum file %s: checksum mismatch, is %u, should be %u",
+ path, checksum, cp.checksum)));
+
+ /* nothing can be active yet, don't lock anything */
+ for (i = 0; i < max_logical_slots; i++)
+ {
+ LogicalDecodingSlot *slot;
+
+ slot = &LogicalDecodingCtl->logical_slots[i];
+
+ if (slot->in_use)
+ continue;
+
+ slot->xmin = cp.slot.xmin;
+ /*
+ * after a crash, always use xmin, not effective_xmin, the
+ * slot obviously survived
+ */
+ slot->effective_xmin = cp.slot.xmin;
+ strcpy(NameStr(slot->name), NameStr(cp.slot.name));
+ strcpy(NameStr(slot->plugin), NameStr(cp.slot.plugin));
+ slot->database = cp.slot.database;
+ slot->restart_decoding = cp.slot.restart_decoding;
+ slot->confirmed_flush = cp.slot.confirmed_flush;
+ /* ignore previous values */
+ slot->candidate_xmin = InvalidTransactionId;
+ slot->candidate_xmin_lsn = InvalidXLogRecPtr;
+ slot->candidate_restart_decoding = InvalidXLogRecPtr;
+ slot->candidate_restart_valid = InvalidXLogRecPtr;
+ slot->in_use = true;
+ slot->active = false;
+ restored = true;
+ break;
+ }
+
+ if (!restored)
+ ereport(PANIC,
+ (errmsg("too many logical slots active before shutdown, increase max_logical_slots and try again")));
+
+ END_CRIT_SECTION();
+}
+
+
+static void
+LoadOutputPlugin(OutputPluginCallbacks *callbacks, char *plugin)
+{
+ /* lookup symbols in the shared libarary */
+
+ /* optional */
+ callbacks->init_cb = (LogicalDecodeInitCB)
+ load_external_function(plugin, "pg_decode_init", false, NULL);
+
+ /* required */
+ callbacks->begin_cb = (LogicalDecodeBeginCB)
+ load_external_function(plugin, "pg_decode_begin_txn", true, NULL);
+
+ /* required */
+ callbacks->change_cb = (LogicalDecodeChangeCB)
+ load_external_function(plugin, "pg_decode_change", true, NULL);
+
+ /* required */
+ callbacks->commit_cb = (LogicalDecodeCommitCB)
+ load_external_function(plugin, "pg_decode_commit_txn", true, NULL);
+
+ /* optional */
+ callbacks->cleanup_cb = (LogicalDecodeCleanupCB)
+ load_external_function(plugin, "pg_decode_cleanup", false, NULL);
+}
+
+/*
+ * Context management functions to coordinate between the different logical
+ * decoding pieces.
+ */
+
+typedef struct LogicalErrorCallbackState
+{
+ LogicalDecodingContext *ctx;
+ const char *callback;
+} LogicalErrorCallbackState;
+
+static void
+output_plugin_error_callback(void *arg)
+{
+ LogicalErrorCallbackState *state = (LogicalErrorCallbackState *) arg;
+ /* XXX: Add the current LSN? */
+ errcontext("slot \"%s\", output plugin \"%s\" during the %s callback",
+ NameStr(state->ctx->slot->name),
+ NameStr(state->ctx->slot->plugin),
+ state->callback);
+}
+
+static void
+init_slot_wrapper(LogicalDecodingContext *ctx, bool is_init)
+{
+ LogicalErrorCallbackState state;
+ ErrorContextCallback errcallback;
+
+ /* Push callback + info on the error context stack */
+ state.ctx = ctx;
+ state.callback = "pg_decode_init";
+ errcallback.callback = output_plugin_error_callback;
+ errcallback.arg = (void *) &state;
+ errcallback.previous = error_context_stack;
+ error_context_stack = &errcallback;
+
+ /* do the actual work: call callback */
+ ctx->callbacks.init_cb(ctx, is_init);
+
+ /* Pop the error context stack */
+ error_context_stack = errcallback.previous;
+}
+
+static void
+cleanup_slot_wrapper(LogicalDecodingContext *ctx)
+{
+ LogicalErrorCallbackState state;
+ ErrorContextCallback errcallback;
+
+ /* Push callback + info on the error context stack */
+ state.ctx = ctx;
+ state.callback = "pg_decode_cleanup";
+ errcallback.callback = output_plugin_error_callback;
+ errcallback.arg = (void *) &state;
+ errcallback.previous = error_context_stack;
+ error_context_stack = &errcallback;
+
+ /* do the actual work: call callback */
+ ctx->callbacks.cleanup_cb(ctx);
+
+ /* Pop the error context stack */
+ error_context_stack = errcallback.previous;
+}
+
+
+/*
+ * Callbacks for ReorderBuffer which add in some more information and then call
+ * output_plugin.h plugins.
+ */
+static void
+begin_txn_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn)
+{
+ LogicalDecodingContext *ctx = cache->private_data;
+ LogicalErrorCallbackState state;
+ ErrorContextCallback errcallback;
+
+ /* Push callback + info on the error context stack */
+ state.ctx = ctx;
+ state.callback = "pg_decode_begin_txn";
+ errcallback.callback = output_plugin_error_callback;
+ errcallback.arg = (void *) &state;
+ errcallback.previous = error_context_stack;
+ error_context_stack = &errcallback;
+
+ /* do the actual work: call callback */
+ ctx->callbacks.begin_cb(ctx, txn);
+
+ /* Pop the error context stack */
+ error_context_stack = errcallback.previous;
+}
+
+static void
+commit_txn_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn, XLogRecPtr commit_lsn)
+{
+ LogicalDecodingContext *ctx = cache->private_data;
+ LogicalErrorCallbackState state;
+ ErrorContextCallback errcallback;
+
+ /* Push callback + info on the error context stack */
+ state.ctx = ctx;
+ state.callback = "pg_decode_commit_txn";
+ errcallback.callback = output_plugin_error_callback;
+ errcallback.arg = (void *) &state;
+ errcallback.previous = error_context_stack;
+ error_context_stack = &errcallback;
+
+ /* do the actual work: call callback */
+ ctx->callbacks.commit_cb(ctx, txn, commit_lsn);
+
+ /* Pop the error context stack */
+ error_context_stack = errcallback.previous;
+}
+
+static void
+change_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
+ Relation relation, ReorderBufferChange *change)
+{
+ LogicalDecodingContext *ctx = cache->private_data;
+ LogicalErrorCallbackState state;
+ ErrorContextCallback errcallback;
+
+ /* Push callback + info on the error context stack */
+ state.ctx = ctx;
+ state.callback = "pg_decode_change";
+ errcallback.callback = output_plugin_error_callback;
+ errcallback.arg = (void *) &state;
+ errcallback.previous = error_context_stack;
+ error_context_stack = &errcallback;
+
+ ctx->callbacks.change_cb(ctx, txn, relation, change);
+
+ /* Pop the error context stack */
+ error_context_stack = errcallback.previous;
+}
+
+LogicalDecodingContext *
+CreateLogicalDecodingContext(LogicalDecodingSlot *slot,
+ bool is_init,
+ XLogRecPtr start_lsn,
+ List *output_plugin_options,
+ XLogPageReadCB read_page,
+ LogicalOutputPluginWriterPrepareWrite prepare_write,
+ LogicalOutputPluginWriterWrite do_write)
+{
+ MemoryContext context;
+ MemoryContext old_context;
+ TransactionId xmin_horizon;
+ LogicalDecodingContext *ctx;
+
+ context = AllocSetContextCreate(TopMemoryContext,
+ "ReorderBuffer",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+ old_context = MemoryContextSwitchTo(context);
+ ctx = palloc0(sizeof(LogicalDecodingContext));
+
+
+ /* load output plugins first, so we detect a wrong output plugin early */
+ LoadOutputPlugin(&ctx->callbacks, NameStr(slot->plugin));
+
+ if (is_init && start_lsn != InvalidXLogRecPtr)
+ elog(ERROR, "Cannot INIT_LOGICAL_REPLICATION at a specified LSN");
+
+ if (is_init)
+ xmin_horizon = slot->xmin;
+ else
+ xmin_horizon = InvalidTransactionId;
+
+ ctx->slot = slot;
+
+ ctx->reader = XLogReaderAllocate(read_page, ctx);
+ ctx->reader->private_data = ctx;
+
+ ctx->reorder = ReorderBufferAllocate();
+ ctx->snapshot_builder =
+ AllocateSnapshotBuilder(ctx->reorder, xmin_horizon, start_lsn);
+
+ ctx->reorder->private_data = ctx;
+
+ /* wrap output plugin callbacks, so we can add error context information */
+ ctx->reorder->begin = begin_txn_wrapper;
+ ctx->reorder->apply_change = change_wrapper;
+ ctx->reorder->commit = commit_txn_wrapper;
+
+ ctx->out = makeStringInfo();
+ ctx->prepare_write = prepare_write;
+ ctx->write = do_write;
+
+ ctx->output_plugin_options = output_plugin_options;
+
+ if (is_init)
+ ctx->stop_after_consistent = true;
+ else
+ ctx->stop_after_consistent = false;
+
+ /* call output plugin initialization callback */
+ if (ctx->callbacks.init_cb != NULL)
+ init_slot_wrapper(ctx, is_init);
+
+ MemoryContextSwitchTo(old_context);
+
+ return ctx;
+}
+
+void
+FreeLogicalDecodingContext(LogicalDecodingContext *ctx)
+{
+ if (ctx->callbacks.cleanup_cb != NULL)
+ cleanup_slot_wrapper(ctx);
+}
+
+
+/* has the initial snapshot found a consistent state? */
+bool
+LogicalDecodingContextReady(LogicalDecodingContext *ctx)
+{
+ return SnapBuildCurrentState(ctx->snapshot_builder) == SNAPBUILD_CONSISTENT;
+}
--- /dev/null
+/*-------------------------------------------------------------------------
+ *
+ * logicalfuncs.c
+ *
+ * Support functions for using xlog decoding
+ *
+ *
+ * Copyright (c) 2012-2013, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ * src/backend/replication/logicalfuncs.c
+ *
+ */
+
+#include "postgres.h"
+
+#include <unistd.h>
+
+#include "fmgr.h"
+#include "funcapi.h"
+#include "miscadmin.h"
+#include "utils/builtins.h"
+#include "storage/fd.h"
+
+#include "replication/decode.h"
+#include "replication/logical.h"
+#include "replication/logicalfuncs.h"
+
+Datum init_logical_replication(PG_FUNCTION_ARGS);
+Datum stop_logical_replication(PG_FUNCTION_ARGS);
+Datum pg_stat_get_logical_decoding_slots(PG_FUNCTION_ARGS);
+
+/* FIXME: duplicate code with pg_xlogdump, similar to walsender.c */
+static void
+XLogRead(char *buf, TimeLineID tli, XLogRecPtr startptr, Size count)
+{
+ char *p;
+ XLogRecPtr recptr;
+ Size nbytes;
+
+ static int sendFile = -1;
+ static XLogSegNo sendSegNo = 0;
+ static uint32 sendOff = 0;
+
+ p = buf;
+ recptr = startptr;
+ nbytes = count;
+
+ while (nbytes > 0)
+ {
+ uint32 startoff;
+ int segbytes;
+ int readbytes;
+
+ startoff = recptr % XLogSegSize;
+
+ if (sendFile < 0 || !XLByteInSeg(recptr, sendSegNo))
+ {
+ char path[MAXPGPATH];
+
+ /* Switch to another logfile segment */
+ if (sendFile >= 0)
+ close(sendFile);
+
+ XLByteToSeg(recptr, sendSegNo);
+
+ XLogFilePath(path, tli, sendSegNo);
+
+ sendFile = BasicOpenFile(path, O_RDONLY | PG_BINARY, 0);
+
+ if (sendFile < 0)
+ {
+ if (errno == ENOENT)
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("requested WAL segment %s has already been removed",
+ path)));
+ else
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not open file \"%s\": %m",
+ path)));
+ }
+ sendOff = 0;
+ }
+
+ /* Need to seek in the file? */
+ if (sendOff != startoff)
+ {
+ if (lseek(sendFile, (off_t) startoff, SEEK_SET) < 0)
+ {
+ char path[MAXPGPATH];
+
+ XLogFilePath(path, tli, sendSegNo);
+
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not seek in log segment %s to offset %u: %m",
+ path, startoff)));
+ }
+ sendOff = startoff;
+ }
+
+ /* How many bytes are within this segment? */
+ if (nbytes > (XLogSegSize - startoff))
+ segbytes = XLogSegSize - startoff;
+ else
+ segbytes = nbytes;
+
+ readbytes = read(sendFile, p, segbytes);
+ if (readbytes <= 0)
+ {
+ char path[MAXPGPATH];
+
+ XLogFilePath(path, tli, sendSegNo);
+
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not read from log segment %s, offset %u, length %lu: %m",
+ path, sendOff, (unsigned long) segbytes)));
+ }
+
+ /* Update state for read */
+ recptr += readbytes;
+
+ sendOff += readbytes;
+ nbytes -= readbytes;
+ p += readbytes;
+ }
+}
+
+int
+logical_read_local_xlog_page(XLogReaderState *state, XLogRecPtr targetPagePtr,
+ int reqLen, XLogRecPtr targetRecPtr, char *cur_page, TimeLineID *pageTLI)
+{
+ XLogRecPtr flushptr,
+ loc;
+ int count;
+
+ loc = targetPagePtr + reqLen;
+ while (1)
+ {
+ /*
+ * FIXME: we're going to have to do something more intelligent about
+ * timelines on standby's. Use readTimeLineHistory() and
+ * tliOfPointInHistory() to get the proper LSN?
+ */
+ if (!RecoveryInProgress())
+ {
+ *pageTLI = ThisTimeLineID;
+ flushptr = GetFlushRecPtr();
+ }
+ else
+ flushptr = GetXLogReplayRecPtr(pageTLI);
+
+ if (loc <= flushptr)
+ break;
+
+ /*
+ * XXX: It'd be way nicer to be able to use the walsender waiting logic
+ * here, but that's not available in all environments.
+ */
+ CHECK_FOR_INTERRUPTS();
+ pg_usleep(1000L);
+ }
+
+ /* more than one block available */
+ if (targetPagePtr + XLOG_BLCKSZ <= flushptr)
+ count = XLOG_BLCKSZ;
+ /* not enough data there */
+ else if (targetPagePtr + reqLen > flushptr)
+ return -1;
+ /* part of the page available */
+ else
+ count = flushptr - targetPagePtr;
+
+ /* XXX: more sensible/efficient implementation */
+ XLogRead(cur_page, *pageTLI, targetPagePtr, XLOG_BLCKSZ);
+
+ return count;
+}
+
+static void
+DummyWrite(LogicalDecodingContext *ctx, XLogRecPtr lsn, TransactionId xid)
+{
+ elog(ERROR, "init_logical_replication shouldn't be writing anything");
+}
+
+Datum
+init_logical_replication(PG_FUNCTION_ARGS)
+{
+ Name name = PG_GETARG_NAME(0);
+ Name plugin = PG_GETARG_NAME(1);
+
+ char xpos[MAXFNAMELEN];
+
+ TupleDesc tupdesc;
+ HeapTuple tuple;
+ Datum result;
+ Datum values[2];
+ bool nulls[2];
+
+ if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+ elog(ERROR, "return type must be a row type");
+
+ /*
+ * Acquire a logical decoding slot, this will check for conflicting
+ * names.
+ */
+ CheckLogicalDecodingRequirements();
+ LogicalDecodingAcquireFreeSlot(NameStr(*name), NameStr(*plugin));
+
+ elog(LOG, "starting to decode from %X/%X",
+ (uint32)(MyLogicalDecodingSlot->restart_decoding >> 32),
+ (uint32)MyLogicalDecodingSlot->restart_decoding);
+
+ /* make sure we don't end up with an unreleased slot */
+ PG_TRY();
+ {
+ LogicalDecodingContext *ctx = NULL;
+ XLogRecPtr startptr;
+
+ /*
+ * Create logical decoding context, to build the initial snapshot.
+ */
+ ctx = CreateLogicalDecodingContext(MyLogicalDecodingSlot, true,
+ InvalidXLogRecPtr, NIL,
+ logical_read_local_xlog_page,
+ DummyWrite, DummyWrite);
+
+ /* setup from where to read xlog */
+ startptr = ctx->slot->restart_decoding;
+
+ /* Wait for a consistent starting point */
+ for (;;)
+ {
+ XLogRecord *record;
+ XLogRecordBuffer buf;
+ char *err = NULL;
+
+ /* the read_page callback waits for new WAL */
+ record = XLogReadRecord(ctx->reader, startptr, &err);
+ if (err)
+ elog(ERROR, "%s", err);
+
+ Assert(record);
+
+ startptr = InvalidXLogRecPtr;
+
+ buf.origptr = ctx->reader->ReadRecPtr;
+ buf.record = *record;
+ buf.record_data = XLogRecGetData(record);
+ DecodeRecordIntoReorderBuffer(ctx, &buf);
+
+ /* only continue till we found a consistent spot */
+ if (LogicalDecodingContextReady(ctx))
+ break;
+ }
+
+ /* Extract the values we want */
+ MyLogicalDecodingSlot->confirmed_flush = ctx->reader->EndRecPtr;
+ snprintf(xpos, sizeof(xpos), "%X/%X",
+ (uint32) (MyLogicalDecodingSlot->confirmed_flush >> 32),
+ (uint32) MyLogicalDecodingSlot->confirmed_flush);
+
+ /* don't need the decoding context anymore */
+ FreeLogicalDecodingContext(ctx);
+ }
+ PG_CATCH();
+ {
+ LogicalDecodingReleaseSlot();
+ PG_RE_THROW();
+ }
+ PG_END_TRY();
+
+ values[0] = CStringGetTextDatum(NameStr(MyLogicalDecodingSlot->name));
+ values[1] = CStringGetTextDatum(xpos);
+
+ memset(nulls, 0, sizeof(nulls));
+
+ tuple = heap_form_tuple(tupdesc, values, nulls);
+ result = HeapTupleGetDatum(tuple);
+
+ LogicalDecodingReleaseSlot();
+
+ PG_RETURN_DATUM(result);
+}
+
+Datum
+stop_logical_replication(PG_FUNCTION_ARGS)
+{
+ Name name = PG_GETARG_NAME(0);
+
+ CheckLogicalDecodingRequirements();
+ LogicalDecodingFreeSlot(NameStr(*name));
+
+ PG_RETURN_INT32(0);
+}
+
+/*
+ * Return one row for each logical decoding slot currently in use.
+ */
+Datum
+pg_stat_get_logical_decoding_slots(PG_FUNCTION_ARGS)
+{
+#define PG_STAT_GET_LOGICAL_DECODING_SLOTS_COLS 6
+ ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+ TupleDesc tupdesc;
+ Tuplestorestate *tupstore;
+ MemoryContext per_query_ctx;
+ MemoryContext oldcontext;
+ int i;
+
+ /* check to see if caller supports us returning a tuplestore */
+ if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ errmsg("set-valued function called in context that cannot accept a set")));
+ if (!(rsinfo->allowedModes & SFRM_Materialize))
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ errmsg("materialize mode required, but it is not " \
+ "allowed in this context")));
+
+ /* Build a tuple descriptor for our result type */
+ if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+ elog(ERROR, "return type must be a row type");
+
+ per_query_ctx = rsinfo->econtext->ecxt_per_query_memory;
+ oldcontext = MemoryContextSwitchTo(per_query_ctx);
+
+ tupstore = tuplestore_begin_heap(true, false, work_mem);
+ rsinfo->returnMode = SFRM_Materialize;
+ rsinfo->setResult = tupstore;
+ rsinfo->setDesc = tupdesc;
+
+ MemoryContextSwitchTo(oldcontext);
+
+ for (i = 0; i < max_logical_slots; i++)
+ {
+ LogicalDecodingSlot *slot = &LogicalDecodingCtl->logical_slots[i];
+ Datum values[PG_STAT_GET_LOGICAL_DECODING_SLOTS_COLS];
+ bool nulls[PG_STAT_GET_LOGICAL_DECODING_SLOTS_COLS];
+ char location[MAXFNAMELEN];
+ const char *slot_name;
+ const char *plugin;
+ TransactionId xmin;
+ XLogRecPtr last_req;
+ bool active;
+ Oid database;
+
+ SpinLockAcquire(&slot->mutex);
+ if (!slot->in_use)
+ {
+ SpinLockRelease(&slot->mutex);
+ continue;
+ }
+ else
+ {
+ xmin = slot->xmin;
+ active = slot->active;
+ database = slot->database;
+ last_req = slot->restart_decoding;
+ slot_name = pstrdup(NameStr(slot->name));
+ plugin = pstrdup(NameStr(slot->plugin));
+ }
+ SpinLockRelease(&slot->mutex);
+
+ memset(nulls, 0, sizeof(nulls));
+
+ snprintf(location, sizeof(location), "%X/%X",
+ (uint32) (last_req >> 32), (uint32) last_req);
+
+ values[0] = CStringGetTextDatum(slot_name);
+ values[1] = CStringGetTextDatum(plugin);
+ values[2] = database;
+ values[3] = BoolGetDatum(active);
+ values[4] = TransactionIdGetDatum(xmin);
+ values[5] = CStringGetTextDatum(location);
+
+ tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+ }
+
+ tuplestore_donestoring(tupstore);
+
+ return (Datum) 0;
+}
--- /dev/null
+/*-------------------------------------------------------------------------
+ *
+ * reorderbuffer.c
+ *
+ * PostgreSQL logical replay buffer management
+ *
+ *
+ * Copyright (c) 2012-2013, PostgreSQL Global Development Group
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/replication/reorderbuffer.c
+ *
+ * NOTES
+ * This module gets handed individual pieces of transactions in the order
+ * they are written to the WAL and is responsible to reassemble them into
+ * toplevel transaction sized pieces. When a transaction is completely
+ * reassembled - signalled by reading the transaction commit record - it
+ * will then call the output plugin (c.f. ReorderBufferCommit()) with the
+ * individual changes. The output plugins rely on snapshots built by
+ * snapbuild.c which hands them to us.
+ *
+ * Transactions and subtransactions/savepoints in postgres are not
+ * immediately linked to each other from outside the performing
+ * backend. Only at commit/abort (or special xact_assignment records) they
+ * are linked together. Which means that we will have to splice together a
+ * toplevel transaction from its subtransactions. To do that efficiently we
+ * build a binary heap indexed by the smallest current lsn of the individual
+ * subtransactions' changestreams. As the individual streams are inherently
+ * ordered by LSN - since that is where we build them from - the transaction
+ * can easily be reassembled by always using the subtransaction with the
+ * smallest current LSN from the heap.
+ *
+ * In order to cope with large transactions - which can be several times as
+ * big as the available memory - this module supports spooling the contents
+ * of a large transactions to disk. When the transaction is replayed the
+ * contents of individual (sub-)transactions will be read from disk in
+ * chunks.
+ *
+ * This module also has to deal with reassembling toast records from the
+ * individual chunks stored in WAL. When a new (or initial) version of a
+ * tuple is stored in WAL it will always be preceded by the toast chunks
+ * emitted for the columns stored out of line. Within a single toplevel
+ * transaction there will be no other data carrying records between a row's
+ * toast chunks and the row data itself. See ReorderBufferToast* for
+ * details.
+ * -------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include <unistd.h>
+
+#include "miscadmin.h"
+
+#include "access/transam.h"
+#include "access/xact.h"
+#include "access/rewriteheap.h"
+
+#include "catalog/catalog.h"
+
+#include "common/relpath.h"
+
+#include "lib/binaryheap.h"
+
+#include "replication/reorderbuffer.h"
+#include "replication/snapbuild.h" /* just for SnapBuildSnapDecRefcount */
+#include "replication/logical.h"
+
+#include "storage/bufmgr.h"
+#include "storage/fd.h"
+#include "storage/sinval.h"
+
+#include "utils/builtins.h"
+#include "utils/combocid.h"
+#include "utils/memdebug.h"
+#include "utils/memutils.h"
+#include "utils/relcache.h"
+#include "utils/relfilenodemap.h"
+#include "utils/resowner.h"
+#include "utils/tqual.h"
+
+/*
+ * For efficiency and simplicity reasons we want to keep Snapshots, CommandIds
+ * and ComboCids in the same list with the user visible INSERT/UPDATE/DELETE
+ * changes. We don't want to leak those internal values to external users
+ * though (they would just use switch()...default:) because that would make it
+ * harder to add to new user visible values.
+ *
+ * This needs to be synchronized with ReorderBufferChangeType! Adjust the
+ * StaticAssertExpr's in ReorderBufferAllocate if you add anything!
+ */
+typedef enum
+{
+ REORDER_BUFFER_CHANGE_INTERNAL_INSERT,
+ REORDER_BUFFER_CHANGE_INTERNAL_UPDATE,
+ REORDER_BUFFER_CHANGE_INTERNAL_DELETE,
+ REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT,
+ REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID,
+ REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID
+} ReorderBufferChangeTypeInternal;
+
+/* entry for a hash table we use to map from xid to our transaction state */
+typedef struct ReorderBufferTXNByIdEnt
+{
+ TransactionId xid;
+ ReorderBufferTXN *txn;
+} ReorderBufferTXNByIdEnt;
+
+/* data structures for (relfilenode, ctid) => (cmin, cmax) mapping */
+typedef struct ReorderBufferTupleCidKey
+{
+ RelFileNode relnode;
+ ItemPointerData tid;
+} ReorderBufferTupleCidKey;
+
+typedef struct ReorderBufferTupleCidEnt
+{
+ ReorderBufferTupleCidKey key;
+ CommandId cmin;
+ CommandId cmax;
+ CommandId combocid; /* just for debugging */
+} ReorderBufferTupleCidEnt;
+
+/* k-way in-order change iteration support structures */
+typedef struct ReorderBufferIterTXNEntry
+{
+ XLogRecPtr lsn;
+ ReorderBufferChange *change;
+ ReorderBufferTXN *txn;
+ int fd;
+ XLogSegNo segno;
+} ReorderBufferIterTXNEntry;
+
+typedef struct ReorderBufferIterTXNState
+{
+ binaryheap *heap;
+ Size nr_txns;
+ dlist_head old_change;
+ ReorderBufferIterTXNEntry entries[FLEXIBLE_ARRAY_MEMBER];
+} ReorderBufferIterTXNState;
+
+/* toast datastructures */
+typedef struct ReorderBufferToastEnt
+{
+ Oid chunk_id; /* toast_table.chunk_id */
+ int32 last_chunk_seq; /* toast_table.chunk_seq of the last chunk we
+ * have seen */
+ Size num_chunks; /* number of chunks we've already seen */
+ Size size; /* combined size of chunks seen */
+ dlist_head chunks; /* linked list of chunks */
+ struct varlena *reconstructed; /* reconstructed varlena now pointed
+ * to in main tup */
+} ReorderBufferToastEnt;
+
+
+/* number of changes kept in memory, per transaction */
+const Size max_memtries = 4096;
+
+/* Size of the slab caches used for frequently allocated objects */
+const Size max_cached_changes = 4096 * 2;
+const Size max_cached_tuplebufs = 4096 * 2; /* ~8MB */
+const Size max_cached_transactions = 512;
+
+
+/* ---------------------------------------
+ * primary reorderbuffer support routines
+ * ---------------------------------------
+ */
+static ReorderBufferTXN *ReorderBufferGetTXN(ReorderBuffer *rb);
+static void ReorderBufferReturnTXN(ReorderBuffer *rb, ReorderBufferTXN *txn);
+static ReorderBufferTXN *ReorderBufferTXNByXid(ReorderBuffer *rb,
+ TransactionId xid, bool create, bool *is_new,
+ XLogRecPtr lsn, bool create_as_top);
+
+static void AssertTXNLsnOrder(ReorderBuffer *rb);
+
+/* ---------------------------------------
+ * support functions for lsn-order iterating over the ->changes of a
+ * transaction and its subtransactions
+ *
+ * used for iteration over the k-way heap merge of a transaction and its
+ * subtransactions
+ * ---------------------------------------
+ */
+static ReorderBufferIterTXNState *ReorderBufferIterTXNInit(ReorderBuffer *rb, ReorderBufferTXN *txn);
+static ReorderBufferChange *
+ ReorderBufferIterTXNNext(ReorderBuffer *rb, ReorderBufferIterTXNState *state);
+static void ReorderBufferIterTXNFinish(ReorderBuffer *rb,
+ ReorderBufferIterTXNState *state);
+static void ReorderBufferExecuteInvalidations(ReorderBuffer *rb, ReorderBufferTXN *txn);
+
+/*
+ * ---------------------------------------
+ * Disk serialization support functions
+ * ---------------------------------------
+ */
+static void ReorderBufferCheckSerializeTXN(ReorderBuffer *rb, ReorderBufferTXN *txn);
+static void ReorderBufferSerializeTXN(ReorderBuffer *rb, ReorderBufferTXN *txn);
+static void ReorderBufferSerializeChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
+ int fd, ReorderBufferChange *change);
+static Size ReorderBufferRestoreChanges(ReorderBuffer *rb, ReorderBufferTXN *txn,
+ int *fd, XLogSegNo *segno);
+static void ReorderBufferRestoreChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
+ char *change);
+static void ReorderBufferRestoreCleanup(ReorderBuffer *rb, ReorderBufferTXN *txn);
+
+static void ReorderBufferFreeSnap(ReorderBuffer *rb, Snapshot snap);
+static Snapshot ReorderBufferCopySnap(ReorderBuffer *rb, Snapshot orig_snap,
+ ReorderBufferTXN *txn, CommandId cid);
+
+/* ---------------------------------------
+ * toast reassembly support
+ * ---------------------------------------
+ */
+/* Size of an EXTERNAL datum that contains a standard TOAST pointer */
+#define TOAST_POINTER_SIZE (VARHDRSZ_EXTERNAL + sizeof(struct varatt_external))
+
+/* Size of an indirect datum that contains a standard TOAST pointer */
+#define INDIRECT_POINTER_SIZE (VARHDRSZ_EXTERNAL + sizeof(struct varatt_indirect))
+
+static void ReorderBufferToastInitHash(ReorderBuffer *rb, ReorderBufferTXN *txn);
+static void ReorderBufferToastReset(ReorderBuffer *rb, ReorderBufferTXN *txn);
+static void ReorderBufferToastReplace(ReorderBuffer *rb, ReorderBufferTXN *txn,
+ Relation relation, ReorderBufferChange *change);
+static void ReorderBufferToastAppendChunk(ReorderBuffer *rb, ReorderBufferTXN *txn,
+ Relation relation, ReorderBufferChange *change);
+
+
+/*
+ * Allocate a new ReorderBuffer
+ */
+ReorderBuffer *
+ReorderBufferAllocate(void)
+{
+ ReorderBuffer *buffer;
+ HASHCTL hash_ctl;
+ MemoryContext new_ctx;
+
+ StaticAssertExpr((int) REORDER_BUFFER_CHANGE_INTERNAL_INSERT == (int) REORDER_BUFFER_CHANGE_INSERT, "out of sync enums");
+ StaticAssertExpr((int) REORDER_BUFFER_CHANGE_INTERNAL_UPDATE == (int) REORDER_BUFFER_CHANGE_UPDATE, "out of sync enums");
+ StaticAssertExpr((int) REORDER_BUFFER_CHANGE_INTERNAL_DELETE == (int) REORDER_BUFFER_CHANGE_DELETE, "out of sync enums");
+
+ new_ctx = AllocSetContextCreate(TopMemoryContext,
+ "ReorderBuffer",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ buffer =
+ (ReorderBuffer *) MemoryContextAlloc(new_ctx, sizeof(ReorderBuffer));
+
+ memset(&hash_ctl, 0, sizeof(hash_ctl));
+
+ buffer->context = new_ctx;
+
+ hash_ctl.keysize = sizeof(TransactionId);
+ hash_ctl.entrysize = sizeof(ReorderBufferTXNByIdEnt);
+ hash_ctl.hash = tag_hash;
+ hash_ctl.hcxt = buffer->context;
+
+ buffer->by_txn = hash_create("ReorderBufferByXid", 1000, &hash_ctl,
+ HASH_ELEM | HASH_FUNCTION | HASH_CONTEXT);
+
+ buffer->by_txn_last_xid = InvalidTransactionId;
+ buffer->by_txn_last_txn = NULL;
+
+ buffer->nr_cached_transactions = 0;
+ buffer->nr_cached_changes = 0;
+ buffer->nr_cached_tuplebufs = 0;
+
+ buffer->outbuf = NULL;
+ buffer->outbufsize = 0;
+
+ buffer->current_restart_decoding_lsn = InvalidXLogRecPtr;
+
+ dlist_init(&buffer->toplevel_by_lsn);
+ dlist_init(&buffer->cached_transactions);
+ dlist_init(&buffer->cached_changes);
+ slist_init(&buffer->cached_tuplebufs);
+
+ return buffer;
+}
+
+/*
+ * Free a ReorderBuffer
+ */
+void
+ReorderBufferFree(ReorderBuffer *rb)
+{
+ MemoryContext context = rb->context;
+
+ /*
+ * We free separately allocated data by entirely scrapping oure personal
+ * memory context.
+ */
+ MemoryContextDelete(context);
+}
+
+/*
+ * Get a unused, possibly preallocated, ReorderBufferTXN.
+ */
+static ReorderBufferTXN *
+ReorderBufferGetTXN(ReorderBuffer *rb)
+{
+ ReorderBufferTXN *txn;
+
+ if (rb->nr_cached_transactions > 0)
+ {
+ rb->nr_cached_transactions--;
+ txn = (ReorderBufferTXN *)
+ dlist_container(ReorderBufferTXN, node,
+ dlist_pop_head_node(&rb->cached_transactions));
+ }
+ else
+ {
+ txn = (ReorderBufferTXN *)
+ MemoryContextAlloc(rb->context, sizeof(ReorderBufferTXN));
+ }
+
+ memset(txn, 0, sizeof(ReorderBufferTXN));
+
+ dlist_init(&txn->changes);
+ dlist_init(&txn->tuplecids);
+ dlist_init(&txn->subtxns);
+
+ return txn;
+}
+
+/*
+ * Free an ReorderBufferTXN. Deallocation might be delayed for efficiency
+ * purposes.
+ */
+void
+ReorderBufferReturnTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
+{
+ /* clean the lookup cache if we were cached (quite likely) */
+ if (rb->by_txn_last_xid == txn->xid)
+ {
+ rb->by_txn_last_xid = InvalidTransactionId;
+ rb->by_txn_last_txn = NULL;
+ }
+
+ if (txn->tuplecid_hash != NULL)
+ {
+ hash_destroy(txn->tuplecid_hash);
+ txn->tuplecid_hash = NULL;
+ }
+
+ if (txn->invalidations)
+ {
+ pfree(txn->invalidations);
+ txn->invalidations = NULL;
+ }
+
+ if (rb->nr_cached_transactions < max_cached_transactions)
+ {
+ rb->nr_cached_transactions++;
+ dlist_push_head(&rb->cached_transactions, &txn->node);
+ VALGRIND_MAKE_MEM_UNDEFINED(txn, sizeof(ReorderBufferTXN));
+ VALGRIND_MAKE_MEM_DEFINED(&txn->node, sizeof(txn->node));
+ }
+ else
+ {
+ pfree(txn);
+ }
+}
+
+/*
+ * Get a unused, possibly preallocated, ReorderBufferChange.
+ */
+ReorderBufferChange *
+ReorderBufferGetChange(ReorderBuffer *rb)
+{
+ ReorderBufferChange *change;
+
+ if (rb->nr_cached_changes)
+ {
+ rb->nr_cached_changes--;
+ change = (ReorderBufferChange *)
+ dlist_container(ReorderBufferChange, node,
+ dlist_pop_head_node(&rb->cached_changes));
+ }
+ else
+ {
+ change = (ReorderBufferChange *)
+ MemoryContextAlloc(rb->context, sizeof(ReorderBufferChange));
+ }
+
+ memset(change, 0, sizeof(ReorderBufferChange));
+ return change;
+}
+
+/*
+ * Free an ReorderBufferChange. Deallocation might be delayed for efficiency
+ * purposes.
+ */
+void
+ReorderBufferReturnChange(ReorderBuffer *rb, ReorderBufferChange *change)
+{
+ switch ((ReorderBufferChangeTypeInternal) change->action_internal)
+ {
+ case REORDER_BUFFER_CHANGE_INTERNAL_INSERT:
+ case REORDER_BUFFER_CHANGE_INTERNAL_UPDATE:
+ case REORDER_BUFFER_CHANGE_INTERNAL_DELETE:
+ if (change->tp.newtuple)
+ {
+ ReorderBufferReturnTupleBuf(rb, change->tp.newtuple);
+ change->tp.newtuple = NULL;
+ }
+
+ if (change->tp.oldtuple)
+ {
+ ReorderBufferReturnTupleBuf(rb, change->tp.oldtuple);
+ change->tp.oldtuple = NULL;
+ }
+ break;
+ case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
+ if (change->snapshot)
+ {
+ ReorderBufferFreeSnap(rb, change->snapshot);
+ change->snapshot = NULL;
+ }
+ break;
+ case REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID:
+ break;
+ case REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID:
+ break;
+ }
+
+ if (rb->nr_cached_changes < max_cached_changes)
+ {
+ rb->nr_cached_changes++;
+ dlist_push_head(&rb->cached_changes, &change->node);
+ VALGRIND_MAKE_MEM_UNDEFINED(change, sizeof(ReorderBufferChange));
+ VALGRIND_MAKE_MEM_DEFINED(&change->node, sizeof(change->node));
+ }
+ else
+ {
+ pfree(change);
+ }
+}
+
+
+/*
+ * Get a unused, possibly preallocated, ReorderBufferTupleBuf
+ */
+ReorderBufferTupleBuf *
+ReorderBufferGetTupleBuf(ReorderBuffer *rb)
+{
+ ReorderBufferTupleBuf *tuple;
+
+ if (rb->nr_cached_tuplebufs)
+ {
+ rb->nr_cached_tuplebufs--;
+ tuple = slist_container(ReorderBufferTupleBuf, node,
+ slist_pop_head_node(&rb->cached_tuplebufs));
+#ifdef USE_ASSERT_CHECKING
+ memset(tuple, 0xdeadbeef, sizeof(ReorderBufferTupleBuf));
+#endif
+ }
+ else
+ {
+ tuple = (ReorderBufferTupleBuf *)
+ MemoryContextAlloc(rb->context, sizeof(ReorderBufferTupleBuf));
+ }
+
+ return tuple;
+}
+
+/*
+ * Free an ReorderBufferTupleBuf. Deallocation might be delayed for efficiency
+ * purposes.
+ */
+void
+ReorderBufferReturnTupleBuf(ReorderBuffer *rb, ReorderBufferTupleBuf *tuple)
+{
+ if (rb->nr_cached_tuplebufs < max_cached_tuplebufs)
+ {
+ rb->nr_cached_tuplebufs++;
+ slist_push_head(&rb->cached_tuplebufs, &tuple->node);
+ VALGRIND_MAKE_MEM_UNDEFINED(tuple, sizeof(ReorderBufferTupleBuf));
+ VALGRIND_MAKE_MEM_DEFINED(&tuple->node, sizeof(tuple->node));
+ }
+ else
+ {
+ pfree(tuple);
+ }
+}
+
+/*
+ * Return the ReorderBufferTXN from the given buffer, specified by Xid.
+ * If create is true, and a transaction doesn't already exist, create it
+ * (with the given LSN, and as top transaction if that's specified);
+ * when this happens, is_new is set to true.
+ */
+static ReorderBufferTXN *
+ReorderBufferTXNByXid(ReorderBuffer *rb, TransactionId xid, bool create,
+ bool *is_new, XLogRecPtr lsn, bool create_as_top)
+{
+ ReorderBufferTXN *txn;
+ ReorderBufferTXNByIdEnt *ent;
+ bool found;
+
+ Assert(TransactionIdIsValid(xid));
+ Assert(!create || lsn != InvalidXLogRecPtr);
+
+ /*
+ * Check the one-entry lookup cache first
+ */
+ if (TransactionIdIsValid(rb->by_txn_last_xid) &&
+ rb->by_txn_last_xid == xid)
+ {
+ txn = rb->by_txn_last_txn;
+
+ if (txn != NULL)
+ {
+ /* found it, and it's valid */
+ if (is_new)
+ *is_new = false;
+ return txn;
+ }
+
+ /*
+ * cached as non-existant, and asked not to create? Then nothing else
+ * to do.
+ */
+ if (!create)
+ return NULL;
+ /* otherwise fall through to create it */
+ }
+
+ /*
+ * If the cache wasn't hit or it yielded an "does-not-exist" and we want
+ * to create an entry.
+ */
+
+ /* search the lookup table */
+ ent = (ReorderBufferTXNByIdEnt *)
+ hash_search(rb->by_txn,
+ (void *) &xid,
+ create ? HASH_ENTER : HASH_FIND,
+ &found);
+ if (found)
+ txn = ent->txn;
+ else if (create)
+ {
+ /* initialize the new entry, if creation was requested */
+ Assert(ent != NULL);
+
+ ent->txn = ReorderBufferGetTXN(rb);
+ ent->txn->xid = xid;
+ txn = ent->txn;
+ txn->first_lsn = lsn;
+ txn->restart_decoding_lsn = rb->current_restart_decoding_lsn;
+
+ if (create_as_top)
+ {
+ dlist_push_tail(&rb->toplevel_by_lsn, &txn->node);
+ AssertTXNLsnOrder(rb);
+ }
+ }
+ else
+ txn = NULL; /* not found and not asked to create */
+
+ /* update cache */
+ rb->by_txn_last_xid = xid;
+ rb->by_txn_last_txn = txn;
+
+ if (is_new)
+ *is_new = !found;
+
+ Assert(!create || !!txn);
+ return txn;
+}
+
+/*
+ * Queue a change into a transaction so it can be replayed upon commit.
+ */
+void
+ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
+ ReorderBufferChange *change)
+{
+ ReorderBufferTXN *txn;
+
+ txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
+
+ change->lsn = lsn;
+ Assert(InvalidXLogRecPtr != lsn);
+ dlist_push_tail(&txn->changes, &change->node);
+ txn->nentries++;
+ txn->nentries_mem++;
+
+ ReorderBufferCheckSerializeTXN(rb, txn);
+}
+
+static void
+AssertTXNLsnOrder(ReorderBuffer *rb)
+{
+#ifdef USE_ASSERT_CHECKING
+ dlist_iter iter;
+ XLogRecPtr prev_first_lsn = InvalidXLogRecPtr;
+
+ dlist_foreach(iter, &rb->toplevel_by_lsn)
+ {
+ ReorderBufferTXN *cur_txn;
+
+ cur_txn = dlist_container(ReorderBufferTXN, node, iter.cur);
+ Assert(cur_txn->first_lsn != InvalidXLogRecPtr);
+
+ if (cur_txn->end_lsn != InvalidXLogRecPtr)
+ Assert(cur_txn->first_lsn <= cur_txn->end_lsn);
+
+ if (prev_first_lsn != InvalidXLogRecPtr)
+ Assert(prev_first_lsn < cur_txn->first_lsn);
+
+ Assert(!cur_txn->is_known_as_subxact);
+ prev_first_lsn = cur_txn->first_lsn;
+ }
+#endif
+}
+
+ReorderBufferTXN *
+ReorderBufferGetOldestTXN(ReorderBuffer *rb)
+{
+ ReorderBufferTXN *txn;
+
+ if (dlist_is_empty(&rb->toplevel_by_lsn))
+ return NULL;
+
+ AssertTXNLsnOrder(rb);
+
+ txn = dlist_head_element(ReorderBufferTXN, node, &rb->toplevel_by_lsn);
+
+ Assert(!txn->is_known_as_subxact);
+ Assert(txn->first_lsn != InvalidXLogRecPtr);
+ return txn;
+}
+
+void
+ReorderBufferSetRestartPoint(ReorderBuffer *rb, XLogRecPtr ptr)
+{
+ rb->current_restart_decoding_lsn = ptr;
+}
+
+void
+ReorderBufferAssignChild(ReorderBuffer *rb, TransactionId xid,
+ TransactionId subxid, XLogRecPtr lsn)
+{
+ ReorderBufferTXN *txn;
+ ReorderBufferTXN *subtxn;
+ bool new_top;
+ bool new_sub;
+
+ txn = ReorderBufferTXNByXid(rb, xid, true, &new_top, lsn, true);
+ subtxn = ReorderBufferTXNByXid(rb, subxid, true, &new_sub, lsn, false);
+
+ if (new_sub)
+ {
+ /*
+ * we assign subtransactions to top level transaction even if we don't
+ * have data for it yet, assignment records frequently reference xids
+ * that have not yet produced any records. Knowing those aren't top
+ * level xids allows us to make processing cheaper in some places.
+ */
+ dlist_push_tail(&txn->subtxns, &subtxn->node);
+ txn->nsubtxns++;
+ }
+ else if (!subtxn->is_known_as_subxact)
+ {
+ subtxn->is_known_as_subxact = true;
+ Assert(subtxn->nsubtxns == 0);
+
+ /* remove from lsn order list of top-level transactions */
+ dlist_delete(&subtxn->node);
+
+ /* add to toplevel transaction */
+ dlist_push_tail(&txn->subtxns, &subtxn->node);
+ txn->nsubtxns++;
+ }
+ else if (new_top)
+ {
+ elog(ERROR, "existing subxact assigned to unknown toplevel xact");
+ }
+}
+
+/*
+ * Associate a subtransaction with its toplevel transaction at commit
+ * time. There may be no further changes added after this.
+ */
+void
+ReorderBufferCommitChild(ReorderBuffer *rb, TransactionId xid,
+ TransactionId subxid, XLogRecPtr commit_lsn,
+ XLogRecPtr end_lsn)
+{
+ ReorderBufferTXN *txn;
+ ReorderBufferTXN *subtxn;
+
+ subtxn = ReorderBufferTXNByXid(rb, subxid, false, NULL,
+ InvalidXLogRecPtr, false);
+
+ /*
+ * No need to do anything if that subtxn didn't contain any changes
+ */
+ if (!subtxn)
+ return;
+
+ txn = ReorderBufferTXNByXid(rb, xid, false, NULL, commit_lsn, true);
+
+ if (txn == NULL)
+ elog(ERROR, "subxact logged without previous toplevel record");
+
+ subtxn->final_lsn = commit_lsn;
+ subtxn->end_lsn = end_lsn;
+
+ if (!subtxn->is_known_as_subxact)
+ {
+ subtxn->is_known_as_subxact = true;
+ Assert(subtxn->nsubtxns == 0);
+
+ /* remove from lsn order list of top-level transactions */
+ dlist_delete(&subtxn->node);
+
+ /* add to subtransaction list */
+ dlist_push_tail(&txn->subtxns, &subtxn->node);
+ txn->nsubtxns++;
+ }
+}
+
+
+/*
+ * Support for efficiently iterating over a transaction's and its
+ * subtransactions' changes.
+ *
+ * We do by doing a k-way merge between transactions/subtransactions. For that
+ * we model the current heads of the different transactions as a binary heap
+ * so we easily know which (sub-)transaction has the change with the smallest
+ * lsn next.
+ *
+ * We assume the changes in individual transactions are already sorted by LSN.
+ */
+
+/*
+ * Binary heap comparison function.
+ */
+static int
+ReorderBufferIterCompare(Datum a, Datum b, void *arg)
+{
+ ReorderBufferIterTXNState *state = (ReorderBufferIterTXNState *) arg;
+ XLogRecPtr pos_a = state->entries[DatumGetInt32(a)].lsn;
+ XLogRecPtr pos_b = state->entries[DatumGetInt32(b)].lsn;
+
+ if (pos_a < pos_b)
+ return 1;
+ else if (pos_a == pos_b)
+ return 0;
+ return -1;
+}
+
+/*
+ * Allocate & initialize an iterator which iterates in lsn order over a
+ * transaction and all its subtransactions.
+ */
+static ReorderBufferIterTXNState *
+ReorderBufferIterTXNInit(ReorderBuffer *rb, ReorderBufferTXN *txn)
+{
+ Size nr_txns = 0;
+ ReorderBufferIterTXNState *state;
+ dlist_iter cur_txn_i;
+ int32 off;
+
+ /*
+ * Calculate the size of our heap: one element for every transaction that
+ * contains changes. (Besides the transactions already in the reorder
+ * buffer, we count the one we were directly passed.)
+ */
+ if (txn->nentries > 0)
+ nr_txns++;
+
+ dlist_foreach(cur_txn_i, &txn->subtxns)
+ {
+ ReorderBufferTXN *cur_txn;
+
+ cur_txn = dlist_container(ReorderBufferTXN, node, cur_txn_i.cur);
+
+ if (cur_txn->nentries > 0)
+ nr_txns++;
+ }
+
+ /*
+ * XXX: Add fastpath for the rather common nr_txns=1 case, no need to
+ * allocate/build a heap in that case.
+ */
+
+ /* allocate iteration state */
+ state = (ReorderBufferIterTXNState *)
+ MemoryContextAllocZero(rb->context,
+ sizeof(ReorderBufferIterTXNState) +
+ sizeof(ReorderBufferIterTXNEntry) * nr_txns);
+
+ state->nr_txns = nr_txns;
+ dlist_init(&state->old_change);
+
+ for (off = 0; off < state->nr_txns; off++)
+ {
+ state->entries[off].fd = -1;
+ state->entries[off].segno = 0;
+ }
+
+ /* allocate heap */
+ state->heap = binaryheap_allocate(state->nr_txns,
+ ReorderBufferIterCompare,
+ state);
+
+ /*
+ * Now insert items into the binary heap, unordered. (We will run a heap
+ * assembly step at the end; this is more efficient.)
+ */
+
+ off = 0;
+
+ /* add toplevel transaction if it contains changes */
+ if (txn->nentries > 0)
+ {
+ ReorderBufferChange *cur_change;
+
+ if (txn->nentries != txn->nentries_mem)
+ ReorderBufferRestoreChanges(rb, txn, &state->entries[off].fd,
+ &state->entries[off].segno);
+
+ cur_change = dlist_head_element(ReorderBufferChange, node,
+ &txn->changes);
+
+ state->entries[off].lsn = cur_change->lsn;
+ state->entries[off].change = cur_change;
+ state->entries[off].txn = txn;
+
+ binaryheap_add_unordered(state->heap, Int32GetDatum(off++));
+ }
+
+ /* add subtransactions if they contain changes */
+ dlist_foreach(cur_txn_i, &txn->subtxns)
+ {
+ ReorderBufferTXN *cur_txn;
+
+ cur_txn = dlist_container(ReorderBufferTXN, node, cur_txn_i.cur);
+
+ if (cur_txn->nentries > 0)
+ {
+ ReorderBufferChange *cur_change;
+
+ if (txn->nentries != txn->nentries_mem)
+ ReorderBufferRestoreChanges(rb, cur_txn,
+ &state->entries[off].fd,
+ &state->entries[off].segno);
+
+ cur_change = dlist_head_element(ReorderBufferChange, node,
+ &cur_txn->changes);
+
+ state->entries[off].lsn = cur_change->lsn;
+ state->entries[off].change = cur_change;
+ state->entries[off].txn = cur_txn;
+
+ binaryheap_add_unordered(state->heap, Int32GetDatum(off++));
+ }
+ }
+
+ /* assemble a valid binary heap */
+ binaryheap_build(state->heap);
+
+ return state;
+}
+
+/*
+ * FIXME: better comment and/or name
+ */
+static void
+ReorderBufferRestoreCleanup(ReorderBuffer *rb, ReorderBufferTXN *txn)
+{
+ XLogSegNo first;
+ XLogSegNo cur;
+ XLogSegNo last;
+
+ Assert(txn->first_lsn != InvalidXLogRecPtr);
+ Assert(txn->final_lsn != InvalidXLogRecPtr);
+
+ XLByteToSeg(txn->first_lsn, first);
+ XLByteToSeg(txn->final_lsn, last);
+
+ /* iterate over all possible filenames, and delete them */
+ for (cur = first; cur <= last; cur++)
+ {
+ char path[MAXPGPATH];
+ XLogRecPtr recptr;
+
+ XLogSegNoOffsetToRecPtr(cur, 0, recptr);
+
+ sprintf(path, "pg_llog/%s/xid-%u-lsn-%X-%X.snap",
+ NameStr(MyLogicalDecodingSlot->name), txn->xid,
+ (uint32) (recptr >> 32), (uint32) recptr);
+ if (unlink(path) != 0 && errno != ENOENT)
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not unlink file \"%s\": %m", path)));
+ }
+}
+
+/*
+ * Return the next change when iterating over a transaction and its
+ * subtransaction.
+ *
+ * Returns NULL when no further changes exist.
+ */
+static ReorderBufferChange *
+ReorderBufferIterTXNNext(ReorderBuffer *rb, ReorderBufferIterTXNState *state)
+{
+ ReorderBufferChange *change;
+ ReorderBufferIterTXNEntry *entry;
+ int32 off;
+
+ /* nothing there anymore */
+ if (state->heap->bh_size == 0)
+ return NULL;
+
+ off = DatumGetInt32(binaryheap_first(state->heap));
+ entry = &state->entries[off];
+
+ /* free memory we might have "leaked" in the previous *Next call */
+ if (!dlist_is_empty(&state->old_change))
+ {
+ change = dlist_container(ReorderBufferChange, node,
+ dlist_pop_head_node(&state->old_change));
+ ReorderBufferReturnChange(rb, change);
+ Assert(dlist_is_empty(&state->old_change));
+ }
+
+ change = entry->change;
+
+ /*
+ * update heap with information about which transaction has the next
+ * relevant change in LSN order
+ */
+
+ /* there are in-memory changes */
+ if (dlist_has_next(&entry->txn->changes, &entry->change->node))
+ {
+ dlist_node *next = dlist_next_node(&entry->txn->changes, &change->node);
+ ReorderBufferChange *next_change =
+ dlist_container(ReorderBufferChange, node, next);
+
+ /* txn stays the same */
+ state->entries[off].lsn = next_change->lsn;
+ state->entries[off].change = next_change;
+
+ binaryheap_replace_first(state->heap, Int32GetDatum(off));
+ return change;
+ }
+
+ /* try to load changes from disk */
+ if (entry->txn->nentries != entry->txn->nentries_mem)
+ {
+ /*
+ * Ugly: restoring changes will reuse *Change records, thus delete the
+ * current one from the per-tx list and only free in the next call.
+ */
+ dlist_delete(&change->node);
+ dlist_push_tail(&state->old_change, &change->node);
+
+ if (ReorderBufferRestoreChanges(rb, entry->txn, &entry->fd,
+ &state->entries[off].segno))
+ {
+ /* successfully restored changes from disk */
+ ReorderBufferChange *next_change =
+ dlist_head_element(ReorderBufferChange, node,
+ &entry->txn->changes);
+
+ elog(DEBUG2, "restored %u/%u changes from disk",
+ (uint32) entry->txn->nentries_mem,
+ (uint32) entry->txn->nentries);
+
+ Assert(entry->txn->nentries_mem);
+ /* txn stays the same */
+ state->entries[off].lsn = next_change->lsn;
+ state->entries[off].change = next_change;
+ binaryheap_replace_first(state->heap, Int32GetDatum(off));
+
+ return change;
+ }
+ }
+
+ /* ok, no changes there anymore, remove */
+ binaryheap_remove_first(state->heap);
+
+ return change;
+}
+
+/*
+ * Deallocate the iterator
+ */
+static void
+ReorderBufferIterTXNFinish(ReorderBuffer *rb,
+ ReorderBufferIterTXNState *state)
+{
+ int32 off;
+
+ for (off = 0; off < state->nr_txns; off++)
+ {
+ if (state->entries[off].fd != -1)
+ CloseTransientFile(state->entries[off].fd);
+ }
+
+ /* free memory we might have "leaked" in the last *Next call */
+ if (!dlist_is_empty(&state->old_change))
+ {
+ ReorderBufferChange *change;
+
+ change = dlist_container(ReorderBufferChange, node,
+ dlist_pop_head_node(&state->old_change));
+ ReorderBufferReturnChange(rb, change);
+ Assert(dlist_is_empty(&state->old_change));
+ }
+
+ binaryheap_free(state->heap);
+ pfree(state);
+}
+
+/*
+ * Cleanup the contents of a transaction, usually after the transaction
+ * committed or aborted.
+ */
+static void
+ReorderBufferCleanupTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
+{
+ bool found;
+ dlist_mutable_iter iter;
+
+ /* cleanup subtransactions & their changes */
+ dlist_foreach_modify(iter, &txn->subtxns)
+ {
+ ReorderBufferTXN *subtxn;
+
+ subtxn = dlist_container(ReorderBufferTXN, node, iter.cur);
+
+ /*
+ * Subtransactions are always associated to the toplevel TXN, even if
+ * they originally were happening inside another subtxn, so we won't
+ * ever recurse more than one level deep here.
+ */
+ Assert(subtxn->is_known_as_subxact);
+ Assert(subtxn->nsubtxns == 0);
+
+ ReorderBufferCleanupTXN(rb, subtxn);
+ }
+
+ /* cleanup changes in the toplevel txn */
+ dlist_foreach_modify(iter, &txn->changes)
+ {
+ ReorderBufferChange *change;
+
+ change = dlist_container(ReorderBufferChange, node, iter.cur);
+
+ ReorderBufferReturnChange(rb, change);
+ }
+
+ /*
+ * Cleanup the tuplecids we stored for decoding catalog snapshot
+ * access. They are always stored in the toplevel transaction.
+ */
+ dlist_foreach_modify(iter, &txn->tuplecids)
+ {
+ ReorderBufferChange *change;
+
+ change = dlist_container(ReorderBufferChange, node, iter.cur);
+ Assert(change->action_internal == REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID);
+ ReorderBufferReturnChange(rb, change);
+ }
+
+ if (txn->base_snapshot != NULL)
+ {
+ SnapBuildSnapDecRefcount(txn->base_snapshot);
+ txn->base_snapshot = NULL;
+ }
+
+ /* delete from list of known subxacts */
+ if (txn->is_known_as_subxact)
+ {
+ /* FIXME: adjust nsubxacts count of parent */
+ dlist_delete(&txn->node);
+ }
+ /* delete from LSN ordered list of toplevel TXNs */
+ else
+ {
+ dlist_delete(&txn->node);
+ }
+
+ /* now remove reference from buffer */
+ hash_search(rb->by_txn,
+ (void *) &txn->xid,
+ HASH_REMOVE,
+ &found);
+ Assert(found);
+
+ /* remove entries spilled to disk */
+ if (txn->nentries != txn->nentries_mem)
+ ReorderBufferRestoreCleanup(rb, txn);
+
+ /* deallocate */
+ ReorderBufferReturnTXN(rb, txn);
+}
+
+/*
+ * Build a hash with a (relfilenode, ctid) -> (cmin, cmax) mapping for use by
+ * tqual.c's HeapTupleSatisfiesMVCCDuringDecoding.
+ */
+static void
+ReorderBufferBuildTupleCidHash(ReorderBuffer *rb, ReorderBufferTXN *txn)
+{
+ dlist_iter iter;
+ HASHCTL hash_ctl;
+
+ if (!txn->has_catalog_changes || dlist_is_empty(&txn->tuplecids))
+ return;
+
+ memset(&hash_ctl, 0, sizeof(hash_ctl));
+
+ hash_ctl.keysize = sizeof(ReorderBufferTupleCidKey);
+ hash_ctl.entrysize = sizeof(ReorderBufferTupleCidEnt);
+ hash_ctl.hash = tag_hash;
+ hash_ctl.hcxt = rb->context;
+
+ /*
+ * create the hash with the exact number of to-be-stored tuplecids from
+ * the start
+ */
+ txn->tuplecid_hash =
+ hash_create("ReorderBufferTupleCid", txn->ntuplecids, &hash_ctl,
+ HASH_ELEM | HASH_FUNCTION | HASH_CONTEXT);
+
+ dlist_foreach(iter, &txn->tuplecids)
+ {
+ ReorderBufferTupleCidKey key;
+ ReorderBufferTupleCidEnt *ent;
+ bool found;
+ ReorderBufferChange *change;
+
+ change = dlist_container(ReorderBufferChange, node, iter.cur);
+
+ Assert(change->action_internal == REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID);
+
+ /* be careful about padding */
+ memset(&key, 0, sizeof(ReorderBufferTupleCidKey));
+
+ key.relnode = change->tuplecid.node;
+
+ ItemPointerCopy(&change->tuplecid.tid,
+ &key.tid);
+
+ ent = (ReorderBufferTupleCidEnt *)
+ hash_search(txn->tuplecid_hash,
+ (void *) &key,
+ HASH_ENTER | HASH_FIND,
+ &found);
+ if (!found)
+ {
+ ent->cmin = change->tuplecid.cmin;
+ ent->cmax = change->tuplecid.cmax;
+ ent->combocid = change->tuplecid.combocid;
+ }
+ else
+ {
+ Assert(ent->cmin == change->tuplecid.cmin);
+ Assert(ent->cmax == InvalidCommandId ||
+ ent->cmax == change->tuplecid.cmax);
+
+ /*
+ * if the tuple got valid in this transaction and now got deleted
+ * we already have a valid cmin stored. The cmax will be
+ * InvalidCommandId though.
+ */
+ ent->cmax = change->tuplecid.cmax;
+ }
+ }
+}
+
+/*
+ * Copy a provided snapshot so we can modify it privately. This is needed so
+ * that catalog modifying transactions can look into intermediate catalog
+ * states.
+ */
+static Snapshot
+ReorderBufferCopySnap(ReorderBuffer *rb, Snapshot orig_snap,
+ ReorderBufferTXN *txn, CommandId cid)
+{
+ Snapshot snap;
+ dlist_iter iter;
+ int i = 0;
+ Size size;
+
+ size = sizeof(SnapshotData) +
+ sizeof(TransactionId) * orig_snap->xcnt +
+ sizeof(TransactionId) * (txn->nsubtxns + 1);
+
+ snap = MemoryContextAllocZero(rb->context, size);
+ memcpy(snap, orig_snap, sizeof(SnapshotData));
+
+ snap->copied = true;
+ snap->active_count = 0;
+ snap->regd_count = 0;
+ snap->xip = (TransactionId *) (snap + 1);
+
+ memcpy(snap->xip, orig_snap->xip, sizeof(TransactionId) * snap->xcnt);
+
+ /*
+ * ->subxip contains all txids that belong to our transaction which we
+ * need to check via cmin/cmax. Thats why we store the toplevel
+ * transaction in there as well.
+ */
+ snap->subxip = snap->xip + snap->xcnt;
+ snap->subxip[i++] = txn->xid;
+
+ /*
+ * XXX: ->nsubxcnt can be out of date when subtransactions abort, count
+ * manually.
+ */
+ snap->subxcnt = 1;
+
+ dlist_foreach(iter, &txn->subtxns)
+ {
+ ReorderBufferTXN *sub_txn;
+
+ sub_txn = dlist_container(ReorderBufferTXN, node, iter.cur);
+ snap->subxip[i++] = sub_txn->xid;
+ snap->subxcnt++;
+ }
+
+ /* sort so we can bsearch() later */
+ qsort(snap->subxip, snap->subxcnt, sizeof(TransactionId), xidComparator);
+
+ /* store the specified current CommandId */
+ snap->curcid = cid;
+
+ return snap;
+}
+
+/*
+ * Free a previously ReorderBufferCopySnap'ed snapshot
+ */
+static void
+ReorderBufferFreeSnap(ReorderBuffer *rb, Snapshot snap)
+{
+ if (snap->copied)
+ pfree(snap);
+ else
+ SnapBuildSnapDecRefcount(snap);
+}
+
+/*
+ * Commit a transaction and replay all actions that previously have been
+ * ReorderBufferQueueChange'd in the toplevel TX or any of the subtransactions
+ * assigned via ReorderBufferCommitChild.
+ */
+void
+ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
+ XLogRecPtr commit_lsn, XLogRecPtr end_lsn,
+ TimestampTz commit_time)
+{
+ ReorderBufferTXN *txn;
+ ReorderBufferIterTXNState *iterstate = NULL;
+ ReorderBufferChange *change;
+
+ volatile CommandId command_id = FirstCommandId;
+ volatile Snapshot snapshot_now;
+ volatile bool tx_started = false;
+ volatile bool subtx_started = false;
+
+ txn = ReorderBufferTXNByXid(rb, xid, false, NULL, InvalidXLogRecPtr,
+ false);
+
+ /* empty transaction */
+ if (txn == NULL)
+ return;
+
+ txn->final_lsn = commit_lsn;
+ txn->end_lsn = end_lsn;
+ txn->commit_time = commit_time;
+
+ /* serialize the last bunch of changes if we need start earlier anyway */
+ if (txn->nentries_mem != txn->nentries)
+ ReorderBufferSerializeTXN(rb, txn);
+
+ /*
+ * If this transaction didn't have any real changes in our database, it's
+ * OK not to have a snapshot.
+ */
+ if (txn->base_snapshot == NULL)
+ {
+ Assert(txn->ninvalidations == 0);
+ ReorderBufferCleanupTXN(rb, txn);
+ return;
+ }
+
+ snapshot_now = txn->base_snapshot;
+
+ ReorderBufferBuildTupleCidHash(rb, txn);
+
+ /* setup initial snapshot */
+ SetupDecodingSnapshots(snapshot_now, txn->tuplecid_hash);
+
+ PG_TRY();
+ {
+ tx_started = false;
+
+ /*
+ * Decoding needs access to syscaches et al., which in turn use
+ * heavyweight locks and such. Thus we need to have enough state around
+ * to keep track of those. The easiest way is to simply use a
+ * transaction internally. That also allows us to easily enforce that
+ * nothing writes to the database by checking for xid assignments.
+ *
+ * When we're called via the SQL SRF there's already a transaction
+ * started, so start an explicit subtransaction there.
+ */
+ if (IsTransactionOrTransactionBlock())
+ {
+ BeginInternalSubTransaction("replay");
+ subtx_started = true;
+
+ if (GetTopTransactionIdIfAny() != InvalidTransactionId)
+ elog(ERROR, "cannot replay using sub, already allocated xid %u",
+ GetTopTransactionIdIfAny());
+ }
+ else
+ {
+ StartTransactionCommand();
+ tx_started = true;
+
+ if (GetTopTransactionIdIfAny() != InvalidTransactionId)
+ elog(ERROR, "cannot replay using top, already allocated xid %u",
+ GetTopTransactionIdIfAny());
+
+ }
+
+ rb->begin(rb, txn);
+
+ iterstate = ReorderBufferIterTXNInit(rb, txn);
+ while ((change = ReorderBufferIterTXNNext(rb, iterstate)))
+ {
+ Relation relation = NULL;
+ Oid reloid;
+
+ switch ((ReorderBufferChangeTypeInternal) change->action_internal)
+ {
+ case REORDER_BUFFER_CHANGE_INTERNAL_INSERT:
+ case REORDER_BUFFER_CHANGE_INTERNAL_UPDATE:
+ case REORDER_BUFFER_CHANGE_INTERNAL_DELETE:
+ Assert(snapshot_now);
+
+ reloid = RelidByRelfilenode(change->tp.relnode.spcNode,
+ change->tp.relnode.relNode);
+
+ /*
+ * catalog tuple without data, while catalog has been
+ * rewritten
+ */
+ if (reloid == InvalidOid &&
+ change->tp.newtuple == NULL &&
+ change->tp.oldtuple == NULL)
+ continue;
+ else if (reloid == InvalidOid)
+ elog(ERROR, "could not lookup relation %s",
+ relpathperm(change->tp.relnode, MAIN_FORKNUM));
+
+ relation = RelationIdGetRelation(reloid);
+
+ if (relation == NULL)
+ elog(ERROR, "could open relation descriptor %s",
+ relpathperm(change->tp.relnode, MAIN_FORKNUM));
+
+ if (RelationIsLogicallyLogged(relation))
+ {
+ /* user-triggered change */
+ if (relation->rd_rel->relkind == RELKIND_SEQUENCE)
+ {
+ }
+ else if (!IsToastRelation(relation))
+ {
+ ReorderBufferToastReplace(rb, txn, relation, change);
+ rb->apply_change(rb, txn, relation, change);
+ ReorderBufferToastReset(rb, txn);
+ }
+ /* we're not interested in toast deletions */
+ else if (change->action == REORDER_BUFFER_CHANGE_INSERT)
+ {
+ /*
+ * need to reassemble change in memory, ensure it
+ * doesn't get reused till we're done.
+ */
+ dlist_delete(&change->node);
+ ReorderBufferToastAppendChunk(rb, txn, relation,
+ change);
+ }
+
+ }
+ RelationClose(relation);
+ break;
+ case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
+ /* XXX: we could skip snapshots in non toplevel txns */
+
+ /* get rid of the old */
+ RevertFromDecodingSnapshots(false);
+
+ if (snapshot_now->copied)
+ {
+ ReorderBufferFreeSnap(rb, snapshot_now);
+ snapshot_now =
+ ReorderBufferCopySnap(rb, change->snapshot,
+ txn, command_id);
+ }
+
+ /*
+ * restored from disk, we need to be careful not to double
+ * free. We could introduce refcounting for that, but for
+ * now this seems infrequent enough not to care.
+ */
+ else if (change->snapshot->copied)
+ {
+ snapshot_now =
+ ReorderBufferCopySnap(rb, change->snapshot,
+ txn, command_id);
+ }
+ else
+ {
+ snapshot_now = change->snapshot;
+ }
+
+
+ /* and start with the new one */
+ SetupDecodingSnapshots(snapshot_now, txn->tuplecid_hash);
+ break;
+
+ case REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID:
+ Assert(change->command_id != InvalidCommandId);
+
+ if (command_id < change->command_id)
+ {
+ command_id = change->command_id;
+
+ if (!snapshot_now->copied)
+ {
+ /* we don't use the global one anymore */
+ snapshot_now = ReorderBufferCopySnap(rb, snapshot_now,
+ txn, command_id);
+ }
+
+ snapshot_now->curcid = command_id;
+
+ RevertFromDecodingSnapshots(false);
+ SetupDecodingSnapshots(snapshot_now, txn->tuplecid_hash);
+
+ /*
+ * everytime the CommandId is incremented, we could see
+ * new catalog contents
+ */
+ ReorderBufferExecuteInvalidations(rb, txn);
+ }
+
+ break;
+
+ case REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID:
+ elog(ERROR, "tuplecid value in normal queue");
+ break;
+ }
+ }
+
+ ReorderBufferIterTXNFinish(rb, iterstate);
+
+ /* call commit callback */
+ rb->commit(rb, txn, commit_lsn);
+
+ /* make sure nothing has written anything */
+ if (GetTopTransactionIdIfAny() != InvalidTransactionId)
+ elog(ERROR, "cannot write during replay, allocated xid %u",
+ GetTopTransactionIdIfAny());
+
+ /* make sure there's no cache pollution */
+ ReorderBufferExecuteInvalidations(rb, txn);
+
+ /* cleanup */
+ RevertFromDecodingSnapshots(false);
+
+ /*
+ * Abort subtransaction or aborting transaction as a whole has the
+ * right semantics. We want all locks acquired in here to be released,
+ * not reassinged to the parent and we do not want any database access
+ * have persistent effects.
+ */
+ if (subtx_started)
+ RollbackAndReleaseCurrentSubTransaction();
+ else if (tx_started)
+ AbortCurrentTransaction();
+
+ if (snapshot_now->copied)
+ ReorderBufferFreeSnap(rb, snapshot_now);
+
+ ReorderBufferCleanupTXN(rb, txn);
+ }
+ PG_CATCH();
+ {
+ /* TODO: Encapsulate cleanup from the PG_TRY and PG_CATCH blocks */
+ if (iterstate)
+ ReorderBufferIterTXNFinish(rb, iterstate);
+
+ ReorderBufferExecuteInvalidations(rb, txn);
+
+ RevertFromDecodingSnapshots(true);
+
+ if (subtx_started)
+ RollbackAndReleaseCurrentSubTransaction();
+ else if (tx_started)
+ AbortCurrentTransaction();
+
+ if (snapshot_now->copied)
+ ReorderBufferFreeSnap(rb, snapshot_now);
+
+ /*
+ * don't do a ReorderBufferCleanupTXN here, with the vague idea of
+ * allowing to retry decoding.
+ */
+ PG_RE_THROW();
+ }
+ PG_END_TRY();
+}
+
+/*
+ * Abort a transaction that possibly has previous changes. Needs to be done
+ * independently for toplevel and subtransactions.
+ */
+void
+ReorderBufferAbort(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn)
+{
+ ReorderBufferTXN *txn;
+
+ txn = ReorderBufferTXNByXid(rb, xid, false, NULL, InvalidXLogRecPtr,
+ false);
+
+ /* no changes in this commit */
+ if (txn == NULL)
+ return;
+
+ txn->final_lsn = lsn;
+
+ ReorderBufferCleanupTXN(rb, txn);
+}
+
+/*
+ * Abort all transactions that aren't actually running anymore because the
+ * server restarted.
+ */
+void
+ReorderBufferAbortOld(ReorderBuffer *rb, TransactionId oldestRunningXid)
+{
+ dlist_mutable_iter it;
+
+ dlist_foreach_modify(it, &rb->toplevel_by_lsn)
+ {
+ ReorderBufferTXN * txn;
+
+ txn = dlist_container(ReorderBufferTXN, node, it.cur);
+
+ /*
+ * Iterate through all (potential) toplevel TXNs and abort all that
+ * are older than what possibly can be running. Once we've found the
+ * first that is alive we stop, there might be some that acquired an
+ * xid earlier but started writing later, but it's unlikely and they
+ * will cleaned up in a later call to ReorderBufferAbortOld().
+ */
+ if (TransactionIdPrecedes(txn->xid, oldestRunningXid))
+ {
+ elog(DEBUG1, "aborting old transaction %u", txn->xid);
+ ReorderBufferCleanupTXN(rb, txn);
+ }
+ else
+ return;
+ }
+}
+
+/*
+ * Check whether a transaction is already known in this module
+ */
+bool
+ReorderBufferIsXidKnown(ReorderBuffer *rb, TransactionId xid)
+{
+ ReorderBufferTXN *txn;
+
+ txn = ReorderBufferTXNByXid(rb, xid, false, NULL, InvalidXLogRecPtr,
+ false);
+ return txn != NULL;
+}
+
+/*
+ * Add a new snapshot to this transaction that is only used after lsn 'lsn'.
+ */
+void
+ReorderBufferAddSnapshot(ReorderBuffer *rb, TransactionId xid,
+ XLogRecPtr lsn, Snapshot snap)
+{
+ ReorderBufferChange *change = ReorderBufferGetChange(rb);
+
+ change->snapshot = snap;
+ change->action_internal = REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT;
+
+ ReorderBufferQueueChange(rb, xid, lsn, change);
+}
+
+/*
+ * Setup the base snapshot of a transaction. That is the snapshot that is used
+ * to decode all changes until either this transaction modifies the catalog or
+ * another catalog modifying transaction commits.
+ */
+void
+ReorderBufferSetBaseSnapshot(ReorderBuffer *rb, TransactionId xid,
+ XLogRecPtr lsn, Snapshot snap)
+{
+ ReorderBufferTXN *txn;
+ bool is_new;
+
+ txn = ReorderBufferTXNByXid(rb, xid, true, &is_new, lsn, true);
+ Assert(txn->base_snapshot == NULL);
+
+ txn->base_snapshot = snap;
+}
+
+/*
+ * Access the catalog with this CommandId at this point in the changestream.
+ *
+ * May only be called for command ids > 1
+ */
+void
+ReorderBufferAddNewCommandId(ReorderBuffer *rb, TransactionId xid,
+ XLogRecPtr lsn, CommandId cid)
+{
+ ReorderBufferChange *change = ReorderBufferGetChange(rb);
+
+ change->command_id = cid;
+ change->action_internal = REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID;
+
+ ReorderBufferQueueChange(rb, xid, lsn, change);
+}
+
+
+/*
+ * Add new (relfilenode, tid) -> (cmin, cmax) mappings.
+ */
+void
+ReorderBufferAddNewTupleCids(ReorderBuffer *rb, TransactionId xid,
+ XLogRecPtr lsn, RelFileNode node,
+ ItemPointerData tid, CommandId cmin,
+ CommandId cmax, CommandId combocid)
+{
+ ReorderBufferChange *change = ReorderBufferGetChange(rb);
+ ReorderBufferTXN *txn;
+
+ txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
+
+ change->tuplecid.node = node;
+ change->tuplecid.tid = tid;
+ change->tuplecid.cmin = cmin;
+ change->tuplecid.cmax = cmax;
+ change->tuplecid.combocid = combocid;
+ change->lsn = lsn;
+ change->action_internal = REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID;
+
+ dlist_push_tail(&txn->tuplecids, &change->node);
+ txn->ntuplecids++;
+}
+
+/*
+ * Setup the invalidation of the toplevel transaction.
+ *
+ * This needs to be done before ReorderBufferCommit is called!
+ */
+void
+ReorderBufferAddInvalidations(ReorderBuffer *rb, TransactionId xid,
+ XLogRecPtr lsn, Size nmsgs,
+ SharedInvalidationMessage *msgs)
+{
+ ReorderBufferTXN *txn;
+
+ txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
+
+ if (txn->ninvalidations != 0)
+ elog(ERROR, "only ever add one set of invalidations");
+
+ Assert(nmsgs > 0);
+
+ txn->ninvalidations = nmsgs;
+ txn->invalidations = (SharedInvalidationMessage *)
+ MemoryContextAlloc(rb->context,
+ sizeof(SharedInvalidationMessage) * nmsgs);
+ memcpy(txn->invalidations, msgs,
+ sizeof(SharedInvalidationMessage) * nmsgs);
+}
+
+/*
+ * Apply all invalidations we know. Possibly we only need parts at this point
+ * in the changestream but we don't know which those are.
+ */
+static void
+ReorderBufferExecuteInvalidations(ReorderBuffer *rb, ReorderBufferTXN *txn)
+{
+ int i;
+
+ for (i = 0; i < txn->ninvalidations; i++)
+ LocalExecuteInvalidationMessage(&txn->invalidations[i]);
+}
+
+/*
+ * Mark a transaction as containing catalog changes
+ */
+void
+ReorderBufferXidSetCatalogChanges(ReorderBuffer *rb, TransactionId xid,
+ XLogRecPtr lsn)
+{
+ ReorderBufferTXN *txn;
+
+ txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
+
+ txn->has_catalog_changes = true;
+}
+
+/*
+ * Query whether a transaction is already *known* to contain catalog
+ * changes. This can be wrong until directly before the commit!
+ */
+bool
+ReorderBufferXidHasCatalogChanges(ReorderBuffer *rb, TransactionId xid)
+{
+ ReorderBufferTXN *txn;
+
+ txn = ReorderBufferTXNByXid(rb, xid, false, NULL, InvalidXLogRecPtr,
+ false);
+ if (txn == NULL)
+ return false;
+
+ return txn->has_catalog_changes;
+}
+
+/*
+ * Have we already added the first snapshot?
+ */
+bool
+ReorderBufferXidHasBaseSnapshot(ReorderBuffer *rb, TransactionId xid)
+{
+ ReorderBufferTXN *txn;
+
+ txn = ReorderBufferTXNByXid(rb, xid, false, NULL, InvalidXLogRecPtr,
+ false);
+
+ /* transaction isn't known yet, ergo no snapshot */
+ if (txn == NULL)
+ return false;
+
+ return txn->base_snapshot != NULL;
+}
+
+static void
+ReorderBufferSerializeReserve(ReorderBuffer *rb, Size sz)
+{
+ if (!rb->outbufsize)
+ {
+ rb->outbuf = MemoryContextAlloc(rb->context, sz);
+ rb->outbufsize = sz;
+ }
+ else if (rb->outbufsize < sz)
+ {
+ rb->outbuf = repalloc(rb->outbuf, sz);
+ rb->outbufsize = sz;
+ }
+}
+
+typedef struct ReorderBufferDiskChange
+{
+ Size size;
+ ReorderBufferChange change;
+ /* data follows */
+} ReorderBufferDiskChange;
+
+/*
+ * Persistency support
+ */
+static void
+ReorderBufferSerializeChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
+ int fd, ReorderBufferChange *change)
+{
+ ReorderBufferDiskChange *ondisk;
+ Size sz = sizeof(ReorderBufferDiskChange);
+
+ ReorderBufferSerializeReserve(rb, sz);
+
+ ondisk = (ReorderBufferDiskChange *) rb->outbuf;
+ memcpy(&ondisk->change, change, sizeof(ReorderBufferChange));
+
+ switch ((ReorderBufferChangeTypeInternal) change->action_internal)
+ {
+ case REORDER_BUFFER_CHANGE_INTERNAL_INSERT:
+ /* fall through */
+ case REORDER_BUFFER_CHANGE_INTERNAL_UPDATE:
+ /* fall through */
+ case REORDER_BUFFER_CHANGE_INTERNAL_DELETE:
+ {
+ char *data;
+ Size oldlen = 0;
+ Size newlen = 0;
+
+ if (change->tp.oldtuple)
+ oldlen = offsetof(ReorderBufferTupleBuf, data)
+ + change->tp.oldtuple->tuple.t_len
+ - offsetof(HeapTupleHeaderData, t_bits);
+
+ if (change->tp.newtuple)
+ newlen = offsetof(ReorderBufferTupleBuf, data)
+ + change->tp.newtuple->tuple.t_len
+ - offsetof(HeapTupleHeaderData, t_bits);
+
+ sz += oldlen;
+ sz += newlen;
+
+ /* make sure we have enough space */
+ ReorderBufferSerializeReserve(rb, sz);
+
+ data = ((char *) rb->outbuf) + sizeof(ReorderBufferDiskChange);
+ /* might have been reallocated above */
+ ondisk = (ReorderBufferDiskChange *) rb->outbuf;
+
+ if (oldlen)
+ {
+ memcpy(data, change->tp.oldtuple, oldlen);
+ data += oldlen;
+ Assert(&change->tp.oldtuple->header == change->tp.oldtuple->tuple.t_data);
+ }
+
+ if (newlen)
+ {
+ memcpy(data, change->tp.newtuple, newlen);
+ data += newlen;
+ Assert(&change->tp.newtuple->header == change->tp.newtuple->tuple.t_data);
+ }
+ break;
+ }
+ case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
+ {
+ char *data;
+
+ sz += sizeof(SnapshotData) +
+ sizeof(TransactionId) * change->snapshot->xcnt +
+ sizeof(TransactionId) * change->snapshot->subxcnt
+ ;
+
+ /* make sure we have enough space */
+ ReorderBufferSerializeReserve(rb, sz);
+ data = ((char *) rb->outbuf) + sizeof(ReorderBufferDiskChange);
+ /* might have been reallocated above */
+ ondisk = (ReorderBufferDiskChange *) rb->outbuf;
+
+ memcpy(data, change->snapshot, sizeof(SnapshotData));
+ data += sizeof(SnapshotData);
+
+ if (change->snapshot->xcnt)
+ {
+ memcpy(data, change->snapshot->xip,
+ sizeof(TransactionId) + change->snapshot->xcnt);
+ data += sizeof(TransactionId) + change->snapshot->xcnt;
+ }
+
+ if (change->snapshot->subxcnt)
+ {
+ memcpy(data, change->snapshot->subxip,
+ sizeof(TransactionId) + change->snapshot->subxcnt);
+ data += sizeof(TransactionId) + change->snapshot->subxcnt;
+ }
+ break;
+ }
+ case REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID:
+ /* ReorderBufferChange contains everything important */
+ break;
+ case REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID:
+ /* ReorderBufferChange contains everything important */
+ break;
+ }
+
+ ondisk->size = sz;
+
+ if (write(fd, rb->outbuf, ondisk->size) != ondisk->size)
+ {
+ CloseTransientFile(fd);
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not write to xid data file \"%u\": %m",
+ txn->xid)));
+ }
+
+ Assert(ondisk->change.action_internal == change->action_internal);
+}
+
+/*
+ * Check whether we should spill data to disk.
+ */
+static void
+ReorderBufferCheckSerializeTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
+{
+ /* FIXME subtxn handling? */
+ if (txn->nentries_mem >= max_memtries)
+ {
+ ReorderBufferSerializeTXN(rb, txn);
+ Assert(txn->nentries_mem == 0);
+ }
+}
+
+/*
+ * Spill data of a large transaction (and its subtransactions) to disk.
+ */
+static void
+ReorderBufferSerializeTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
+{
+ dlist_iter subtxn_i;
+ dlist_mutable_iter change_i;
+ int fd = -1;
+ XLogSegNo curOpenSegNo = 0;
+ Size spilled = 0;
+ char path[MAXPGPATH];
+
+ elog(DEBUG2, "spill %u changes in tx %u to disk",
+ (uint32) txn->nentries_mem, txn->xid);
+
+ /* do the same to all child TXs */
+ dlist_foreach(subtxn_i, &txn->subtxns)
+ {
+ ReorderBufferTXN *subtxn;
+
+ subtxn = dlist_container(ReorderBufferTXN, node, subtxn_i.cur);
+ ReorderBufferSerializeTXN(rb, subtxn);
+ }
+
+ /* serialize changestream */
+ dlist_foreach_modify(change_i, &txn->changes)
+ {
+ ReorderBufferChange *change;
+
+ change = dlist_container(ReorderBufferChange, node, change_i.cur);
+
+ /*
+ * store in segment in which it belongs by start lsn, don't split over
+ * multiple segments tho
+ */
+ if (fd == -1 || XLByteInSeg(change->lsn, curOpenSegNo))
+ {
+ XLogRecPtr recptr;
+
+ if (fd != -1)
+ CloseTransientFile(fd);
+
+ XLByteToSeg(change->lsn, curOpenSegNo);
+ XLogSegNoOffsetToRecPtr(curOpenSegNo, 0, recptr);
+
+ /*
+ * No need to care about TLIs here, only used during a single run,
+ * so each LSN only maps to a specific WAL record.
+ */
+ sprintf(path, "pg_llog/%s/xid-%u-lsn-%X-%X.snap",
+ NameStr(MyLogicalDecodingSlot->name), txn->xid,
+ (uint32) (recptr >> 32), (uint32) recptr);
+
+ /* open segment, create it if necessary */
+ fd = OpenTransientFile(path,
+ O_CREAT | O_WRONLY | O_APPEND | PG_BINARY,
+ S_IRUSR | S_IWUSR);
+
+ if (fd < 0)
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not open reorderbuffer file %s for writing: %m",
+ path)));
+ }
+
+ ReorderBufferSerializeChange(rb, txn, fd, change);
+ dlist_delete(&change->node);
+ ReorderBufferReturnChange(rb, change);
+
+ spilled++;
+ }
+
+ Assert(spilled == txn->nentries_mem);
+ Assert(dlist_is_empty(&txn->changes));
+ txn->nentries_mem = 0;
+
+ if (fd != -1)
+ CloseTransientFile(fd);
+
+ /* issue write barrier */
+ /* serialize main transaction state */
+}
+
+/*
+ * Restore a number of changes spilled to disk back into memory.
+ */
+static Size
+ReorderBufferRestoreChanges(ReorderBuffer *rb, ReorderBufferTXN *txn,
+ int *fd, XLogSegNo *segno)
+{
+ Size restored = 0;
+ XLogSegNo last_segno;
+ dlist_mutable_iter cleanup_iter;
+
+ Assert(txn->first_lsn != InvalidXLogRecPtr);
+ Assert(txn->final_lsn != InvalidXLogRecPtr);
+
+ /* free current entries, so we have memory for more */
+ dlist_foreach_modify(cleanup_iter, &txn->changes)
+ {
+ ReorderBufferChange *cleanup =
+ dlist_container(ReorderBufferChange, node, cleanup_iter.cur);
+
+ dlist_delete(&cleanup->node);
+ ReorderBufferReturnChange(rb, cleanup);
+ }
+ txn->nentries_mem = 0;
+ Assert(dlist_is_empty(&txn->changes));
+
+ XLByteToSeg(txn->final_lsn, last_segno);
+
+ while (restored < max_memtries && *segno <= last_segno)
+ {
+ int readBytes;
+ ReorderBufferDiskChange *ondisk;
+
+ if (*fd == -1)
+ {
+ XLogRecPtr recptr;
+ char path[MAXPGPATH];
+
+ /* first time in */
+ if (*segno == 0)
+ {
+ XLByteToSeg(txn->first_lsn, *segno);
+ }
+
+ Assert(*segno != 0 || dlist_is_empty(&txn->changes));
+ XLogSegNoOffsetToRecPtr(*segno, 0, recptr);
+
+ /*
+ * No need to care about TLIs here, only used during a single run,
+ * so each LSN only maps to a specific WAL record.
+ */
+ sprintf(path, "pg_llog/%s/xid-%u-lsn-%X-%X.snap",
+ NameStr(MyLogicalDecodingSlot->name), txn->xid,
+ (uint32) (recptr >> 32), (uint32) recptr);
+
+ *fd = OpenTransientFile(path, O_RDONLY | PG_BINARY, 0);
+ if (*fd < 0 && errno == ENOENT)
+ {
+ *fd = -1;
+ (*segno)++;
+ continue;
+ }
+ else if (*fd < 0)
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not open reorderbuffer file %s for reading: %m",
+ path)));
+
+ }
+
+ ReorderBufferSerializeReserve(rb, sizeof(ReorderBufferDiskChange));
+
+
+ /*
+ * read the statically sized part of a change which has information
+ * about the total size. If we couldn't read a record, we're at the
+ * end of this file.
+ */
+
+ readBytes = read(*fd, rb->outbuf, sizeof(ReorderBufferDiskChange));
+
+ /* eof */
+ if (readBytes == 0)
+ {
+ CloseTransientFile(*fd);
+ *fd = -1;
+ (*segno)++;
+ continue;
+ }
+ else if (readBytes < 0)
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not read from reorderbuffer spill file: %m")));
+ else if (readBytes != sizeof(ReorderBufferDiskChange))
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("incomplete read from reorderbuffer spill file: read %d instead of %u",
+ readBytes,
+ (uint32) sizeof(ReorderBufferDiskChange))));
+
+ ondisk = (ReorderBufferDiskChange *) rb->outbuf;
+
+ ReorderBufferSerializeReserve(rb,
+ sizeof(ReorderBufferDiskChange) + ondisk->size);
+ ondisk = (ReorderBufferDiskChange *) rb->outbuf;
+
+ readBytes = read(*fd, rb->outbuf + sizeof(ReorderBufferDiskChange),
+ ondisk->size - sizeof(ReorderBufferDiskChange));
+
+ if (readBytes < 0)
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not read from reorderbuffer spill file: %m")));
+ else if (readBytes != ondisk->size - sizeof(ReorderBufferDiskChange))
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("incomplete read from reorderbuffer spill file: read %d instead of %u",
+ readBytes,
+ (uint32) (ondisk->size - sizeof(ReorderBufferDiskChange)))));
+
+ /*
+ * ok, read a full change from disk, now restore it into proper
+ * in-memory format
+ */
+ ReorderBufferRestoreChange(rb, txn, rb->outbuf);
+ restored++;
+ }
+
+ return restored;
+}
+
+/*
+ * Convert change from its on-disk format to in-memory format and queue it onto
+ * the TXN's ->changes list.
+ */
+static void
+ReorderBufferRestoreChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
+ char *data)
+{
+ ReorderBufferDiskChange *ondisk;
+ ReorderBufferChange *change;
+
+ ondisk = (ReorderBufferDiskChange *) data;
+
+ change = ReorderBufferGetChange(rb);
+
+ /* copy static part */
+ memcpy(change, &ondisk->change, sizeof(ReorderBufferChange));
+
+ data += sizeof(ReorderBufferDiskChange);
+
+ /* restore individual stuff */
+ switch ((ReorderBufferChangeTypeInternal) change->action_internal)
+ {
+ case REORDER_BUFFER_CHANGE_INTERNAL_INSERT:
+ /* fall through */
+ case REORDER_BUFFER_CHANGE_INTERNAL_UPDATE:
+ /* fall through */
+ case REORDER_BUFFER_CHANGE_INTERNAL_DELETE:
+ if (change->tp.newtuple)
+ {
+ Size len = offsetof(ReorderBufferTupleBuf, data)
+ +((ReorderBufferTupleBuf *) data)->tuple.t_len
+ - offsetof(HeapTupleHeaderData, t_bits);
+
+ change->tp.newtuple = ReorderBufferGetTupleBuf(rb);
+ memcpy(change->tp.newtuple, data, len);
+ change->tp.newtuple->tuple.t_data = &change->tp.newtuple->header;
+
+ data += len;
+ }
+
+ if (change->tp.oldtuple)
+ {
+ Size len = offsetof(ReorderBufferTupleBuf, data)
+ +((ReorderBufferTupleBuf *) data)->tuple.t_len
+ - offsetof(HeapTupleHeaderData, t_bits);
+
+ change->tp.oldtuple = ReorderBufferGetTupleBuf(rb);
+ memcpy(change->tp.oldtuple, data, len);
+ change->tp.oldtuple->tuple.t_data = &change->tp.oldtuple->header;
+ data += len;
+ }
+ break;
+ case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
+ {
+ Snapshot oldsnap = (Snapshot) data;
+ Size size = sizeof(SnapshotData) +
+ sizeof(TransactionId) * oldsnap->xcnt +
+ sizeof(TransactionId) * (oldsnap->subxcnt + 0)
+ ;
+
+ Assert(change->snapshot != NULL);
+
+ change->snapshot = MemoryContextAllocZero(rb->context, size);
+
+ memcpy(change->snapshot, data, size);
+ change->snapshot->xip = (TransactionId *)
+ (((char *) change->snapshot) + sizeof(SnapshotData));
+ change->snapshot->subxip =
+ change->snapshot->xip + change->snapshot->xcnt + 0;
+ change->snapshot->copied = true;
+ break;
+ }
+ /* nothing needs to be done */
+ case REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID:
+ case REORDER_BUFFER_CHANGE_INTERNAL_TUPLECID:
+ break;
+ }
+
+ dlist_push_tail(&txn->changes, &change->node);
+ txn->nentries_mem++;
+}
+
+/*
+ * Delete all data spilled to disk after we've restarted/crashed. It will be
+ * recreated when the respective slots are reused.
+ */
+void
+ReorderBufferStartup(void)
+{
+ DIR *logical_dir;
+ struct dirent *logical_de;
+
+ DIR *spill_dir;
+ struct dirent *spill_de;
+
+ logical_dir = AllocateDir("pg_llog");
+ while ((logical_de = ReadDir(logical_dir, "pg_llog")) != NULL)
+ {
+ char path[MAXPGPATH];
+
+ if (strcmp(logical_de->d_name, ".") == 0 ||
+ strcmp(logical_de->d_name, "..") == 0)
+ continue;
+
+ /* one of our own directories */
+ if (strcmp(logical_de->d_name, "snapshots") == 0)
+ continue;
+
+ /*
+ * ok, has to be a surviving logical slot, iterate and delete
+ * everythign starting with xid-*
+ */
+ sprintf(path, "pg_llog/%s", logical_de->d_name);
+
+ spill_dir = AllocateDir(path);
+ while ((spill_de = ReadDir(spill_dir, "pg_llog")) != NULL)
+ {
+ if (strcmp(spill_de->d_name, ".") == 0 ||
+ strcmp(spill_de->d_name, "..") == 0)
+ continue;
+
+ if (strncmp(spill_de->d_name, "xid", 3) == 0)
+ {
+ sprintf(path, "pg_llog/%s/%s", logical_de->d_name,
+ spill_de->d_name);
+
+ if (unlink(path) != 0)
+ ereport(PANIC,
+ (errcode_for_file_access(),
+ errmsg("could not remove xid data file \"%s\": %m",
+ path)));
+ }
+ /* XXX: WARN? */
+ }
+ FreeDir(spill_dir);
+ }
+ FreeDir(logical_dir);
+}
+
+/*
+ * toast support
+ */
+
+/*
+ * copied stuff from tuptoaster.c. Perhaps there should be toast_internal.h?
+ */
+#define VARATT_EXTERNAL_GET_POINTER(toast_pointer, attr) \
+do { \
+ varattrib_1b_e *attre = (varattrib_1b_e *) (attr); \
+ Assert(VARATT_IS_EXTERNAL(attre)); \
+ Assert(VARSIZE_EXTERNAL(attre) == sizeof(toast_pointer) + VARHDRSZ_EXTERNAL); \
+ memcpy(&(toast_pointer), VARDATA_EXTERNAL(attre), sizeof(toast_pointer)); \
+} while (0)
+
+#define VARATT_EXTERNAL_IS_COMPRESSED(toast_pointer) \
+ ((toast_pointer).va_extsize < (toast_pointer).va_rawsize - VARHDRSZ)
+
+/*
+ * Initialize per tuple toast reconstruction support.
+ */
+static void
+ReorderBufferToastInitHash(ReorderBuffer *rb, ReorderBufferTXN *txn)
+{
+ HASHCTL hash_ctl;
+
+ Assert(txn->toast_hash == NULL);
+
+ memset(&hash_ctl, 0, sizeof(hash_ctl));
+ hash_ctl.keysize = sizeof(Oid);
+ hash_ctl.entrysize = sizeof(ReorderBufferToastEnt);
+ hash_ctl.hash = tag_hash;
+ hash_ctl.hcxt = rb->context;
+ txn->toast_hash = hash_create("ReorderBufferToastHash", 5, &hash_ctl,
+ HASH_ELEM | HASH_FUNCTION | HASH_CONTEXT);
+}
+
+/*
+ * Per toast-chunk handling for toast reconstruction
+ *
+ * Appends a toast chunk so we can reconstruct it when the tuple "owning" the
+ * toasted Datum comes along.
+ */
+static void
+ReorderBufferToastAppendChunk(ReorderBuffer *rb, ReorderBufferTXN *txn,
+ Relation relation, ReorderBufferChange *change)
+{
+ ReorderBufferToastEnt *ent;
+ bool found;
+ int32 chunksize;
+ bool isnull;
+ Pointer chunk;
+ TupleDesc desc = RelationGetDescr(relation);
+ Oid chunk_id;
+ Oid chunk_seq;
+
+ if (txn->toast_hash == NULL)
+ ReorderBufferToastInitHash(rb, txn);
+
+ Assert(IsToastRelation(relation));
+
+ chunk_id = DatumGetObjectId(fastgetattr(&change->tp.newtuple->tuple, 1, desc, &isnull));
+ Assert(!isnull);
+ chunk_seq = DatumGetInt32(fastgetattr(&change->tp.newtuple->tuple, 2, desc, &isnull));
+ Assert(!isnull);
+
+ ent = (ReorderBufferToastEnt *)
+ hash_search(txn->toast_hash,
+ (void *) &chunk_id,
+ HASH_ENTER,
+ &found);
+
+ if (!found)
+ {
+ Assert(ent->chunk_id == chunk_id);
+ ent->num_chunks = 0;
+ ent->last_chunk_seq = 0;
+ ent->size = 0;
+ ent->reconstructed = NULL;
+ dlist_init(&ent->chunks);
+
+ if (chunk_seq != 0)
+ elog(ERROR, "got sequence entry %d for toast chunk %u instead of seq 0",
+ chunk_seq, chunk_id);
+ }
+ else if (found && chunk_seq != ent->last_chunk_seq + 1)
+ elog(ERROR, "got sequence entry %d for toast chunk %u instead of seq %d",
+ chunk_seq, chunk_id, ent->last_chunk_seq + 1);
+
+ chunk = DatumGetPointer(fastgetattr(&change->tp.newtuple->tuple, 3, desc, &isnull));
+ Assert(!isnull);
+
+ /* calculate size so we can allocate the right size at once later */
+ if (!VARATT_IS_EXTENDED(chunk))
+ chunksize = VARSIZE(chunk) - VARHDRSZ;
+ else if (VARATT_IS_SHORT(chunk))
+ /* could happen due to heap_form_tuple doing its thing */
+ chunksize = VARSIZE_SHORT(chunk) - VARHDRSZ_SHORT;
+ else
+ elog(ERROR, "unexpected type of toast chunk");
+
+ ent->size += chunksize;
+ ent->last_chunk_seq = chunk_seq;
+ ent->num_chunks++;
+ dlist_push_tail(&ent->chunks, &change->node);
+}
+
+/*
+ * Rejigger change->newtuple to point to in-memory toast tuples instead to
+ * on-disk toast tuples that may not longer exist (think DROP TABLE or VACUUM).
+ *
+ * We cannot replace unchanged toast tuples though, so those will still point
+ * to on-disk toast data.
+ */
+static void
+ReorderBufferToastReplace(ReorderBuffer *rb, ReorderBufferTXN *txn,
+ Relation relation, ReorderBufferChange *change)
+{
+ TupleDesc desc;
+ int natt;
+ Datum *attrs;
+ bool *isnull;
+ bool *free;
+ HeapTuple newtup;
+ Relation toast_rel;
+ TupleDesc toast_desc;
+ MemoryContext oldcontext;
+
+ /* no toast tuples changed */
+ if (txn->toast_hash == NULL)
+ return;
+
+ oldcontext = MemoryContextSwitchTo(rb->context);
+
+ /* we should only have toast tuples in an INSERT or UPDATE */
+ Assert(change->tp.newtuple);
+
+ desc = RelationGetDescr(relation);
+
+ toast_rel = RelationIdGetRelation(relation->rd_rel->reltoastrelid);
+ toast_desc = RelationGetDescr(toast_rel);
+
+ /* should we allocate from stack instead? */
+ attrs = palloc0(sizeof(Datum) * desc->natts);
+ isnull = palloc0(sizeof(bool) * desc->natts);
+ free = palloc0(sizeof(bool) * desc->natts);
+
+ heap_deform_tuple(&change->tp.newtuple->tuple, desc,
+ attrs, isnull);
+
+ for (natt = 0; natt < desc->natts; natt++)
+ {
+ Form_pg_attribute attr = desc->attrs[natt];
+ ReorderBufferToastEnt *ent;
+ struct varlena *varlena;
+
+ /* va_rawsize is the size of the original datum -- including header */
+ struct varatt_external toast_pointer;
+ struct varatt_indirect redirect_pointer;
+ struct varlena *new_datum = NULL;
+ struct varlena *reconstructed;
+ dlist_iter it;
+ Size data_done = 0;
+
+ /* system columns aren't toasted */
+ if (attr->attnum < 0)
+ continue;
+
+ if (attr->attisdropped)
+ continue;
+
+ /* not a varlena datatype */
+ if (attr->attlen != -1)
+ continue;
+
+ /* no data */
+ if (isnull[natt])
+ continue;
+
+ /* ok, we know we have a toast datum */
+ varlena = (struct varlena *) DatumGetPointer(attrs[natt]);
+
+ /* no need to do anything if the tuple isn't external */
+ if (!VARATT_IS_EXTERNAL(varlena))
+ continue;
+
+ VARATT_EXTERNAL_GET_POINTER(toast_pointer, varlena);
+
+ /*
+ * check whether the toast tuple changed, replace if so.
+ */
+ ent = (ReorderBufferToastEnt *)
+ hash_search(txn->toast_hash,
+ (void *) &toast_pointer.va_valueid,
+ HASH_FIND,
+ NULL);
+ if (ent == NULL)
+ continue;
+
+ new_datum =
+ (struct varlena *) palloc0(INDIRECT_POINTER_SIZE);
+
+ free[natt] = true;
+
+ reconstructed = palloc0(toast_pointer.va_rawsize);
+
+ ent->reconstructed = reconstructed;
+
+ /* stitch toast tuple back together from its parts */
+ dlist_foreach(it, &ent->chunks)
+ {
+ bool isnull;
+ ReorderBufferTupleBuf *tup =
+ dlist_container(ReorderBufferChange, node, it.cur)->tp.newtuple;
+ Pointer chunk =
+ DatumGetPointer(fastgetattr(&tup->tuple, 3, toast_desc, &isnull));
+
+ Assert(!isnull);
+ Assert(!VARATT_IS_EXTERNAL(chunk));
+ Assert(!VARATT_IS_SHORT(chunk));
+
+ memcpy(VARDATA(reconstructed) + data_done,
+ VARDATA(chunk),
+ VARSIZE(chunk) - VARHDRSZ);
+ data_done += VARSIZE(chunk) - VARHDRSZ;
+ }
+ Assert(data_done == toast_pointer.va_extsize);
+
+ /* make sure its marked as compressed or not */
+ if (VARATT_EXTERNAL_IS_COMPRESSED(toast_pointer))
+ SET_VARSIZE_COMPRESSED(reconstructed, data_done + VARHDRSZ);
+ else
+ SET_VARSIZE(reconstructed, data_done + VARHDRSZ);
+
+ memset(&redirect_pointer, 0, sizeof(redirect_pointer));
+ redirect_pointer.pointer = reconstructed;
+
+ SET_VARTAG_EXTERNAL(new_datum, VARTAG_INDIRECT);
+ memcpy(VARDATA_EXTERNAL(new_datum), &redirect_pointer,
+ sizeof(redirect_pointer));
+
+ attrs[natt] = PointerGetDatum(new_datum);
+ }
+
+ /*
+ * Build tuple in separate memory & copy tuple back into the tuplebuf
+ * passed to the output plugin. We can't directly heap_fill_tuple() into
+ * the tuplebuf because attrs[] will point back into the current content.
+ */
+ newtup = heap_form_tuple(desc, attrs, isnull);
+ Assert(change->tp.newtuple->tuple.t_len <= MaxHeapTupleSize);
+ Assert(&change->tp.newtuple->header == change->tp.newtuple->tuple.t_data);
+
+ memcpy(change->tp.newtuple->tuple.t_data,
+ newtup->t_data,
+ newtup->t_len);
+ change->tp.newtuple->tuple.t_len = newtup->t_len;
+
+ /*
+ * free resources we won't further need, more persistent stuff will be
+ * free'd in ReorderBufferToastReset().
+ */
+ RelationClose(toast_rel);
+ pfree(newtup);
+ for (natt = 0; natt < desc->natts; natt++)
+ {
+ if (free[natt])
+ pfree(DatumGetPointer(attrs[natt]));
+ }
+ pfree(attrs);
+ pfree(free);
+ pfree(isnull);
+
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Free all resources allocated for toast reconstruction.
+ */
+static void
+ReorderBufferToastReset(ReorderBuffer *rb, ReorderBufferTXN *txn)
+{
+ HASH_SEQ_STATUS hstat;
+ ReorderBufferToastEnt *ent;
+
+ if (txn->toast_hash == NULL)
+ return;
+
+ /* sequentially walk over the hash and free everything */
+ hash_seq_init(&hstat, txn->toast_hash);
+ while ((ent = (ReorderBufferToastEnt *) hash_seq_search(&hstat)) != NULL)
+ {
+ dlist_mutable_iter it;
+
+ if (ent->reconstructed != NULL)
+ pfree(ent->reconstructed);
+
+ dlist_foreach_modify(it, &ent->chunks)
+ {
+ ReorderBufferChange *change =
+ dlist_container(ReorderBufferChange, node, it.cur);
+
+ dlist_delete(&change->node);
+ ReorderBufferReturnChange(rb, change);
+ }
+ }
+
+ hash_destroy(txn->toast_hash);
+ txn->toast_hash = NULL;
+}
+
+
+/*
+ * Visibility support routines
+ */
+
+/*-------------------------------------------------------------------------
+ * Lookup actual cmin/cmax values when using decoding snapshot. We can't
+ * always rely on stored cmin/cmax values because of two scenarios:
+ *
+ * * A tuple got changed multiple times during a single transaction and thus
+ * has got a combocid. Combocid's are only valid for the duration of a
+ * single transaction.
+ * * A tuple with a cmin but no cmax (and thus no combocid) got
+ * deleted/updated in another transaction than the one which created it
+ * which we are looking at right now. As only one of cmin, cmax or combocid
+ * is actually stored in the heap we don't have access to the the value we
+ * need anymore.
+ *
+ * To resolve those problems we have a per-transaction hash of (cmin,
+ * cmax) tuples keyed by (relfilenode, ctid) which contains the actual
+ * (cmin, cmax) values. That also takes care of combocids by simply
+ * not caring about them at all. As we have the real cmin/cmax values
+ * combocids aren't interesting.
+ *
+ * As we only care about catalog tuples here the overhead of this
+ * hashtable should be acceptable.
+ *
+ * Heap rewrites complicate this a bit, check rewriteheap.c for
+ * details.
+ * -------------------------------------------------------------------------
+ */
+
+
+#include "storage/fd.h"
+
+/* struct for qsort()ing mapping files by lsn somewhat efficiently */
+typedef struct RewriteMappingFile
+{
+ XLogRecPtr lsn;
+ char fname[MAXPGPATH];
+} RewriteMappingFile;
+
+#if NOT_USED
+static void
+DisplayMapping(HTAB *tuplecid_data)
+{
+ HASH_SEQ_STATUS hstat;
+ ReorderBufferTupleCidEnt *ent;
+
+ hash_seq_init(&hstat, tuplecid_data);
+ while ((ent = (ReorderBufferTupleCidEnt *) hash_seq_search(&hstat)) != NULL)
+ {
+ elog(DEBUG3, "mapping: node: %u/%u/%u tid: %u/%u cmin: %u, cmax: %u",
+ ent->key.relnode.dbNode,
+ ent->key.relnode.spcNode,
+ ent->key.relnode.relNode,
+ BlockIdGetBlockNumber(&ent->key.tid.ip_blkid),
+ ent->key.tid.ip_posid,
+ ent->cmin,
+ ent->cmax
+ );
+ }
+}
+#endif
+
+/*
+ * Apply a single mapping file to tuplecid_data.
+ *
+ * The mapping file has to have been verified to be a) committed b) for our
+ * transaction c) applied in LSN order.
+ */
+static void
+ApplyLogicalMappingFile(HTAB *tuplecid_data, Oid relid, const char *fname)
+{
+ char path[MAXPGPATH];
+ int fd;
+ int readBytes;
+ LogicalRewriteMappingData map;
+
+ sprintf(path, "pg_llog/mappings/%s", fname);
+ fd = OpenTransientFile(path, O_RDONLY | PG_BINARY, 0);
+ if (fd < 0)
+ ereport(ERROR, (errmsg("could not open mappingfile %s: %m", path)));
+
+ while (true)
+ {
+ ReorderBufferTupleCidKey key;
+ ReorderBufferTupleCidEnt *ent;
+ ReorderBufferTupleCidEnt *new_ent;
+ bool found;
+
+ /* be careful about padding */
+ memset(&key, 0, sizeof(ReorderBufferTupleCidKey));
+
+ /* read all mappings till the end of the file */
+ readBytes = read(fd, &map, sizeof(LogicalRewriteMappingData));
+
+ if (readBytes < 0)
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not read logical rewrite mapping \"%s\": %m",
+ path)));
+ else if (readBytes == 0) /* EOF */
+ break;
+ else if (readBytes != sizeof(LogicalRewriteMappingData))
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not read logical rewrite mapping %s, read %d instead of %d",
+ path, readBytes,
+ (int32) sizeof(LogicalRewriteMappingData))));
+
+ key.relnode = map.old_node;
+ ItemPointerCopy(&map.old_tid,
+ &key.tid);
+
+
+ ent = (ReorderBufferTupleCidEnt *)
+ hash_search(tuplecid_data,
+ (void *) &key,
+ HASH_FIND,
+ NULL);
+
+ /* no existing mapping, no need to update */
+ if (!ent)
+ continue;
+
+ key.relnode = map.new_node;
+ ItemPointerCopy(&map.new_tid,
+ &key.tid);
+
+ new_ent = (ReorderBufferTupleCidEnt *)
+ hash_search(tuplecid_data,
+ (void *) &key,
+ HASH_ENTER,
+ &found);
+
+ if (found)
+ {
+ /*
+ * Make sure the existing mapping makes sense. We sometime update
+ * old records that did not yet have a cmax (e.g. pg_class' own
+ * entry while rewriting it) during rewrites, so allow that.
+ */
+ Assert(ent->cmin == InvalidCommandId || ent->cmin == new_ent->cmin);
+ Assert(ent->cmax == InvalidCommandId || ent->cmax == new_ent->cmax);
+ }
+ else
+ {
+ /* update mapping */
+ new_ent->cmin = ent->cmin;
+ new_ent->cmax = ent->cmax;
+ new_ent->combocid = ent->combocid;
+ }
+ }
+}
+
+
+/*
+ * check whether the transaciont id 'xid' in in the pre-sorted array 'xip'.
+ */
+static bool
+TransactionIdInArray(TransactionId xid, TransactionId *xip, Size num)
+{
+ return bsearch(&xid, xip, num,
+ sizeof(TransactionId), xidComparator) != NULL;
+}
+
+/*
+ * qsort() comparator for sorting RewriteMappingFiles in LSN order.
+ */
+static int
+file_sort_by_lsn(const void *a_p, const void *b_p)
+{
+ RewriteMappingFile *a = *(RewriteMappingFile **)a_p;
+ RewriteMappingFile *b = *(RewriteMappingFile **)b_p;
+
+ if (a->lsn < b->lsn)
+ return -1;
+ else if (a->lsn > b->lsn)
+ return 1;
+ return 0;
+}
+
+/*
+ * Apply any existing logical remapping files if there are any targeted at our
+ * transaction for relid.
+ */
+static void
+UpdateLogicalMappings(HTAB *tuplecid_data, Oid relid, Snapshot snapshot)
+{
+ DIR *mapping_dir;
+ struct dirent *mapping_de;
+ List *files = NIL;
+ ListCell *file;
+ RewriteMappingFile **files_a;
+ size_t off;
+ Oid dboid = IsSharedRelation(relid) ? InvalidOid : MyDatabaseId;
+
+ mapping_dir = AllocateDir("pg_llog/mappings");
+ while ((mapping_de = ReadDir(mapping_dir, "pg_llog/mappings")) != NULL)
+ {
+ Oid f_dboid;
+ Oid f_relid;
+ TransactionId f_mapped_xid;
+ TransactionId f_create_xid;
+ XLogRecPtr f_lsn;
+ RewriteMappingFile *f;
+
+ if (strcmp(mapping_de->d_name, ".") == 0 ||
+ strcmp(mapping_de->d_name, "..") == 0)
+ continue;
+
+ /* XXX: should we warn about such files? */
+ if (strncmp(mapping_de->d_name, "map-", 4) != 0)
+ continue;
+
+ if (sscanf(mapping_de->d_name, LOGICAL_REWRITE_FORMAT,
+ &f_dboid, &f_relid, &f_lsn,
+ &f_mapped_xid, &f_create_xid) != 5)
+ elog(ERROR, "could not parse fname %s", mapping_de->d_name);
+
+ /* mapping for another database */
+ if (f_dboid != dboid)
+ continue;
+
+ /* mapping for another relation */
+ if (f_relid != relid)
+ continue;
+
+ /* did the creating transaction abort? */
+ if (!TransactionIdDidCommit(f_create_xid))
+ continue;
+
+ /* not for our transaction */
+ if (!TransactionIdInArray(f_mapped_xid, snapshot->subxip, snapshot->subxcnt))
+ continue;
+
+ /* ok, relevant, queue for apply */
+ f = palloc(sizeof(RewriteMappingFile));
+ f->lsn = f_lsn;
+ strcpy(f->fname, mapping_de->d_name);
+ files = lappend(files, f);
+ }
+ FreeDir(mapping_dir);
+
+ /* build array we can easily sort */
+ files_a = palloc(list_length(files) * sizeof(RewriteMappingFile *));
+ off = 0;
+ foreach(file, files)
+ {
+ files_a[off++] = lfirst(file);
+ }
+
+ /* sort files so we apply them in LSN order */
+ qsort(files_a, list_length(files), sizeof(RewriteMappingFile *),
+ file_sort_by_lsn);
+
+ for(off = 0; off < list_length(files); off++)
+ {
+ RewriteMappingFile *f = files_a[off];
+ elog(DEBUG1, "applying mapping: %s in %u", f->fname,
+ snapshot->subxip[0]);
+ ApplyLogicalMappingFile(tuplecid_data, relid, f->fname);
+ pfree(f);
+ }
+}
+
+extern bool
+ResolveCminCmaxDuringDecoding(HTAB *tuplecid_data,
+ Snapshot snapshot,
+ HeapTuple htup, Buffer buffer,
+ CommandId *cmin, CommandId *cmax)
+{
+ ReorderBufferTupleCidKey key;
+ ReorderBufferTupleCidEnt *ent;
+ ForkNumber forkno;
+ BlockNumber blockno;
+ bool updated_mapping = false;
+
+ /* be careful about padding */
+ memset(&key, 0, sizeof(key));
+
+ Assert(!BufferIsLocal(buffer));
+
+ /*
+ * get relfilenode from the buffer, no convenient way to access it other
+ * than that.
+ */
+ BufferGetTag(buffer, &key.relnode, &forkno, &blockno);
+
+ /* tuples can only be in the main fork */
+ Assert(forkno == MAIN_FORKNUM);
+ Assert(blockno == ItemPointerGetBlockNumber(&htup->t_self));
+
+ ItemPointerCopy(&htup->t_self,
+ &key.tid);
+
+restart:
+ ent = (ReorderBufferTupleCidEnt *)
+ hash_search(tuplecid_data,
+ (void *) &key,
+ HASH_FIND,
+ NULL);
+
+ /*
+ * failed to find a mapping, check whether the table was rewritten and
+ * apply mapping if so, but only do that once - there can be no new
+ * mappings while we are in here since we have to hold a lock.
+ */
+ if (ent == NULL && !updated_mapping)
+ {
+ UpdateLogicalMappings(tuplecid_data, htup->t_tableOid, snapshot);
+ /* now check but don't update for a mapping again */
+ updated_mapping = true;
+ goto restart;
+ }
+ else if (ent == NULL)
+ return false;
+
+ if (cmin)
+ *cmin = ent->cmin;
+ if (cmax)
+ *cmax = ent->cmax;
+ return true;
+}
--- /dev/null
+/*-------------------------------------------------------------------------
+ *
+ * snapbuild.c
+ *
+ * Support for building snapshots based on the contents of the WAL which
+ * then can be used to decode contents of WAL written under a corresponding
+ * snapshot.
+ *
+ * NOTES:
+ *
+ * We build snapshots which can *only* be used to read catalog contents by
+ * reading and interpreting the WAL stream. The aim is to build a snapshot
+ * that behaves the same as a freshly taken MVCC snapshot would have at the
+ * time the XLogRecord was generated.
+ *
+ * To build the snapshots we reuse the infrastructure built for Hot
+ * Standby. The in-memory snapshots we build look different than HS' because
+ * we have different needs. To successfully decode data from the WAL we only
+ * need to access catalog tables and (sys|rel|cat)cache, not the actual user
+ * tables since the data we decode is contained in the WAL records. Also, our
+ * snapshots need to be different in comparison to normal MVCC ones because in
+ * contrast to those we cannot fully rely on the clog and pg_subtrans for
+ * information about committed transactions because they might commit in the
+ * future from the POV of the wal entry we're currently decoding.
+ *
+ * As the percentage of transactions modifying the catalog normally is fairly
+ * small in comparisons to ones only manipulating user data, we keep track of
+ * the committed catalog modifying ones inside (xmin, xmax) instead of keeping
+ * track of all running transactions like its done in a normal snapshot. Note
+ * that we're generally only looking at transactions that have acquired an
+ * xid. That is we keep a list of transactions between snapshot->(xmin, xmax)
+ * that we consider committed, everything else is considered aborted/in
+ * progress. That also allows us not to care about subtransactions before they
+ * have committed which means this modules, in contrast to HS, doesn't have to
+ * care about suboverflowed subtransactions and similar.
+ *
+ * One complexity of doing this is that to e.g. handle mixed DDL/DML
+ * transactions we need Snapshots that see intermediate versions of the
+ * catalog in a transaction. During normal operation this is achieved by using
+ * CommandIds/cmin/cmax. The problem with that however is that for space
+ * efficiency reasons only one value of that is stored
+ * (c.f. combocid.c). Since ComboCids are only available in memory we log
+ * additional information which allows us to get the original (cmin, cmax)
+ * pair during visibility checks. Check the reorderbuffer.c's comment above
+ * ResolveCminCmaxDuringDecoding() for details.
+ *
+ * To facilitate all this we need our own visibility routine, as the normal
+ * ones are optimized for different usecases. To make sure no unexpected
+ * database access bypassing our special snapshot is possible - which would
+ * possibly load invalid data into caches - we temporarily overload the
+ * .satisfies methods of the usual snapshots while changeset extraction.
+ *
+ * To replace the normal catalog snapshots with decoding ones use the
+ * SetupDecodingSnapshots and RevertFromDecodingSnapshots functions.
+ *
+ * Copyright (c) 2012-2013, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ * src/backend/replication/snapbuild.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include <sys/stat.h>
+#include <sys/types.h>
+#include <unistd.h>
+
+#include "miscadmin.h"
+
+#include "access/heapam_xlog.h"
+#include "access/transam.h"
+#include "access/xact.h"
+
+#include "replication/logical.h"
+#include "replication/reorderbuffer.h"
+#include "replication/snapbuild.h"
+
+#include "utils/builtins.h"
+#include "utils/memutils.h"
+#include "utils/snapshot.h"
+#include "utils/snapmgr.h"
+#include "utils/tqual.h"
+
+#include "storage/block.h" /* debugging output */
+#include "storage/fd.h"
+#include "storage/lmgr.h"
+#include "storage/proc.h"
+#include "storage/procarray.h"
+#include "storage/standby.h"
+
+struct SnapBuild
+{
+ /* how far are we along building our first full snapshot */
+ SnapBuildState state;
+
+ /* private memory context used to allocate memory for this module. */
+ MemoryContext context;
+
+ /* all transactions < than this have committed/aborted */
+ TransactionId xmin;
+
+ /* all transactions >= than this are uncommitted */
+ TransactionId xmax;
+
+ /*
+ * Don't replay commits from an LSN <= this LSN. This can be set
+ * externally but it will also be advanced (never retreat) from within
+ * snapbuild.c.
+ */
+ XLogRecPtr transactions_after;
+
+ /*
+ * Don't start decoding WAL until the "xl_running_xacts" information
+ * indicates there are no running xids with a xid smaller than this.
+ */
+ TransactionId initial_xmin_horizon;
+
+ /*
+ * Snapshot thats valid to see all currently committed transactions that
+ * see catalog modifications.
+ */
+ Snapshot snapshot;
+
+ /*
+ * LSN of the last location we are sure a snapshot has been serialized to.
+ */
+ XLogRecPtr last_serialized_snapshot;
+
+ ReorderBuffer *reorder;
+
+ /*
+ * Information about initially running transactions
+ *
+ * When we start building a snapshot there already may be transactions in
+ * progress. Those are stored in running.xip. We don't have enough
+ * information about those to decode their contents, so until they are
+ * finished (xcnt=0) we cannot switch to a CONSISTENT state.
+ */
+ struct
+ {
+ /*
+ * As long as running.xcnt all XIDs < running.xmin and > running.xmax
+ * have to be checked whether they still are running.
+ */
+ TransactionId xmin;
+ TransactionId xmax;
+
+ size_t xcnt; /* number of used xip entries */
+ size_t xcnt_space; /* allocated size of xip */
+ TransactionId *xip; /* running xacts array, xidComparator-sorted */
+ } running;
+
+ /*
+ * Array of transactions which could have catalog changes that committed
+ * between xmin and xmax
+ */
+ struct
+ {
+ /* number of committed transactions */
+ size_t xcnt;
+
+ /* available space for committed transactions */
+ size_t xcnt_space;
+
+ /*
+ * Until we reach a CONSISTENT state, we record commits of all
+ * transactions, not just the catalog changing ones. Record when that
+ * changes so we know we cannot export a snapshot safely anymore.
+ */
+ bool includes_all_transactions;
+
+ /*
+ * Array of committed transactions that have modified the catalog.
+ *
+ * As this array is frequently modified we do *not* keep it in
+ * xidComparator order. Instead we sort the array when building &
+ * distributing a snapshot.
+ *
+ * XXX: That doesn't seem to be good reasoning anymore. Everytime we
+ * add something here after becoming consistent will also require
+ * distributing a snapshot. Storing them sorted would potentially make
+ * it easier to purge as well (but more complicated wrt wraparound?).
+ */
+ TransactionId *xip;
+ } committed;
+};
+
+/*
+ * Starting a transaction -- which we need to do while exporting a snapshot --
+ * removes knowledge about the previously used resowner, so we save it here.
+ */
+ResourceOwner SavedResourceOwnerDuringExport = NULL;
+bool ExportInProgress = false;
+
+/* transaction state manipulation functions */
+static void SnapBuildEndTxn(SnapBuild *builder, XLogRecPtr lsn, TransactionId xid);
+
+/* ->running manipulation */
+static bool SnapBuildTxnIsRunning(SnapBuild *builder, TransactionId xid);
+
+/* ->committed manipulation */
+static void SnapBuildPurgeCommittedTxn(SnapBuild *builder);
+
+/* snapshot building/manipulation/distribution functions */
+static Snapshot SnapBuildBuildSnapshot(SnapBuild *builder, TransactionId xid);
+
+static void SnapBuildFreeSnapshot(Snapshot snap);
+
+static void SnapBuildSnapIncRefcount(Snapshot snap);
+
+static void SnapBuildDistributeNewCatalogSnapshot(SnapBuild *builder, XLogRecPtr lsn);
+
+/* xlog reading helper functions for SnapBuildProcessRecord */
+static bool SnapBuildFindSnapshot(SnapBuild *builder, XLogRecPtr lsn, xl_running_xacts *running);
+
+/* serialization functions */
+static void SnapBuildSerialize(SnapBuild *builder, XLogRecPtr lsn);
+static bool SnapBuildRestore(SnapBuild *builder, XLogRecPtr lsn);
+
+
+/*
+ * Allocate a new snapshot builder.
+ */
+SnapBuild *
+AllocateSnapshotBuilder(ReorderBuffer *reorder,
+ TransactionId xmin_horizon,
+ XLogRecPtr start_lsn)
+{
+ MemoryContext context;
+ MemoryContext oldcontext;
+ SnapBuild *builder;
+
+ context = AllocSetContextCreate(TopMemoryContext,
+ "snapshot builder context",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+ oldcontext = MemoryContextSwitchTo(context);
+
+ builder = palloc0(sizeof(SnapBuild));
+
+ builder->state = SNAPBUILD_START;
+ builder->context = context;
+ builder->reorder = reorder;
+ /* Other struct members initialized by zeroing via palloc0 above */
+
+ builder->committed.xcnt = 0;
+ builder->committed.xcnt_space = 128; /* arbitrary number */
+ builder->committed.xip =
+ palloc0(builder->committed.xcnt_space * sizeof(TransactionId));
+ builder->committed.includes_all_transactions = true;
+ builder->committed.xip =
+ palloc0(builder->committed.xcnt_space * sizeof(TransactionId));
+ builder->initial_xmin_horizon = xmin_horizon;
+ builder->transactions_after = start_lsn;
+
+ MemoryContextSwitchTo(oldcontext);
+
+ return builder;
+}
+
+/*
+ * Free a snapshot builder.
+ */
+void
+FreeSnapshotBuilder(SnapBuild *builder)
+{
+ MemoryContext context = builder->context;
+
+ if (builder->snapshot != NULL)
+ SnapBuildFreeSnapshot(builder->snapshot);
+
+ if (builder->running.xip != NULL)
+ pfree(builder->running.xip);
+
+ if (builder->committed.xip != NULL)
+ pfree(builder->committed.xip);
+
+ pfree(builder);
+
+ MemoryContextDelete(context);
+}
+
+/*
+ * Free an unreferenced snapshot that has previously been built by us.
+ */
+static void
+SnapBuildFreeSnapshot(Snapshot snap)
+{
+ /* make sure we don't get passed an external snapshot */
+ Assert(snap->satisfies == HeapTupleSatisfiesMVCCDuringDecoding);
+
+ /* make sure nobody modified our snapshot */
+ Assert(snap->curcid == FirstCommandId);
+ Assert(!snap->suboverflowed);
+ Assert(!snap->takenDuringRecovery);
+ Assert(!snap->regd_count);
+
+ /* slightly more likely, so it's checked even without c-asserts */
+ if (snap->copied)
+ elog(ERROR, "can't free a copied snapshot");
+
+ if (snap->active_count)
+ elog(ERROR, "can't free an active snapshot");
+
+ pfree(snap);
+}
+
+/*
+ * In which state of snapshot building ar we?
+ */
+SnapBuildState
+SnapBuildCurrentState(SnapBuild *builder)
+{
+ return builder->state;
+}
+
+/*
+ * Should the contents of transaction ending at 'ptr' be decoded?
+ */
+bool
+SnapBuildXactNeedsSkip(SnapBuild *builder, XLogRecPtr ptr)
+{
+ return ptr <= builder->transactions_after;
+}
+
+/*
+ * Increase refcount of a snapshot.
+ *
+ * This is used when handing out a snapshot to some external resource or when
+ * adding a Snapshot as builder->snapshot.
+ */
+static void
+SnapBuildSnapIncRefcount(Snapshot snap)
+{
+ snap->active_count++;
+}
+
+/*
+ * Decrease refcount of a snapshot and free if the refcount reaches zero.
+ *
+ * Externally visible so external resources that have been handed an IncRef'ed
+ * Snapshot can free it easily.
+ */
+void
+SnapBuildSnapDecRefcount(Snapshot snap)
+{
+ /* make sure we don't get passed an external snapshot */
+ Assert(snap->satisfies == HeapTupleSatisfiesMVCCDuringDecoding);
+
+ /* make sure nobody modified our snapshot */
+ Assert(snap->curcid == FirstCommandId);
+ Assert(!snap->suboverflowed);
+ Assert(!snap->takenDuringRecovery);
+ Assert(!snap->regd_count);
+
+ Assert(snap->active_count);
+
+ /* slightly more likely, so its checked even without casserts */
+ if (snap->copied)
+ elog(ERROR, "can't free a copied snapshot");
+
+ snap->active_count--;
+ if (!snap->active_count)
+ SnapBuildFreeSnapshot(snap);
+}
+
+/*
+ * Build a new snapshot, based on currently committed catalog-modifying
+ * transactions.
+ *
+ * In-progress transactions with catalog access are *not* allowed to modify
+ * these snapshots; they have to copy them and fill in appropriate ->curcid
+ * and ->subxip/subxcnt values.
+ */
+static Snapshot
+SnapBuildBuildSnapshot(SnapBuild *builder, TransactionId xid)
+{
+ Snapshot snapshot;
+ Size ssize;
+
+ Assert(builder->state >= SNAPBUILD_FULL_SNAPSHOT);
+
+ ssize = sizeof(SnapshotData)
+ + sizeof(TransactionId) * builder->committed.xcnt
+ + sizeof(TransactionId) * 1 /* toplevel xid */ ;
+
+ snapshot = MemoryContextAllocZero(builder->context, ssize);
+
+ snapshot->satisfies = HeapTupleSatisfiesMVCCDuringDecoding;
+
+ /*
+ * We misuse the original meaning of SnapshotData's xip and subxip fields
+ * to make the more fitting for our needs.
+ *
+ * In the 'xip' array we store transactions that have to be treated as
+ * committed. Since we will only ever look at tuples from transactions
+ * that have modified the catalog its more efficient to store those few
+ * that exist between xmin and xmax (frequently there are none).
+ *
+ * Snapshots that are used in transactions that have modified the catalog
+ * also use the 'subxip' array to store their toplevel xid and all the
+ * subtransaction xids so we can recognize when we need to treat rows as
+ * visible that are not in xip but still need to be visible. Subxip only
+ * gets filled when the transaction is copied into the context of a
+ * catalog modifying transaction since we otherwise share a snapshot
+ * between transactions. As long as a txn hasn't modified the catalog it
+ * doesn't need to treat any uncommitted rows as visible, so there is no
+ * need for those xids.
+ *
+ * Both arrays are qsort'ed so that we can use bsearch() on them.
+ *
+ * XXX: Do we want extra fields instead of misusing existing ones instead?
+ */
+ Assert(TransactionIdIsNormal(builder->xmin));
+ Assert(TransactionIdIsNormal(builder->xmax));
+
+ snapshot->xmin = builder->xmin;
+ snapshot->xmax = builder->xmax;
+
+ /* store all transactions to be treated as committed by this snapshot */
+ snapshot->xip =
+ (TransactionId *) ((char *) snapshot + sizeof(SnapshotData));
+ snapshot->xcnt = builder->committed.xcnt;
+ memcpy(snapshot->xip,
+ builder->committed.xip,
+ builder->committed.xcnt * sizeof(TransactionId));
+
+ /* sort so we can bsearch() */
+ qsort(snapshot->xip, snapshot->xcnt, sizeof(TransactionId), xidComparator);
+
+ /*
+ * Initially, subxip is empty, i.e. it's a snapshot to be used by
+ * transactions that don't modify the catalog. Will be filled by
+ * ReorderBufferCopySnap() if necessary.
+ */
+ snapshot->subxcnt = 0;
+ snapshot->subxip = NULL;
+
+ snapshot->suboverflowed = false;
+ snapshot->takenDuringRecovery = false;
+ snapshot->copied = false;
+ snapshot->curcid = FirstCommandId;
+ snapshot->active_count = 0;
+ snapshot->regd_count = 0;
+
+ return snapshot;
+}
+
+/*
+ * Export a snapshot so it can be set in another session with SET TRANSACTION
+ * SNAPSHOT.
+ *
+ * For that we need to start a transaction in the current backend as the
+ * importing side checks whether the source transaction is still open to make
+ * sure the xmin horizon hasn't advanced since then.
+ *
+ * After that we convert a locally built snapshot into the normal variant
+ * understood by HeapTupleSatisfiesMVCC et al.
+ */
+const char *
+SnapBuildExportSnapshot(SnapBuild *builder)
+{
+ Snapshot snap;
+ char *snapname;
+ TransactionId xid;
+ TransactionId *newxip;
+ int newxcnt = 0;
+
+ if (builder->state != SNAPBUILD_CONSISTENT)
+ elog(ERROR, "cannot export a snapshot before reaching a consistent state");
+
+ if (!builder->committed.includes_all_transactions)
+ elog(ERROR, "cannot export a snapshot, not all transactions are monitored anymore");
+
+ /* so we don't overwrite the existing value */
+ if (TransactionIdIsValid(MyPgXact->xmin))
+ elog(ERROR, "cannot export a snapshot when MyPgXact->xmin already is valid");
+
+ if (IsTransactionOrTransactionBlock())
+ elog(ERROR, "cannot export a snapshot from within a transaction");
+
+ if (SavedResourceOwnerDuringExport)
+ elog(ERROR, "can only export one snapshot at a time");
+
+ SavedResourceOwnerDuringExport = CurrentResourceOwner;
+ ExportInProgress = true;
+
+ StartTransactionCommand();
+
+ Assert(!FirstSnapshotSet);
+
+ /* There doesn't seem to a nice API to set these */
+ XactIsoLevel = XACT_REPEATABLE_READ;
+ XactReadOnly = true;
+
+ snap = SnapBuildBuildSnapshot(builder, GetTopTransactionId());
+
+ /*
+ * We know that snap->xmin is alive, enforced by the logical xmin
+ * mechanism. Due to that we can do this without locks, we're only
+ * changing our own value.
+ */
+ MyPgXact->xmin = snap->xmin;
+
+ /* allocate in transaction context */
+ newxip = (TransactionId *)
+ palloc(sizeof(TransactionId) * GetMaxSnapshotXidCount());
+
+ /*
+ * snapbuild.c builds transactions in an "inverted" manner, which means it
+ * stores committed transactions in ->xip, not ones in progress. Build a
+ * classical snapshot by marking all non-committed transactions as
+ * in-progress. This can be expensive.
+ */
+ for (xid = snap->xmin; NormalTransactionIdPrecedes(xid, snap->xmax);)
+ {
+ void *test;
+
+ /*
+ * Check whether transaction committed using the decoding snapshot
+ * meaning of ->xip.
+ */
+ test = bsearch(&xid, snap->xip, snap->xcnt,
+ sizeof(TransactionId), xidComparator);
+
+ if (test == NULL)
+ {
+ if (newxcnt >= GetMaxSnapshotXidCount())
+ elog(ERROR, "snapshot too large");
+
+ newxip[newxcnt++] = xid;
+ }
+
+ TransactionIdAdvance(xid);
+ }
+
+ snap->xcnt = newxcnt;
+ snap->xip = newxip;
+
+ /*
+ * now that we've built a plain snapshot, use the normal mechanisms for
+ * exporting it
+ */
+ snapname = ExportSnapshot(snap);
+
+ ereport(LOG,
+ (errmsg("exported snapbuild snapshot: \"%s\" with %u xids",
+ snapname, snap->xcnt)));
+ return snapname;
+}
+
+/*
+ * Reset a previously SnapBuildExportSnapshot()'ed snapshot if there is
+ * any. Aborts the previously started transaction and resets the resource
+ * owner back to it's original value.
+ */
+void
+SnapBuildClearExportedSnapshot()
+{
+ /* nothing exported, thats the usual case */
+ if (!ExportInProgress)
+ return;
+
+ if (!IsTransactionState())
+ elog(ERROR, "clearing exported snapshot in wrong transaction state");
+
+ /* make sure nothing could have ever happened */
+ AbortCurrentTransaction();
+
+ CurrentResourceOwner = SavedResourceOwnerDuringExport;
+ SavedResourceOwnerDuringExport = NULL;
+ ExportInProgress = false;
+}
+
+/*
+ * Handle the effects of a single heap change, appropriate to the current state
+ * of the snapshot builder and returns whether changes made at (xid, lsn) may
+ * be decoded.
+ */
+bool
+SnapBuildProcessChange(SnapBuild *builder, TransactionId xid, XLogRecPtr lsn)
+{
+ bool is_old_tx;
+
+ /*
+ * We can't handle data in transactions if we haven't built a snapshot
+ * yet, so don't store them.
+ */
+ if (builder->state < SNAPBUILD_FULL_SNAPSHOT)
+ return false;
+
+ /*
+ * No point in keeping track of changes in transactions that we don't have
+ * enough information about to decode.
+ */
+ if (builder->state < SNAPBUILD_CONSISTENT &&
+ SnapBuildTxnIsRunning(builder, xid))
+ return false;
+
+ /*
+ * There's a change coming that will need a snapshot to be decoded, add
+ * one if not already present.
+ */
+ is_old_tx = ReorderBufferIsXidKnown(builder->reorder, xid);
+
+ if (!is_old_tx || !ReorderBufferXidHasBaseSnapshot(builder->reorder, xid))
+ {
+ /* only build a new snapshot if we don't have a prebuilt one */
+ if (builder->snapshot == NULL)
+ {
+ builder->snapshot = SnapBuildBuildSnapshot(builder, xid);
+ /* inrease refcount for the snapshot builder */
+ SnapBuildSnapIncRefcount(builder->snapshot);
+ }
+
+ /* increase refcount for the transaction */
+ SnapBuildSnapIncRefcount(builder->snapshot);
+ ReorderBufferSetBaseSnapshot(builder->reorder, xid, lsn,
+ builder->snapshot);
+ }
+
+ return true;
+}
+
+/*
+ * Do CommandId/ComboCid handling after reading a xl_heap_new_cid record. This
+ * implies that a transaction has done some for of write to system catalogs.
+ */
+void
+SnapBuildProcessNewCid(SnapBuild *builder, TransactionId xid,
+ XLogRecPtr lsn, xl_heap_new_cid *xlrec)
+{
+ CommandId cid;
+
+ /*
+ * we only log new_cid's if a catalog tuple was modified, so mark
+ * the transaction as containing catalog modifications
+ */
+ ReorderBufferXidSetCatalogChanges(builder->reorder, xid,lsn);
+
+ ReorderBufferAddNewTupleCids(builder->reorder, xlrec->top_xid, lsn,
+ xlrec->target.node, xlrec->target.tid,
+ xlrec->cmin, xlrec->cmax,
+ xlrec->combocid);
+
+ /* figure out new command id */
+ if (xlrec->cmin != InvalidCommandId &&
+ xlrec->cmax != InvalidCommandId)
+ cid = Max(xlrec->cmin, xlrec->cmax);
+ else if (xlrec->cmax != InvalidCommandId)
+ cid = xlrec->cmax;
+ else if (xlrec->cmin != InvalidCommandId)
+ cid = xlrec->cmin;
+ else
+ {
+ cid = InvalidCommandId; /* silence compiler */
+ elog(ERROR, "broken arrow, no cid?");
+ }
+
+ ReorderBufferAddNewCommandId(builder->reorder, xid, lsn, cid + 1);
+}
+
+/*
+ * Check whether `xid` is currently 'running'. Running transactions in our
+ * parlance are transactions which we didn't observe from the start so we can't
+ * properly decode them. They only exist after we freshly started from an
+ * < CONSISTENT snapshot.
+ */
+static bool
+SnapBuildTxnIsRunning(SnapBuild *builder, TransactionId xid)
+{
+ Assert(builder->state < SNAPBUILD_CONSISTENT);
+ Assert(TransactionIdIsValid(builder->running.xmin));
+ Assert(TransactionIdIsValid(builder->running.xmax));
+
+ if (builder->running.xcnt &&
+ NormalTransactionIdFollows(xid, builder->running.xmin) &&
+ NormalTransactionIdPrecedes(xid, builder->running.xmax))
+ {
+ TransactionId *search =
+ bsearch(&xid, builder->running.xip, builder->running.xcnt_space,
+ sizeof(TransactionId), xidComparator);
+
+ if (search != NULL)
+ {
+ Assert(*search == xid);
+ return true;
+ }
+ }
+
+ return false;
+}
+
+/*
+ * Add a new Snapshot to all transactions we're decoding that currently are
+ * in-progress so they can see new catalog contents made by the transaction
+ * that just committed. This is necessary because those in-progress
+ * transactions will use the new catalog's contents from here on (at the very
+ * least everything they do needs to be compatible with newer catalog
+ * contents).
+ */
+static void
+SnapBuildDistributeNewCatalogSnapshot(SnapBuild *builder, XLogRecPtr lsn)
+{
+ dlist_iter txn_i;
+ ReorderBufferTXN *txn;
+
+ /*
+ * Iterate through all toplevel transactions. This can include
+ * subtransactions which we just don't yet know to be that, but that's
+ * fine, they will just get an unneccesary snapshot queued.
+ */
+ dlist_foreach(txn_i, &builder->reorder->toplevel_by_lsn)
+ {
+ txn = dlist_container(ReorderBufferTXN, node, txn_i.cur);
+
+ Assert(TransactionIdIsValid(txn->xid));
+
+ /*
+ * If we don't have a base snapshot yet, there are no changes in this
+ * transaction which in turn implies we don't yet need a snapshot at
+ * all. We'll add add a snapshot when the first change gets queued.
+ *
+ * XXX: is that fine if only a subtransaction has a base snapshot so
+ * far?
+ */
+ if (!ReorderBufferXidHasBaseSnapshot(builder->reorder, txn->xid))
+ continue;
+
+ elog(DEBUG2, "adding a new snapshot to %u at %X/%X",
+ txn->xid, (uint32) (lsn >> 32), (uint32) lsn);
+
+ /* increase refcount for the transaction */
+ SnapBuildSnapIncRefcount(builder->snapshot);
+ ReorderBufferAddSnapshot(builder->reorder, txn->xid, lsn,
+ builder->snapshot);
+ }
+}
+
+/*
+ * Keep track of a new catalog changing transaction that has committed.
+ */
+static void
+SnapBuildAddCommittedTxn(SnapBuild *builder, TransactionId xid)
+{
+ Assert(TransactionIdIsValid(xid));
+
+ if (builder->committed.xcnt == builder->committed.xcnt_space)
+ {
+ builder->committed.xcnt_space = builder->committed.xcnt_space * 2 + 1;
+
+ /* XXX: put in a limit here as a defense against bugs? */
+
+ elog(DEBUG1, "increasing space for committed transactions to %u",
+ (uint32) builder->committed.xcnt_space);
+
+ builder->committed.xip = repalloc(builder->committed.xip,
+ builder->committed.xcnt_space * sizeof(TransactionId));
+ }
+
+ /*
+ * XXX: It might make sense to keep the array sorted here instead of doing
+ * it everytime we build a new snapshot. On the other hand this gets
+ * called repeatedly when a transaction with subtransactions commits.
+ */
+ builder->committed.xip[builder->committed.xcnt++] = xid;
+}
+
+/*
+ * Remove knowledge about transactions we treat as committed that are smaller
+ * than ->xmin. Those won't ever get checked via the ->commited array but via
+ * the clog machinery, so we don't need to waste memory on them.
+ */
+static void
+SnapBuildPurgeCommittedTxn(SnapBuild *builder)
+{
+ int off;
+ TransactionId *workspace;
+ int surviving_xids = 0;
+
+ /* not ready yet */
+ if (!TransactionIdIsNormal(builder->xmin))
+ return;
+
+ /* XXX: Neater algorithm? */
+ workspace =
+ MemoryContextAlloc(builder->context,
+ builder->committed.xcnt * sizeof(TransactionId));
+
+ /* copy xids that still are interesting to workspace */
+ for (off = 0; off < builder->committed.xcnt; off++)
+ {
+ if (NormalTransactionIdPrecedes(builder->committed.xip[off],
+ builder->xmin))
+ ; /* remove */
+ else
+ workspace[surviving_xids++] = builder->committed.xip[off];
+ }
+
+ /* copy workspace back to persistent state */
+ memcpy(builder->committed.xip, workspace,
+ surviving_xids * sizeof(TransactionId));
+
+ elog(DEBUG3, "purged committed transactions from %u to %u, xmin: %u, xmax: %u",
+ (uint32) builder->committed.xcnt, (uint32) surviving_xids,
+ builder->xmin, builder->xmax);
+ builder->committed.xcnt = surviving_xids;
+
+ pfree(workspace);
+}
+
+/*
+ * Common logic for SnapBuildAbortTxn and SnapBuildCommitTxn dealing with
+ * keeping track of the amount of running transactions.
+ */
+static void
+SnapBuildEndTxn(SnapBuild *builder, XLogRecPtr lsn, TransactionId xid)
+{
+ if (builder->state == SNAPBUILD_CONSISTENT)
+ return;
+
+ if (SnapBuildTxnIsRunning(builder, xid))
+ {
+ Assert(builder->running.xcnt > 0);
+
+ if (!--builder->running.xcnt)
+ {
+ /*
+ * None of the originally running transaction is running anymore,
+ * so our incrementaly built snapshot now is consistent.
+ */
+ ereport(LOG,
+ (errmsg("lsn %X/%X: xid %u finished, found consistent point due to SnapBuildEndTxn + running",
+ (uint32)(lsn >> 32), (uint32)lsn, xid)));
+ builder->state = SNAPBUILD_CONSISTENT;
+ }
+ }
+}
+
+/*
+ * Abort a transaction, throw away all state we kept.
+ */
+void
+SnapBuildAbortTxn(SnapBuild *builder, XLogRecPtr lsn,
+ TransactionId xid,
+ int nsubxacts, TransactionId *subxacts)
+{
+ int i;
+
+ for (i = 0; i < nsubxacts; i++)
+ {
+ TransactionId subxid = subxacts[i];
+
+ SnapBuildEndTxn(builder, lsn, subxid);
+ }
+
+ SnapBuildEndTxn(builder, lsn, xid);
+}
+
+/*
+ * Handle everything that needs to be done when a transaction commits
+ */
+void
+SnapBuildCommitTxn(SnapBuild *builder, XLogRecPtr lsn, TransactionId xid,
+ int nsubxacts, TransactionId *subxacts)
+{
+ int nxact;
+
+ bool forced_timetravel = false;
+ bool sub_needs_timetravel = false;
+ bool top_needs_timetravel = false;
+
+ TransactionId xmax = xid;
+
+ /*
+ * If we couldn't observe every change of a transaction because it was
+ * already running at the point we started to observe we have to assume it
+ * made catalog changes.
+ *
+ * This has the positive benefit that we afterwards have enough
+ * information to build an exportable snapshot thats usable by pg_dump et
+ * al.
+ */
+ if (builder->state < SNAPBUILD_CONSISTENT)
+ {
+ /* ensure that only commits after this are getting replayed */
+ if (builder->transactions_after < lsn)
+ builder->transactions_after = lsn;
+
+ /*
+ * we could avoid treating !SnapBuildTxnIsRunning transactions as
+ * timetravel ones, but we want to be able to export a snapshot when
+ * we reached consistency.
+ */
+ forced_timetravel = true;
+ elog(DEBUG1, "forced to assume catalog changes for xid %u because it was running to early", xid);
+ }
+
+ for (nxact = 0; nxact < nsubxacts; nxact++)
+ {
+ TransactionId subxid = subxacts[nxact];
+
+ /*
+ * make sure txn is not tracked in running txn's anymore, switch state
+ */
+ SnapBuildEndTxn(builder, lsn, subxid);
+
+ /*
+ * If we're forcing timetravel we also need accurate subtransaction
+ * status.
+ */
+ if (forced_timetravel)
+ {
+ SnapBuildAddCommittedTxn(builder, subxid);
+ if (NormalTransactionIdFollows(subxid, xmax))
+ xmax = subxid;
+ }
+
+ /*
+ * add subtransaction to base snapshot, we don't distinguish to
+ * toplevel transactions there.
+ */
+ else if (ReorderBufferXidHasCatalogChanges(builder->reorder, subxid))
+ {
+ sub_needs_timetravel = true;
+
+ elog(DEBUG1, "found subtransaction %u:%u with catalog changes.",
+ xid, subxid);
+
+ SnapBuildAddCommittedTxn(builder, subxid);
+
+ if (NormalTransactionIdFollows(subxid, xmax))
+ xmax = subxid;
+ }
+ }
+
+ /*
+ * make sure txn is not tracked in running txn's anymore, switch state
+ */
+ SnapBuildEndTxn(builder, lsn, xid);
+
+ if (forced_timetravel)
+ {
+ elog(DEBUG2, "forced transaction %u to do timetravel.", xid);
+
+ SnapBuildAddCommittedTxn(builder, xid);
+ }
+ /* add toplevel transaction to base snapshot */
+ else if (ReorderBufferXidHasCatalogChanges(builder->reorder, xid))
+ {
+ elog(DEBUG2, "found top level transaction %u, with catalog changes!",
+ xid);
+
+ top_needs_timetravel = true;
+ SnapBuildAddCommittedTxn(builder, xid);
+ }
+ else if (sub_needs_timetravel)
+ {
+ /* mark toplevel txn as timetravel as well */
+ SnapBuildAddCommittedTxn(builder, xid);
+ }
+
+ if (forced_timetravel || top_needs_timetravel || sub_needs_timetravel)
+ {
+ if (!TransactionIdIsValid(builder->xmax) ||
+ TransactionIdFollowsOrEquals(xmax, builder->xmax))
+ {
+ builder->xmax = xmax;
+ TransactionIdAdvance(builder->xmax);
+ }
+
+ if (builder->state < SNAPBUILD_FULL_SNAPSHOT)
+ return;
+
+ /* decrease the snapshot builder's refcount of the old snapshot */
+ if (builder->snapshot)
+ SnapBuildSnapDecRefcount(builder->snapshot);
+
+ builder->snapshot = SnapBuildBuildSnapshot(builder, xid);
+
+ /* we might need to execute invalidations */
+ if (!ReorderBufferXidHasBaseSnapshot(builder->reorder, xid))
+ {
+ SnapBuildSnapIncRefcount(builder->snapshot);
+ ReorderBufferSetBaseSnapshot(builder->reorder, xid, lsn,
+ builder->snapshot);
+ }
+
+ /* refcount of the snapshot builder for the new snapshot */
+ SnapBuildSnapIncRefcount(builder->snapshot);
+
+ /* add a new SnapshotNow to all currently running transactions */
+ SnapBuildDistributeNewCatalogSnapshot(builder, lsn);
+ }
+ else
+ {
+ /* record that we cannot export a general snapshot anymore */
+ builder->committed.includes_all_transactions = false;
+ }
+}
+
+
+/* -----------------------------------
+ * Snapshot building functions dealing with xlog records
+ * -----------------------------------
+ */
+void
+SnapBuildProcessRunningXacts(SnapBuild *builder, XLogRecPtr lsn, xl_running_xacts *running)
+{
+ ReorderBufferTXN *txn;
+
+ if (builder->state < SNAPBUILD_CONSISTENT)
+ {
+ /* returns false if there's no point in performing cleanup just yet */
+ if (!SnapBuildFindSnapshot(builder, lsn, running))
+ return;
+ }
+ else
+ {
+ SnapBuildSerialize(builder, lsn);
+ }
+
+ /*
+ * update range of interesting xids. We don't increase ->xmax because once
+ * we are in a consistent state we can do that ourselves and much more
+ * efficiently so because we only need to do it for catalog transactions.
+ */
+ builder->xmin = running->oldestRunningXid;
+
+ /*
+ * xmax can be lower than xmin here because we only increase xmax when we
+ * hit a transaction with catalog changes. While odd looking, its correct
+ * and actually more efficient this way since we hit fast paths in
+ * tqual.c.
+ */
+
+ /* Remove transactions we don't need to keep track off anymore */
+ SnapBuildPurgeCommittedTxn(builder);
+
+ elog(DEBUG3, "xmin: %u, xmax: %u, oldestrunning: %u",
+ builder->xmin, builder->xmax,
+ running->oldestRunningXid);
+
+ /*
+ * inrease shared memory state, so vacuum can work on tuples we prevent
+ * from being pruned till now.
+ */
+ IncreaseLogicalXminForSlot(lsn, running->oldestRunningXid);
+
+ /*
+ * Also tell the slot where we can restart decoding from. We don't want to
+ * do that after every commit because changing that implies an fsync of
+ * the logical slot's state file, so we only do it everytime we see a
+ * running xacts record.
+ *
+ * Do so by looking for the oldest in progress transaction (determined by
+ * the first LSN of any of its relevant records). Every transaction
+ * remembers the last location we stored the snapshot to disk before its
+ * beginning. That point is where we can restart from.
+ */
+
+ /*
+ * Can't know about a serialized snapshot's location if we're not
+ * consistent
+ */
+ if (builder->state < SNAPBUILD_CONSISTENT)
+ return;
+
+ txn = ReorderBufferGetOldestTXN(builder->reorder);
+
+ /*
+ * oldest ongoing txn might have started when we didn't yet serialize
+ * anything because we hadn't reached a consistent state yet.
+ */
+ if (txn != NULL && txn->restart_decoding_lsn != InvalidXLogRecPtr)
+ IncreaseRestartDecodingForSlot(lsn, txn->restart_decoding_lsn);
+
+ /*
+ * No in-progress transaction, can reuse the last serialized snapshot if we
+ * have one.
+ */
+ else if (txn == NULL &&
+ builder->reorder->current_restart_decoding_lsn != InvalidXLogRecPtr &&
+ builder->last_serialized_snapshot != InvalidXLogRecPtr)
+ IncreaseRestartDecodingForSlot(lsn, builder->last_serialized_snapshot);
+}
+
+
+/*
+ * Build the start of a snapshot that's capable of decoding the
+ * catalog. Helper function for SnapBuildProcessRunningXacts() while we're not
+ * yet consistent.
+ *
+ * Returns true if there is a point in performing internal maintenance/cleanup
+ * using the xl_running_xacts record.
+ */
+static bool
+SnapBuildFindSnapshot(SnapBuild *builder, XLogRecPtr lsn, xl_running_xacts *running)
+{
+ /* ---
+ * Build catalog decoding snapshot incrementally using information about
+ * the currently running transactions. There are several ways to do that:
+
+ * a) There were no running transactions when the xl_running_xacts record
+ * was inserted, jump to CONSISTENT immediately. We might find such a
+ * state we were waiting for b) and c).
+
+ * b) Wait for all toplevel transactions that were running to end. We
+ * simply track the number of in-progress toplevel transactions and
+ * lower it whenever one commits or aborts. When that number
+ * (builder->running.xcnt) reaches zero, we can go from FULL_SNAPSHOT
+ * to CONSISTENT.
+ * NB: We need to search running.xip when seeing a transaction's end to
+ * make sure it's a toplevel transaction and it's been one of the
+ * intially running ones.
+ * Interestingly, in contrast to HS this allows us not to care about
+ * subtransactions - and by extension suboverflowed xl_running_xacts -
+ * at all.
+ *
+ * c) This (in a previous run) or another decoding slot serialized a
+ * snapshot to disk that we can use.
+ * ---
+ */
+
+ /*
+ * xl_running_xact record is older than what we can use, we might not have
+ * all necessary catalog rows anymore.
+ */
+ if (TransactionIdIsNormal(builder->initial_xmin_horizon) &&
+ NormalTransactionIdPrecedes(running->oldestRunningXid,
+ builder->initial_xmin_horizon))
+ {
+ ereport(LOG,
+ (errmsg("lsn %X/%X: skipping snapshot due to initial xmin horizon of %u vs the snapshot's %u",
+ (uint32) (lsn >> 32), (uint32) lsn,
+ builder->initial_xmin_horizon, running->oldestRunningXid)));
+ return true;
+ }
+
+ /*
+ * a) No transaction were running, we can jump to consistent.
+ *
+ * NB: We might have already started to incrementally assemble a snapshot,
+ * so we need to be careful to deal with that.
+ */
+ if (running->xcnt == 0)
+ {
+ if (builder->transactions_after == InvalidXLogRecPtr ||
+ builder->transactions_after < lsn)
+ builder->transactions_after = lsn;
+
+ builder->xmin = running->oldestRunningXid;
+ builder->xmax = running->latestCompletedXid;
+ TransactionIdAdvance(builder->xmax);
+
+ Assert(TransactionIdIsNormal(builder->xmin));
+ Assert(TransactionIdIsNormal(builder->xmax));
+
+ /* no transactions running now */
+ builder->running.xcnt = 0;
+ builder->running.xmin = InvalidTransactionId;
+ builder->running.xmax = InvalidTransactionId;
+
+ builder->state = SNAPBUILD_CONSISTENT;
+
+ ereport(LOG,
+ (errmsg("lsn %X/%X: found initial snapshot (xmin %u) due to running xacts with xcnt == 0",
+ (uint32)(lsn >> 32), (uint32)lsn,
+ builder->xmin)));
+
+ return false;
+ }
+ /* c) valid on disk state */
+ else if (SnapBuildRestore(builder, lsn))
+ {
+ /* there won't be any state to cleanup */
+ return false;
+ }
+ /*
+ * b) first encounter of a useable xl_running_xacts record. If we had
+ * found one earlier we would either track running transactions
+ * (i.e. builder->running.xcnt != 0) or be consistent (this function
+ * wouldn't get called)..
+ */
+ else if (!builder->running.xcnt)
+ {
+ int off;
+
+ /*
+ * We only care about toplevel xids as those are the ones we
+ * definitely see in the wal stream. As snapbuild.c tracks committed
+ * instead of running transactions we don't need to know anything
+ * about uncommitted subtransactions.
+ */
+ builder->xmin = running->oldestRunningXid;
+ builder->xmax = running->latestCompletedXid;
+ TransactionIdAdvance(builder->xmax);
+
+ /* so we can safely use the faster comparisons */
+ Assert(TransactionIdIsNormal(builder->xmin));
+ Assert(TransactionIdIsNormal(builder->xmax));
+
+ builder->running.xcnt = running->xcnt;
+ builder->running.xcnt_space = running->xcnt;
+ builder->running.xip =
+ MemoryContextAlloc(builder->context,
+ builder->running.xcnt * sizeof(TransactionId));
+ memcpy(builder->running.xip, running->xids,
+ builder->running.xcnt * sizeof(TransactionId));
+
+ /* sort so we can do a binary search */
+ qsort(builder->running.xip, builder->running.xcnt,
+ sizeof(TransactionId), xidComparator);
+
+ builder->running.xmin = builder->running.xip[0];
+ builder->running.xmax = builder->running.xip[running->xcnt - 1];
+
+ /* makes comparisons cheaper later */
+ TransactionIdRetreat(builder->running.xmin);
+ TransactionIdAdvance(builder->running.xmax);
+
+ builder->state = SNAPBUILD_FULL_SNAPSHOT;
+
+ ereport(LOG,
+ (errmsg("lsn %X/%X: found initial snapshot (xmin %u) due to running xacts, %u xacts need to finish",
+ (uint32)(lsn >> 32), (uint32)lsn,
+ builder->xmin, (uint32) builder->running.xcnt)));
+
+ /*
+ * Iterate through all xids, wait for them to finish.
+ *
+ * This isn't required for the correctness of decoding but primarily
+ * to allow isolationtester to notice that we're currently waiting for
+ * something.
+ */
+ for(off = 0; off < builder->running.xcnt; off++)
+ {
+ XactLockTableWait(builder->running.xip[off]);
+ }
+
+ /* nothing could have built up so far */
+ return false;
+ }
+
+ /*
+ * We already started to track running xacts and need to wait for all
+ * in-progress ones to finish. We fall through to the normal processing of
+ * records so incremental cleanup can be performed.
+ */
+ return true;
+}
+
+
+/* -----------------------------------
+ * Snapshot serialization support
+ * -----------------------------------
+ */
+
+/*
+ * We store current state of struct SnapBuild on disk in the following manner:
+ *
+ * struct SnapBuildOnDisk;
+ * TransactionId * running.xcnt_space;
+ * TransactionId * committed.xcnt; (*not xcnt_space*)
+ *
+ */
+typedef struct SnapBuildOnDisk
+{
+ /* first part of this struct needs to be version independent */
+
+ /* data not covered by checksum */
+ uint32 magic;
+ pg_crc32 checksum;
+
+ /* data covered by checksum */
+
+ /* version, in case we want to support pg_upgrade */
+ uint32 version;
+ /* how large is the on disk data, excluding the constant sized part */
+ uint32 length;
+
+ /* version dependent part */
+ SnapBuild builder;
+
+ /* variable amount of TransactionIds follows */
+} SnapBuildOnDisk;
+
+#define SnapBuildOnDiskConstantSize \
+ offsetof(SnapBuildOnDisk, builder)
+#define SnapBuildOnDiskNotChecksummedSize \
+ offsetof(SnapBuildOnDisk, version)
+
+#define SNAPBUILD_MAGIC 0x51A1E001
+#define SNAPBUILD_VERSION 1
+
+/*
+ * Store/Load a snapshot from disk, depending on the snapshot builder's state.
+ *
+ * Supposed to be used by external (i.e. not snapbuild.c) code that just reada
+ * record that's a potential location for a serialized snapshot.
+ */
+void
+SnapBuildSerializationPoint(SnapBuild *builder, XLogRecPtr lsn)
+{
+ if (builder->state < SNAPBUILD_CONSISTENT)
+ SnapBuildRestore(builder, lsn);
+ else
+ SnapBuildSerialize(builder, lsn);
+}
+
+/*
+ * Serialize the snapshot 'builder' at the location 'lsn' if it hasn't already
+ * been done by another decoding process.
+ */
+static void
+SnapBuildSerialize(SnapBuild *builder, XLogRecPtr lsn)
+{
+ Size needed_length;
+ SnapBuildOnDisk *ondisk;
+ char *ondisk_c;
+ int fd;
+ char tmppath[MAXPGPATH];
+ char path[MAXPGPATH];
+ int ret;
+ struct stat stat_buf;
+ uint32 sz;
+
+ Assert(lsn != InvalidXLogRecPtr);
+ Assert(builder->last_serialized_snapshot == InvalidXLogRecPtr ||
+ builder->last_serialized_snapshot <= lsn);
+
+ /*
+ * no point in serializing if we cannot continue to work immediately after
+ * restoring the snapshot
+ */
+ if (builder->state < SNAPBUILD_CONSISTENT)
+ return;
+
+ /*
+ * FIXME: Timeline handling/naming.
+ */
+
+ /*
+ * first check whether some other backend already has written the snapshot
+ * for this LSN. It's perfectly fine if there's none, so we accept ENOENT
+ * as a valid state. Everything else is an unexpected error.
+ */
+ sprintf(path, "pg_llog/snapshots/%X-%X.snap",
+ (uint32) (lsn >> 32), (uint32) lsn);
+
+ ret = stat(path, &stat_buf);
+
+ if (ret != 0 && errno != ENOENT)
+ ereport(ERROR, (errmsg("could not stat snapbuild state file %s", path)));
+ else if (ret == 0)
+ {
+ /*
+ * somebody else has already serialized to this point, don't overwrite
+ * but remember location, so we don't need to read old data again.
+ *
+ * To be sure it has been fsynced after the rename() from the tempfile
+ * filename to the real filename we just repeat the fsync. That ought
+ * to be cheap because in most scenarios it should already be
+ * fsync()ed.
+ */
+ fsync_fname(path, false);
+ fsync_fname("pg_llog/snapshots", true);
+
+ builder->last_serialized_snapshot = lsn;
+ goto out;
+ }
+
+ /*
+ * there is an obvious race condition here between the time we stat(2) the
+ * file and us writing the file. But we rename the file into place
+ * atomically and all files created need to contain the same data anyway,
+ * so this is perfectly fine, although a bit of a resource waste. Locking
+ * seems like pointless complication.
+ */
+ elog(DEBUG1, "serializing snapshot to %s", path);
+
+ /* to make sure only we will write to this tempfile, include pid */
+ sprintf(tmppath, "pg_llog/snapshots/%X-%X.snap.%u.tmp",
+ (uint32) (lsn >> 32), (uint32) lsn, MyProcPid);
+
+ /*
+ * Unlink temporary file if it already exists, needs to have been before a
+ * crash/error since we won't enter this function twice from within a
+ * single decoding slot/backend and the temporary file contains the pid of
+ * the current process.
+ */
+ if (unlink(tmppath) != 0 && errno != ENOENT)
+ ereport(ERROR, (errmsg("could not unlink old snapbuild state file %s", path)));
+
+ needed_length = sizeof(SnapBuildOnDisk) +
+ sizeof(TransactionId) * builder->running.xcnt_space +
+ sizeof(TransactionId) * builder->committed.xcnt;
+
+ ondisk_c = MemoryContextAllocZero(builder->context, needed_length);
+ ondisk = (SnapBuildOnDisk *) ondisk_c;
+ ondisk->magic = SNAPBUILD_MAGIC;
+ ondisk->version = SNAPBUILD_VERSION;
+ ondisk->length = needed_length;
+ INIT_CRC32(ondisk->checksum);
+ COMP_CRC32(ondisk->checksum,
+ ((char *) ondisk) + SnapBuildOnDiskNotChecksummedSize,
+ SnapBuildOnDiskConstantSize - SnapBuildOnDiskNotChecksummedSize);
+ ondisk_c += sizeof(SnapBuildOnDisk);
+
+ /* copy state per struct assignment, lalala lazy. */
+ ondisk->builder = *builder;
+ /* NULL-ify memory-only data */
+ ondisk->builder.context = NULL;
+ ondisk->builder.snapshot = NULL;
+ ondisk->builder.reorder = NULL;
+
+ COMP_CRC32(ondisk->checksum,
+ &ondisk->builder,
+ sizeof(SnapBuild));
+
+ /* copy running xacts */
+ sz = sizeof(TransactionId) * builder->running.xcnt_space;
+ memcpy(ondisk_c, builder->running.xip, sz);
+ COMP_CRC32(ondisk->checksum, ondisk_c, sz);
+ ondisk_c += sz;
+
+ /* copy committed xacts */
+ sz = sizeof(TransactionId) * builder->committed.xcnt;
+ memcpy(ondisk_c, builder->committed.xip, sz);
+ COMP_CRC32(ondisk->checksum, ondisk_c, sz);
+ ondisk_c += sz;
+
+ /* we have valid data now, open tempfile and write it there */
+ fd = OpenTransientFile(tmppath,
+ O_CREAT | O_EXCL | O_WRONLY | PG_BINARY,
+ S_IRUSR | S_IWUSR);
+ if (fd < 0)
+ ereport(ERROR, (errmsg("could not open snapbuild state file %s for writing: %m", path)));
+
+ if ((write(fd, ondisk, needed_length)) != needed_length)
+ {
+ CloseTransientFile(fd);
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not write to snapbuild state file \"%s\": %m",
+ tmppath)));
+ }
+
+ /*
+ * fsync the file before renaming so that even if we crash after this we
+ * have either a fully valid file or nothing.
+ *
+ * TODO: Do the fsync() via checkpoints/restartpoints, doing it here has
+ * some noticeable overhead since it's performed synchronously during
+ * decoding?
+ */
+ if (pg_fsync(fd) != 0)
+ {
+ CloseTransientFile(fd);
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not fsync snapbuild state file \"%s\": %m",
+ tmppath)));
+ }
+
+ CloseTransientFile(fd);
+
+ /*
+ * We may overwrite the work from some other backend, but that's ok, our
+ * snapshot is valid as well.
+ */
+ if (rename(tmppath, path) != 0)
+ {
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not rename snapbuild state file from \"%s\" to \"%s\": %m",
+ tmppath, path)));
+ }
+
+ /* make sure we persist */
+ fsync_fname(path, false);
+ fsync_fname("pg_llog/snapshots", true);
+
+ /*
+ * now there's no way we loose the dumped state anymore, remember
+ * serialization point.
+ */
+ builder->last_serialized_snapshot = lsn;
+
+out:
+ ReorderBufferSetRestartPoint(builder->reorder,
+ builder->last_serialized_snapshot);
+}
+
+/*
+ * Restore a snapshot into 'builder' if previously one has been stored at the
+ * location indicated by 'lsn'. Returns true if successfull, false otherwise.
+ */
+static bool
+SnapBuildRestore(SnapBuild *builder, XLogRecPtr lsn)
+{
+ SnapBuildOnDisk ondisk;
+ int fd;
+ char path[MAXPGPATH];
+ Size sz;
+ int readBytes;
+ pg_crc32 checksum;
+
+ /* no point in loading a snapshot if we're already there */
+ if (builder->state == SNAPBUILD_CONSISTENT)
+ return false;
+
+ sprintf(path, "pg_llog/snapshots/%X-%X.snap",
+ (uint32) (lsn >> 32), (uint32) lsn);
+
+ fd = OpenTransientFile(path, O_RDONLY | PG_BINARY, 0);
+
+ if (fd < 0 && errno == ENOENT)
+ return false;
+ else if (fd < 0)
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not open snapbuild state file %s", path)));
+
+ /* read statically sized portion of snapshot */
+ readBytes = read(fd, &ondisk, SnapBuildOnDiskConstantSize);
+ if (readBytes != SnapBuildOnDiskConstantSize)
+ {
+ CloseTransientFile(fd);
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not read snapbuild file \"%s\": %m",
+ path)));
+ }
+
+ if (ondisk.magic != SNAPBUILD_MAGIC)
+ ereport(ERROR, (errmsg("snapbuild state file has wrong magic %u instead of %u",
+ ondisk.magic, SNAPBUILD_MAGIC)));
+
+ if (ondisk.version != SNAPBUILD_VERSION)
+ ereport(ERROR, (errmsg("snapbuild state file has unsupported version %u instead of %u",
+ ondisk.version, SNAPBUILD_VERSION)));
+
+ INIT_CRC32(checksum);
+ COMP_CRC32(checksum,
+ ((char *) &ondisk) + SnapBuildOnDiskNotChecksummedSize,
+ SnapBuildOnDiskConstantSize - SnapBuildOnDiskNotChecksummedSize);
+
+ /* read SnapBuild */
+ readBytes = read(fd, &ondisk.builder, sizeof(SnapBuild));
+ if (readBytes != sizeof(SnapBuild))
+ {
+ CloseTransientFile(fd);
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not read snapshot builder from snapbuild file \"%s\": %m",
+ path)));
+ }
+ COMP_CRC32(checksum, &ondisk.builder, sizeof(SnapBuild));
+
+ /* restore running xacts information */
+ sz = sizeof(TransactionId) * ondisk.builder.running.xcnt_space;
+ ondisk.builder.running.xip = MemoryContextAlloc(builder->context, sz);
+ if (read(fd, ondisk.builder.running.xip, sz) != sz)
+ {
+ CloseTransientFile(fd);
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not read running xacts from snapbuild file \"%s\": %m",
+ path)));
+ }
+ COMP_CRC32(checksum, &ondisk.builder.running.xip, sz);
+
+ /* restore commited xacts information */
+ sz = sizeof(TransactionId) * ondisk.builder.committed.xcnt;
+ ondisk.builder.committed.xip = MemoryContextAlloc(builder->context, sz);
+ if (read(fd, ondisk.builder.committed.xip, sz) != sz)
+ {
+ CloseTransientFile(fd);
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not read committed xacts from snapbuild file \"%s\": %m",
+ path)));
+ }
+ COMP_CRC32(checksum, &ondisk.builder.committed.xip, sz);
+
+ CloseTransientFile(fd);
+
+ /* verify checksum of what we've read */
+ if (!EQ_CRC32(checksum, ondisk.checksum))
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("snapbuild file %s: checksum mismatch, is %u, should be %u",
+ path, checksum, ondisk.checksum)));
+
+ /*
+ * ok, we now have a sensible snapshot here, figure out if it has more
+ * information than we have.
+ */
+
+ /*
+ * We are only interested in consistent snapshots for now, comparing
+ * whether one imcomplete snapshot is more "advanced" seems to be
+ * unnecessarily complex.
+ */
+ if (ondisk.builder.state < SNAPBUILD_CONSISTENT)
+ goto snapshot_not_interesting;
+
+ /*
+ * Don't use a snapshot that requires an xmin that we cannot guarantee to
+ * be available.
+ */
+ if (TransactionIdPrecedes(ondisk.builder.xmin, builder->initial_xmin_horizon))
+ goto snapshot_not_interesting;
+
+ /*
+ * XXX: transactions_after needs to be updated differently, to be checked
+ * here
+ */
+
+ /* ok, we think the snapshot is sensible, copy over everything important */
+ builder->xmin = ondisk.builder.xmin;
+ builder->xmax = ondisk.builder.xmax;
+ builder->state = ondisk.builder.state;
+
+ builder->committed.xcnt = ondisk.builder.committed.xcnt;
+ /* We only allocated/stored xcnt, not xcnt_space xids ! */
+ /* don't overwrite preallocated xip, if we don't have anything here */
+ if (builder->committed.xcnt > 0)
+ {
+ pfree(builder->committed.xip);
+ builder->committed.xcnt_space = ondisk.builder.committed.xcnt;
+ builder->committed.xip = ondisk.builder.committed.xip;
+ }
+ ondisk.builder.committed.xip = NULL;
+
+ builder->running.xcnt = ondisk.builder.committed.xcnt;
+ if (builder->running.xip)
+ pfree(builder->running.xip);
+ builder->running.xcnt_space = ondisk.builder.committed.xcnt_space;
+ builder->running.xip = ondisk.builder.running.xip;
+
+ /* our snapshot is not interesting anymore, build a new one */
+ if (builder->snapshot != NULL)
+ {
+ SnapBuildSnapDecRefcount(builder->snapshot);
+ }
+ builder->snapshot = SnapBuildBuildSnapshot(builder, InvalidTransactionId);
+ SnapBuildSnapIncRefcount(builder->snapshot);
+
+ ReorderBufferSetRestartPoint(builder->reorder, lsn);
+
+ Assert(builder->state == SNAPBUILD_CONSISTENT);
+
+ ereport(LOG,
+ (errmsg("lsn %X/%X: found initial snapshot (xmin %u) in snapbuild file",
+ (uint32)(lsn >> 32), (uint32)lsn,
+ builder->xmin)));
+ return true;
+
+snapshot_not_interesting:
+ if (ondisk.builder.running.xip != NULL)
+ pfree(ondisk.builder.running.xip);
+ if (ondisk.builder.committed.xip != NULL)
+ pfree(ondisk.builder.committed.xip);
+ return false;
+}
+
+/*
+ * Remove all serialized snapshots that are not required anymore because no
+ * slot can need them. This doesn't actually have to run during a checkpoint,
+ * but it's a convenient point to schedule this.
+ *
+ * NB: We run this during checkpoints even if logical decoding is disabled so
+ * we cleanup old slots at some point after it got disabled.
+ */
+void
+CheckPointSnapBuild(void)
+{
+ XLogRecPtr cutoff;
+ XLogRecPtr redo;
+ DIR *snap_dir;
+ struct dirent *snap_de;
+ char path[MAXPGPATH];
+
+ /*
+ * We start of with a minimum of the last redo pointer. No new replication
+ * slot will start before that, so that's a safe upper bound for removal.
+ */
+ redo = GetRedoRecPtr();
+
+ /* now check for the restart ptrs from existing slots */
+ cutoff = ComputeLogicalRestartLSN();
+
+ /* don't start earlier than the restart lsn */
+ if (redo < cutoff)
+ cutoff = redo;
+
+ snap_dir = AllocateDir("pg_llog/snapshots");
+ while ((snap_de = ReadDir(snap_dir, "pg_llog/snapshots")) != NULL)
+ {
+ uint32 hi;
+ uint32 lo;
+ XLogRecPtr lsn;
+ struct stat statbuf;
+
+ if (strcmp(snap_de->d_name, ".") == 0 ||
+ strcmp(snap_de->d_name, "..") == 0)
+ continue;
+
+ snprintf(path, MAXPGPATH, "pg_llog/snapshots/%s", snap_de->d_name);
+
+ if (lstat(path, &statbuf) == 0 && !S_ISREG(statbuf.st_mode))
+ elog(ERROR, "only regular files expected: %s", path);
+
+ /*
+ * temporary filenames from SnapBuildSerialize() include the LSN and
+ * everything but are postfixed by .$pid.tmp. We can just remove them
+ * the same as other files because there can be none that are currently
+ * being written that are older than cutoff.
+ */
+ if (sscanf(snap_de->d_name, "%X-%X.snap", &hi, &lo) != 2)
+ elog(ERROR, "could not parse filename \"%s",
+ snap_de->d_name);
+ lsn = ((uint64) hi) << 32 | lo;
+
+ /* check whether we still need it */
+ if (lsn < cutoff || cutoff == InvalidXLogRecPtr)
+ {
+ elog(DEBUG1, "removing snapbuild snapshot %s", path);
+ if (unlink(path) < 0)
+ ereport(ERROR, (errmsg("failed while unlinking %s", path)));
+ }
+ }
+ FreeDir(snap_dir);
+}
* everything else has been checked.
*/
if (hot_standby_feedback)
- xmin = GetOldestXmin(true, false);
+ xmin = GetOldestXmin(true, false, false);
else
xmin = InvalidTransactionId;
#include "postmaster/bgworker_internals.h"
#include "postmaster/bgwriter.h"
#include "postmaster/postmaster.h"
+#include "replication/logical.h"
#include "replication/walreceiver.h"
#include "replication/walsender.h"
#include "storage/bufmgr.h"
size = add_size(size, ProcSignalShmemSize());
size = add_size(size, CheckpointerShmemSize());
size = add_size(size, AutoVacuumShmemSize());
+ size = add_size(size, LogicalDecodingShmemSize());
size = add_size(size, WalSndShmemSize());
size = add_size(size, WalRcvShmemSize());
size = add_size(size, BTreeShmemSize());
ProcSignalShmemInit();
CheckpointerShmemInit();
AutoVacuumShmemInit();
+ LogicalDecodingShmemInit();
WalSndShmemInit();
WalRcvShmemInit();
#include "access/xact.h"
#include "access/twophase.h"
#include "miscadmin.h"
+#include "replication/logical.h"
#include "storage/proc.h"
#include "storage/procarray.h"
#include "storage/spin.h"
* GetOldestXmin() move backwards, with no consequences for data integrity.
*/
TransactionId
-GetOldestXmin(bool allDbs, bool ignoreVacuum)
+GetOldestXmin(bool allDbs, bool ignoreVacuum, bool alreadyLocked)
{
ProcArrayStruct *arrayP = procArray;
TransactionId result;
int index;
+ volatile TransactionId logical_xmin = InvalidTransactionId;
/* Cannot look for individual databases during recovery */
Assert(allDbs || !RecoveryInProgress());
- LWLockAcquire(ProcArrayLock, LW_SHARED);
+ if (!alreadyLocked)
+ LWLockAcquire(ProcArrayLock, LW_SHARED);
/*
* We initialize the MIN() calculation with latestCompletedXid + 1. This
}
}
+ /* fetch into volatile var while ProcArrayLock is held */
+ if (max_logical_slots > 0)
+ logical_xmin = LogicalDecodingCtl->xmin;
+
if (RecoveryInProgress())
{
/*
*/
TransactionId kaxmin = KnownAssignedXidsGetOldestXmin();
- LWLockRelease(ProcArrayLock);
+ if (!alreadyLocked)
+ LWLockRelease(ProcArrayLock);
if (TransactionIdIsNormal(kaxmin) &&
TransactionIdPrecedes(kaxmin, result))
}
else
{
- /*
- * No other information needed, so release the lock immediately.
- */
- LWLockRelease(ProcArrayLock);
+ if (!alreadyLocked)
+ LWLockRelease(ProcArrayLock);
/*
* Compute the cutoff XID by subtracting vacuum_defer_cleanup_age,
result = FirstNormalTransactionId;
}
+ /*
+ * after locks are released and defer_cleanup_age has been applied, check
+ * whether we need to back up further to make logical decoding possible.
+ */
+ if (TransactionIdIsValid(logical_xmin) &&
+ NormalTransactionIdPrecedes(logical_xmin, result))
+ result = logical_xmin;
+
return result;
}
* older than this are known not running any more.
* RecentGlobalXmin: the global xmin (oldest TransactionXmin across all
* running transactions, except those running LAZY VACUUM). This is
- * the same computation done by GetOldestXmin(true, true).
+ * the same computation done by GetOldestXmin(true, true, ...).
*
* Note: this function should probably not be called with an argument that's
* not statically allocated (see xip allocation below).
int count = 0;
int subcount = 0;
bool suboverflowed = false;
+ volatile TransactionId logical_xmin = InvalidTransactionId;
Assert(snapshot != NULL);
suboverflowed = true;
}
+
+ /* fetch into volatile var while ProcArrayLock is held */
+ if (max_logical_slots > 0)
+ logical_xmin = LogicalDecodingCtl->xmin;
+
if (!TransactionIdIsValid(MyPgXact->xmin))
MyPgXact->xmin = TransactionXmin = xmin;
+
LWLockRelease(ProcArrayLock);
/*
RecentGlobalXmin = globalxmin - vacuum_defer_cleanup_age;
if (!TransactionIdIsNormal(RecentGlobalXmin))
RecentGlobalXmin = FirstNormalTransactionId;
+ /*
+ * peg the global xmin to the one required for logical decoding if required
+ */
+ if (TransactionIdIsNormal(logical_xmin) &&
+ NormalTransactionIdPrecedes(logical_xmin, RecentGlobalXmin))
+ RecentGlobalXmin = logical_xmin;
+
RecentXmin = xmin;
snapshot->xmin = xmin;
* Similar to GetSnapshotData but returns more information. We include
* all PGXACTs with an assigned TransactionId, even VACUUM processes.
*
- * We acquire XidGenLock, but the caller is responsible for releasing it.
- * This ensures that no new XIDs enter the proc array until the caller has
- * WAL-logged this snapshot, and releases the lock.
+ * We acquire XidGenLock and ProcArrayLock, but the caller is responsible for
+ * releasing them. Acquiring XidGenLock ensures that no new XIDs enter the proc
+ * array until the caller has WAL-logged this snapshot, and releases the
+ * lock. Acquiring ProcArrayLock ensures that no transactions commit until the
+ * lock is released.
*
* The returned data structure is statically allocated; caller should not
* modify it, and must not assume it is valid past the next call.
}
}
+ /*
+ * It's important *not* to track decoding tasks here because
+ * snapbuild.c uses ->oldestRunningXid to manage its xmin. If it
+ * were to be included here the initial value could never
+ * increase.
+ */
+
CurrentRunningXacts->xcnt = count - subcount;
CurrentRunningXacts->subxcnt = subcount;
CurrentRunningXacts->subxid_overflow = suboverflowed;
CurrentRunningXacts->oldestRunningXid = oldestRunningXid;
CurrentRunningXacts->latestCompletedXid = latestCompletedXid;
- /* We don't release XidGenLock here, the caller is responsible for that */
- LWLockRelease(ProcArrayLock);
-
Assert(TransactionIdIsValid(CurrentRunningXacts->nextXid));
Assert(TransactionIdIsValid(CurrentRunningXacts->oldestRunningXid));
Assert(TransactionIdIsNormal(CurrentRunningXacts->latestCompletedXid));
+ /* We don't release the locks here, the caller is responsible for that */
+
return CurrentRunningXacts;
}
* record we write, because standby will open up when it sees this.
*/
running = GetRunningTransactionData();
+
+ /*
+ * GetRunningTransactionData() acquired ProcArrayLock, we must release
+ * it. We can do that before inserting the WAL record because
+ * ProcArrayApplyRecoveryInfo can recheck the commit status using the
+ * clog. If we're doing logical replication we can't do that though, so
+ * hold the lock for a moment longer.
+ */
+ if (wal_level < WAL_LEVEL_LOGICAL)
+ LWLockRelease(ProcArrayLock);
+
recptr = LogCurrentRunningXacts(running);
+ /* Release lock if we kept it longer ... */
+ if (wal_level >= WAL_LEVEL_LOGICAL)
+ LWLockRelease(ProcArrayLock);
+
/* GetRunningTransactionData() acquired XidGenLock, we must release it */
LWLockRelease(XidGenLock);
* Only the local caches are flushed; this does not transmit the message
* to other backends.
*/
-static void
+void
LocalExecuteInvalidationMessage(SharedInvalidationMessage *msg)
{
if (msg->id >= 0)
* since that tells us we've lost some shared-inval messages and hence
* don't know what needs to be invalidated.
*/
-static void
+void
InvalidateSystemCaches(void)
{
int i;
#include "utils/memutils.h"
#include "utils/relmapper.h"
#include "utils/resowner_private.h"
+#include "utils/snapmgr.h"
#include "utils/syscache.h"
#include "utils/tqual.h"
bool isshared, bool hasoids,
int natts, const FormData_pg_attribute *attrs);
-static HeapTuple ScanPgRelation(Oid targetRelId, bool indexOK);
+static HeapTuple ScanPgRelation(Oid targetRelId, bool indexOK, bool suspend_snap);
static Relation AllocateRelationDesc(Form_pg_class relp);
static void RelationParseRelOptions(Relation relation, HeapTuple tuple);
static void RelationBuildTupleDesc(Relation relation);
* and must eventually be freed with heap_freetuple.
*/
static HeapTuple
-ScanPgRelation(Oid targetRelId, bool indexOK)
+ScanPgRelation(Oid targetRelId, bool indexOK, bool suspend_snap)
{
HeapTuple pg_class_tuple;
Relation pg_class_desc;
* scan by setting indexOK == false.
*/
pg_class_desc = heap_open(RelationRelationId, AccessShareLock);
+
+ if (suspend_snap)
+ SuspendDecodingSnapshots();
+
pg_class_scan = systable_beginscan(pg_class_desc, ClassOidIndexId,
indexOK && criticalRelcachesBuilt,
NULL,
pg_class_tuple = systable_getnext(pg_class_scan);
+ if (suspend_snap)
+ UnSuspendDecodingSnapshots();
+
/*
* Must copy tuple before releasing buffer.
*/
/*
* find the tuple in pg_class corresponding to the given relation id
*/
- pg_class_tuple = ScanPgRelation(targetRelId, true);
+ pg_class_tuple = ScanPgRelation(targetRelId, true, false);
/*
* if no such tuple exists, return NULL
else
relation->rd_node.dbNode = MyDatabaseId;
if (relation->rd_rel->relfilenode)
+ {
+ /*
+ * Even if we are using a decoding snapshot that doesn't represent
+ * the current state of the catalog we need to make sure the
+ * filenode points to the current file since the older file will
+ * be gone (or truncated). The new file will still contain older
+ * rows so lookups in them will work correctly. This wouldn't work
+ * correctly if rewrites were allowed to change the schema in a
+ * noncompatible way, but those are prevented both on catalog
+ * tables and on user tables declared as additional catalog
+ * tables.
+ */
+ if (DecodingSnapshotsActive()
+ && RelationIsAccessibleInLogicalDecoding(relation))
+ {
+ HeapTuple phys_tuple;
+ Form_pg_class physrel;
+
+ phys_tuple = ScanPgRelation(RelationGetRelid(relation),
+ RelationGetRelid(relation) != ClassOidIndexId,
+ true);
+ if (!HeapTupleIsValid(phys_tuple))
+ elog(ERROR, "could not find pg_class entry for %u",
+ RelationGetRelid(relation));
+ physrel = (Form_pg_class) GETSTRUCT(phys_tuple);
+
+ relation->rd_rel->reltablespace = physrel->reltablespace;
+ relation->rd_rel->relfilenode = physrel->relfilenode;
+ heap_freetuple(phys_tuple);
+ }
+
relation->rd_node.relNode = relation->rd_rel->relfilenode;
+ }
else
{
/* Consult the relation mapper */
rd = RelationBuildDesc(relationId, true);
if (RelationIsValid(rd))
RelationIncrementReferenceCount(rd);
+
+ /* Make sure we're in a xact, even if this ends up being a cache hit */
+ Assert(rd->rd_isnailed || IsTransactionOrTransactionBlock());
+
return rd;
}
* for pg_class_oid_index ...
*/
indexOK = (RelationGetRelid(relation) != ClassOidIndexId);
- pg_class_tuple = ScanPgRelation(RelationGetRelid(relation), indexOK);
+ pg_class_tuple = ScanPgRelation(RelationGetRelid(relation), indexOK, false);
if (!HeapTupleIsValid(pg_class_tuple))
elog(ERROR, "could not find pg_class tuple for index %u",
RelationGetRelid(relation));
#include "postmaster/postmaster.h"
#include "postmaster/syslogger.h"
#include "postmaster/walwriter.h"
+#include "replication/logical.h"
#include "replication/syncrep.h"
#include "replication/walreceiver.h"
#include "replication/walsender.h"
NULL, NULL, NULL
},
+ {
+ /* see max_connections */
+ {"max_logical_slots", PGC_POSTMASTER, REPLICATION_SENDING,
+ gettext_noop("Sets the maximum number of simultaneously defined WAL decoding slots."),
+ NULL
+ },
+ &max_logical_slots,
+ 0, 0, MAX_BACKENDS /* XXX?*/,
+ NULL, NULL, NULL
+ },
+
{
{"wal_sender_timeout", PGC_SIGHUP, REPLICATION_SENDING,
gettext_noop("Sets the maximum time to wait for WAL replication."),
# Set these on the master and on any standby that will send replication data.
-#max_wal_senders = 0 # max number of walsender processes
+#max_wal_senders = 0 # max number of walsender processes, including
+ # both physical and logical replication senders.
# (change requires restart)
#wal_keep_segments = 0 # in logfile segments, 16MB each; 0 disables
#wal_sender_timeout = 60s # in milliseconds; 0 disables
+#max_logical_slots = 0 # max number of logical replication sender
+ # and receiver processes. Logical senders
+ # (but not receivers) also consume a
+ # max_wal_senders slot.
+ # (change requires restart)
+
# - Master Server -
# These settings are ignored on a standby server.
*/
static SnapshotData CurrentSnapshotData = {HeapTupleSatisfiesMVCC};
static SnapshotData SecondarySnapshotData = {HeapTupleSatisfiesMVCC};
-static SnapshotData CatalogSnapshotData = {HeapTupleSatisfiesMVCC};
+SnapshotData CatalogSnapshotData = {HeapTupleSatisfiesMVCC};
/* Pointers to valid snapshots */
static Snapshot CurrentSnapshot = NULL;
* Returns the token (the file name) that can be used to import this
* snapshot.
*/
-static char *
+char *
ExportSnapshot(Snapshot snapshot)
{
TransactionId topXid;
#include "access/xact.h"
#include "storage/bufmgr.h"
#include "storage/procarray.h"
+#include "utils/builtins.h"
+#include "utils/combocid.h"
#include "utils/tqual.h"
SnapshotData SnapshotAnyData = {HeapTupleSatisfiesAny};
SnapshotData SnapshotToastData = {HeapTupleSatisfiesToast};
+static Snapshot TimetravelSnapshot;
+/* (table, ctid) => (cmin, cmax) mapping during timetravel */
+static HTAB *tuplecid_data = NULL;
+static int timetravel_suspended = 0;
+
+
/* local functions */
static bool XidInMVCCSnapshot(TransactionId xid, Snapshot snapshot);
-
+static bool FailsSatisfies(HeapTuple htup, Snapshot snapshot, Buffer buffer);
+static bool RedirectSatisfiesMVCC(HeapTuple htup, Snapshot snapshot,
+ Buffer buffer);
/*
* SetHintBits()
*/
return true;
}
+
+/*
+ * check whether the transaciont id 'xid' in in the pre-sorted array 'xip'.
+ */
+static bool
+TransactionIdInArray(TransactionId xid, TransactionId *xip, Size num)
+{
+ return bsearch(&xid, xip, num,
+ sizeof(TransactionId), xidComparator) != NULL;
+}
+
+/*
+ * See the comments for HeapTupleSatisfiesMVCC for the semantics this function
+ * obeys.
+ *
+ * Only usable on tuples from catalog tables!
+ *
+ * We don't need to support HEAP_MOVED_(IN|OFF) for now because we only support
+ * reading catalog pages which couldn't have been created in an older version.
+ *
+ * We don't set any hint bits in here as it seems unlikely to be beneficial as
+ * those should already be set by normal access and it seems to be too
+ * dangerous to do so as the semantics of doing so during timetravel are more
+ * complicated than when dealing "only" with the present.
+ */
+bool
+HeapTupleSatisfiesMVCCDuringDecoding(HeapTuple htup, Snapshot snapshot,
+ Buffer buffer)
+{
+ HeapTupleHeader tuple = htup->t_data;
+ TransactionId xmin = HeapTupleHeaderGetXmin(tuple);
+ TransactionId xmax = HeapTupleHeaderGetRawXmax(tuple);
+
+ Assert(ItemPointerIsValid(&htup->t_self));
+ Assert(htup->t_tableOid != InvalidOid);
+
+ /* inserting transaction aborted */
+ if (tuple->t_infomask & HEAP_XMIN_INVALID)
+ {
+ Assert(!TransactionIdDidCommit(xmin));
+ return false;
+ }
+ /* check if its one of our txids, toplevel is also in there */
+ else if (TransactionIdInArray(xmin, snapshot->subxip, snapshot->subxcnt))
+ {
+ bool resolved;
+ CommandId cmin = HeapTupleHeaderGetRawCommandId(tuple);
+ CommandId cmax = InvalidCommandId;
+
+ /*
+ * another transaction might have (tried to) delete this tuple or
+ * cmin/cmax was stored in a combocid. S we need to to lookup the
+ * actual values externally.
+ */
+ resolved = ResolveCminCmaxDuringDecoding(tuplecid_data, snapshot,
+ htup, buffer,
+ &cmin, &cmax);
+
+ if (!resolved)
+ elog(ERROR, "could not resolve cmin/cmax of catalog tuple");
+
+ Assert(cmin != InvalidCommandId);
+
+ if (cmin >= snapshot->curcid)
+ return false; /* inserted after scan started */
+ }
+ /* committed before our xmin horizon. Do a normal visibility check. */
+ else if (TransactionIdPrecedes(xmin, snapshot->xmin))
+ {
+ Assert(!(tuple->t_infomask & HEAP_XMIN_COMMITTED &&
+ !TransactionIdDidCommit(xmin)));
+
+ /* check for hint bit first, consult clog afterwards */
+ if (!(tuple->t_infomask & HEAP_XMIN_COMMITTED) &&
+ !TransactionIdDidCommit(xmin))
+ return false;
+ }
+ /* beyond our xmax horizon, i.e. invisible */
+ else if (TransactionIdFollowsOrEquals(xmin, snapshot->xmax))
+ {
+ return false;
+ }
+ /* check if it's a committed transaction in [xmin, xmax) */
+ else if(TransactionIdInArray(xmin, snapshot->xip, snapshot->xcnt))
+ {
+ }
+ /*
+ * none of the above, i.e. between [xmin, xmax) but hasn't
+ * committed. I.e. invisible.
+ */
+ else
+ {
+ return false;
+ }
+
+ /* at this point we know xmin is visible, go on to check xmax */
+
+ /* xid invalid or aborted */
+ if (tuple->t_infomask & HEAP_XMAX_INVALID)
+ return true;
+ /* locked tuples are always visible */
+ else if (HEAP_XMAX_IS_LOCKED_ONLY(tuple->t_infomask))
+ return true;
+ /*
+ * We can see multis here if we're looking at user tables or if
+ * somebody SELECT ... FOR SHARE/UPDATE a system table.
+ */
+ else if (tuple->t_infomask & HEAP_XMAX_IS_MULTI)
+ {
+ xmax = HeapTupleGetUpdateXid(tuple);
+ }
+
+ /* check if its one of our txids, toplevel is also in there */
+ if (TransactionIdInArray(xmax, snapshot->subxip, snapshot->subxcnt))
+ {
+ bool resolved;
+ CommandId cmin;
+ CommandId cmax = HeapTupleHeaderGetRawCommandId(tuple);
+
+ /* Lookup actual cmin/cmax values */
+ resolved = ResolveCminCmaxDuringDecoding(tuplecid_data, snapshot,
+ htup, buffer,
+ &cmin, &cmax);
+
+ if (!resolved)
+ elog(ERROR, "could not resolve combocid to cmax");
+
+ Assert(cmax != InvalidCommandId);
+
+ if (cmax >= snapshot->curcid)
+ return true; /* deleted after scan started */
+ else
+ return false; /* deleted before scan started */
+ }
+ /* below xmin horizon, normal transaction state is valid */
+ else if (TransactionIdPrecedes(xmax, snapshot->xmin))
+ {
+ Assert(!(tuple->t_infomask & HEAP_XMAX_COMMITTED &&
+ !TransactionIdDidCommit(xmax)));
+
+ /* check hint bit first */
+ if (tuple->t_infomask & HEAP_XMAX_COMMITTED)
+ return false;
+
+ /* check clog */
+ return !TransactionIdDidCommit(xmax);
+ }
+ /* above xmax horizon, we cannot possibly see the deleting transaction */
+ else if (TransactionIdFollowsOrEquals(xmax, snapshot->xmax))
+ return true;
+ /* xmax is between [xmin, xmax), check known committed array */
+ else if (TransactionIdInArray(xmax, snapshot->xip, snapshot->xcnt))
+ return false;
+ /* xmax is between [xmin, xmax), but known not to have committed yet */
+ else
+ return true;
+}
+
+/*
+ * Setup a snapshot that replaces normal catalog snapshots that allows catalog
+ * access to behave just like it did at a certain point in the past.
+ *
+ * Needed for after-the-fact WAL decoding.
+ */
+void
+SetupDecodingSnapshots(Snapshot timetravel_snapshot, HTAB *tuplecids)
+{
+ /* prevent recursively setting up decoding snapshots */
+ Assert(CatalogSnapshotData.satisfies != RedirectSatisfiesMVCC);
+
+ CatalogSnapshotData.satisfies = RedirectSatisfiesMVCC;
+ /* make sure normal snapshots aren't used*/
+ SnapshotSelfData.satisfies = FailsSatisfies;
+ SnapshotAnyData.satisfies = FailsSatisfies;
+ SnapshotToastData.satisfies = FailsSatisfies;
+ /* don't overwrite SnapshotToastData, we want that to behave normally */
+
+ /* setup the timetravel snapshot */
+ TimetravelSnapshot = timetravel_snapshot;
+
+ /* setup (cmin, cmax) lookup hash */
+ tuplecid_data = tuplecids;
+
+ timetravel_suspended = 0;
+}
+
+
+/*
+ * Make catalog snapshots behave normally again.
+ */
+void
+RevertFromDecodingSnapshots(bool is_error)
+{
+ Assert(timetravel_suspended == 0 || is_error);
+
+ TimetravelSnapshot = NULL;
+ tuplecid_data = NULL;
+
+ /* rally to restore sanity and/or boredom */
+ CatalogSnapshotData.satisfies = HeapTupleSatisfiesMVCC;
+ SnapshotSelfData.satisfies = HeapTupleSatisfiesSelf;
+ SnapshotAnyData.satisfies = HeapTupleSatisfiesAny;
+ SnapshotToastData.satisfies = HeapTupleSatisfiesToast;
+ timetravel_suspended = 0;
+}
+
+/*
+ * Disable catalog snapshot timetravel and perform old-fashioned access but
+ * make re-enabling cheap.. This is useful for accessing catalog entries which
+ * must stay up2date like the pg_class entries of system relations.
+ *
+ * Can be called several times in a nested fashion since several of it's
+ * callers suspend timetravel access on several code levels.
+ */
+void
+SuspendDecodingSnapshots(void)
+{
+ timetravel_suspended++;
+}
+
+/*
+ * Enable timetravel again, After SuspendDecodingSnapshots it.
+ */
+void
+UnSuspendDecodingSnapshots(void)
+{
+ Assert(timetravel_suspended > 0);
+ timetravel_suspended--;
+}
+
+bool
+DecodingSnapshotsActive(void)
+{
+ return TimetravelSnapshot != NULL;
+}
+
+/*
+ * Error out if a normal snapshot is used. That is neither legal nor expected
+ * during timetravel, so this is just extra assurance.
+ */
+static bool
+FailsSatisfies(HeapTuple htup, Snapshot snapshot, Buffer buffer)
+{
+ elog(ERROR, "Normal snapshots cannot be used during timetravel access.");
+ return false;
+}
+
+
+/*
+ * Call the replacement SatisifiesMVCC with the required Snapshot data.
+ */
+static bool
+RedirectSatisfiesMVCC(HeapTuple htup, Snapshot snapshot, Buffer buffer)
+{
+ Assert(TimetravelSnapshot != NULL);
+ if (timetravel_suspended > 0)
+ return HeapTupleSatisfiesMVCC(htup, snapshot, buffer);
+ return HeapTupleSatisfiesMVCCDuringDecoding(htup, TimetravelSnapshot,
+ buffer);
+}
"base/1",
"pg_tblspc",
"pg_stat",
- "pg_stat_tmp"
+ "pg_stat_tmp",
+ "pg_llog",
+ "pg_llog/snapshots",
+ "pg_llog/mappings"
};
*/
#define XLOG_HEAP2_FREEZE 0x00
#define XLOG_HEAP2_CLEAN 0x10
-/* 0x20 is free, was XLOG_HEAP2_CLEAN_MOVE */
+#define XLOG_HEAP2_REWRITE 0x20
#define XLOG_HEAP2_CLEANUP_INFO 0x30
#define XLOG_HEAP2_VISIBLE 0x40
#define XLOG_HEAP2_MULTI_INSERT 0x50
xl_heaptid target;
} xl_heap_new_cid;
+/* logical rewrite xlog record header */
+typedef struct xl_heap_rewrite_mapping
+{
+ TransactionId mapped_xid; /* xid that might need to see the row */
+ Oid mapped_db; /* DbOid or InvalidOid for shared rels */
+ Oid mapped_rel; /* Oid of the mapped relation */
+ off_t offset; /* How far have we written so far */
+ uint32 num_mappings; /* Number of in-memory mappings */
+ XLogRecPtr start_lsn; /* Insert LSN at begin of rewrite */
+} xl_heap_rewrite_mapping;
+
#define SizeOfHeapNewCid (offsetof(xl_heap_new_cid, target) + SizeOfHeapTid)
extern void HeapTupleHeaderAdvanceLatestRemovedXid(HeapTupleHeader tuple,
extern void heap_desc(StringInfo buf, uint8 xl_info, char *rec);
extern void heap2_redo(XLogRecPtr lsn, XLogRecord *rptr);
extern void heap2_desc(StringInfo buf, uint8 xl_info, char *rec);
+extern void heap_xlog_logical_rewrite(XLogRecPtr lsn, XLogRecord *r);
extern XLogRecPtr log_heap_cleanup_info(RelFileNode rnode,
TransactionId latestRemovedXid);
#define REWRITE_HEAP_H
#include "access/htup.h"
+#include "storage/itemptr.h"
+#include "storage/relfilenode.h"
#include "utils/relcache.h"
/* struct definition is private to rewriteheap.c */
typedef struct RewriteStateData *RewriteState;
-extern RewriteState begin_heap_rewrite(Relation NewHeap,
+extern RewriteState begin_heap_rewrite(Relation OldHeap, Relation NewHeap,
TransactionId OldestXmin, TransactionId FreezeXid,
MultiXactId MultiXactCutoff, bool use_wal);
extern void end_heap_rewrite(RewriteState state);
HeapTuple newTuple);
extern bool rewrite_heap_dead_tuple(RewriteState state, HeapTuple oldTuple);
+/*
+ * On-Disk data format for an individual logical rewrite mapping.
+ */
+typedef struct LogicalRewriteMappingData
+{
+ RelFileNode old_node;
+ RelFileNode new_node;
+ ItemPointerData old_tid;
+ ItemPointerData new_tid;
+} LogicalRewriteMappingData;
+
+/* ---
+ * The filename consists out of the following, dash separated,
+ * components:
+ * 1) database oid or InvalidOid for shared relations
+ * 2) the oid of the relation
+ * 3) xid we are mapping for
+ * 4) LSN at which a rewrite started
+ * 5) xid of the xact performing the mapping
+ *
+ * XXX: use hex format for the LSN as well?
+ * ---
+ */
+#define LOGICAL_REWRITE_FORMAT "map-%x-%x-"UINT64_FORMAT"-%x-%x"
+void CheckpointLogicalRewriteHeap(void);
+
#endif /* REWRITE_HEAP_H */
(AssertMacro(TransactionIdIsNormal(id1) && TransactionIdIsNormal(id2)), \
(int32) ((id1) - (id2)) < 0)
+/* compare two XIDs already known to be normal; this is a macro for speed */
+#define NormalTransactionIdFollows(id1, id2) \
+ (AssertMacro(TransactionIdIsNormal(id1) && TransactionIdIsNormal(id2)), \
+ (int32) ((id1) - (id2)) > 0)
+
/* ----------
* Object ID (OID) zero is InvalidOid.
*
#ifndef XLOGREADER_H
#define XLOGREADER_H
+#include "access/xlog.h"
#include "access/xlog_internal.h"
typedef struct XLogReaderState XLogReaderState;
char *errormsg_buf;
};
-/* Get a new XLogReader */
+
extern XLogReaderState *XLogReaderAllocate(XLogPageReadCB pagereadfunc,
void *private_data);
+
+typedef struct XLogRecordBuffer
+{
+ XLogRecPtr origptr;
+ XLogRecPtr endptr;
+ XLogRecord record;
+ char *record_data;
+} XLogRecordBuffer;
+
+
/* Free an XLogReader */
extern void XLogReaderFree(XLogReaderState *state);
DESCR("statistics: information about currently active backends");
DATA(insert OID = 3099 ( pg_stat_get_wal_senders PGNSP PGUID 12 1 10 0 0 f f f f f t s 0 0 2249 "" "{23,25,25,25,25,25,23,25}" "{o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,sync_priority,sync_state}" _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
DESCR("statistics: information about currently active replication");
+DATA(insert OID = 3475 ( pg_stat_get_logical_decoding_slots PGNSP PGUID 12 1 10 0 0 f f f f f t s 0 0 2249 "" "{25,25,26,16,28,25}" "{o,o,o,o,o,o}" "{slot_name,plugin,datoid,active,xmin,restart_decoding_lsn}" _null_ pg_stat_get_logical_decoding_slots _null_ _null_ _null_ ));
+DESCR("statistics: information about logical replication slots currently in use");
DATA(insert OID = 2026 ( pg_backend_pid PGNSP PGUID 12 1 0 0 0 f f f f t f s 0 0 23 "" _null_ _null_ _null_ _null_ pg_backend_pid _null_ _null_ _null_ ));
DESCR("statistics: current backend PID");
DATA(insert OID = 1937 ( pg_stat_get_backend_pid PGNSP PGUID 12 1 0 0 0 f f f f t f s 1 0 23 "23" _null_ _null_ _null_ _null_ pg_stat_get_backend_pid _null_ _null_ _null_ ));
DATA(insert OID = 3473 ( spg_range_quad_leaf_consistent PGNSP PGUID 12 1 0 0 0 f f f f t f i 2 0 16 "2281 2281" _null_ _null_ _null_ _null_ spg_range_quad_leaf_consistent _null_ _null_ _null_ ));
DESCR("SP-GiST support for quad tree over range");
+DATA(insert OID = 3779 ( init_logical_replication PGNSP PGUID 12 1 0 0 0 f f f f f f v 2 0 2249 "19 19" "{19,19,25,25}" "{i,i,o,o}" "{slotname,plugin,slotname,xlog_position}" _null_ init_logical_replication _null_ _null_ _null_ ));
+DESCR("set up a logical replication slot");
+DATA(insert OID = 3780 ( stop_logical_replication PGNSP PGUID 12 1 0 0 0 f f f f f f v 1 0 23 "19" _null_ _null_ _null_ _null_ stop_logical_replication _null_ _null_ _null_ ));
+DESCR("stop logical replication");
/* event triggers */
DATA(insert OID = 3566 ( pg_event_trigger_dropped_objects PGNSP PGUID 12 10 100 0 0 f f f f t t s 0 0 2249 "" "{26,26,23,25,25,25,25}" "{o,o,o,o,o,o,o}" "{classid, objid, objsubid, object_type, schema_name, object_name, object_identity}" _null_ pg_event_trigger_dropped_objects _null_ _null_ _null_ ));
--- /dev/null
+/*-------------------------------------------------------------------------
+ * decode.h
+ * PostgreSQL WAL to logical transformation
+ *
+ * Portions Copyright (c) 1996-2013, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef DECODE_H
+#define DECODE_H
+
+#include "access/xlogreader.h"
+#include "replication/reorderbuffer.h"
+#include "replication/logical.h"
+
+void DecodeRecordIntoReorderBuffer(LogicalDecodingContext *ctx,
+ XLogRecordBuffer *buf);
+
+#endif
--- /dev/null
+/*-------------------------------------------------------------------------
+ * logical.h
+ * PostgreSQL WAL to logical transformation
+ *
+ * Copyright (c) 2012-2013, PostgreSQL Global Development Group
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef LOGICAL_H
+#define LOGICAL_H
+
+#include "access/xlog.h"
+#include "access/xlogreader.h"
+#include "replication/output_plugin.h"
+#include "storage/shmem.h"
+#include "storage/spin.h"
+
+/*
+ * Shared memory state of a single logical decoding slot
+ */
+typedef struct LogicalDecodingSlot
+{
+ /* lock, on same cacheline as effective_xmin */
+ slock_t mutex;
+
+ /* on-disk xmin, updated first */
+ TransactionId xmin;
+
+ /* in-memory xmin, updated after syncing to disk */
+ TransactionId effective_xmin;
+
+ /* is this slot defined */
+ bool in_use;
+
+ /* is somebody streaming out changes for this slot */
+ bool active;
+
+ /* ----
+ * If we shutdown, crash, whatever where do we have to restart decoding
+ * from to
+ * a) find a valid & ready snapshot
+ * b) the complete content for all in-progress xacts
+ * ----
+ */
+ XLogRecPtr restart_decoding;
+
+ /*
+ * Last location we know the client has confirmed to have safely received
+ * data to. No earlier data can be decoded after a restart/crash.
+ */
+ XLogRecPtr confirmed_flush;
+
+ /* ----
+ * When the client has confirmed flushes >= candidate_xmin_after we can
+ * a) advance the pegged xmin
+ * b) advance restart_decoding_from so we have to read/keep less WAL
+ * ----
+ */
+ TransactionId candidate_xmin;
+ XLogRecPtr candidate_xmin_lsn;
+ XLogRecPtr candidate_restart_valid;
+ XLogRecPtr candidate_restart_decoding;
+
+ /* database the slot is active on */
+ Oid database;
+
+ /* slot identifier */
+ NameData name;
+
+ /* plugin name */
+ NameData plugin;
+} LogicalDecodingSlot;
+
+/*
+ * Shared memory control area for all of logical decoding
+ */
+typedef struct LogicalDecodingCtlData
+{
+ /*
+ * Xmin across all logical slots.
+ *
+ * Protected by ProcArrayLock.
+ */
+ TransactionId xmin;
+
+ LogicalDecodingSlot logical_slots[FLEXIBLE_ARRAY_MEMBER];
+} LogicalDecodingCtlData;
+
+/*
+ * Pointers to shared memory
+ */
+extern LogicalDecodingCtlData *LogicalDecodingCtl;
+extern LogicalDecodingSlot *MyLogicalDecodingSlot;
+
+struct LogicalDecodingContext;
+
+typedef void (*LogicalOutputPluginWriterWrite) (
+ struct LogicalDecodingContext *lr,
+ XLogRecPtr Ptr,
+ TransactionId xid
+);
+
+typedef LogicalOutputPluginWriterWrite LogicalOutputPluginWriterPrepareWrite;
+
+/*
+ * Output plugin callbacks
+ */
+typedef struct OutputPluginCallbacks
+{
+ LogicalDecodeInitCB init_cb;
+ LogicalDecodeBeginCB begin_cb;
+ LogicalDecodeChangeCB change_cb;
+ LogicalDecodeCommitCB commit_cb;
+ LogicalDecodeCleanupCB cleanup_cb;
+} OutputPluginCallbacks;
+
+typedef struct LogicalDecodingContext
+{
+ struct XLogReaderState *reader;
+ struct LogicalDecodingSlot *slot;
+ struct ReorderBuffer *reorder;
+ struct SnapBuild *snapshot_builder;
+
+ struct OutputPluginCallbacks callbacks;
+
+ bool stop_after_consistent;
+
+ /*
+ * User specified options
+ */
+ List *output_plugin_options;
+
+ /*
+ * User-Provided callback for writing/streaming out data.
+ */
+ LogicalOutputPluginWriterPrepareWrite prepare_write;
+ LogicalOutputPluginWriterWrite write;
+
+ /*
+ * Output buffer.
+ */
+ StringInfo out;
+
+ /*
+ * Private data pointer for the creator of the logical decoding context.
+ */
+ void *owner_private;
+
+ /*
+ * Private data pointer of the output plugin.
+ */
+ void *output_plugin_private;
+
+ /*
+ * Private data pointer for the data writer.
+ */
+ void *output_writer_private;
+} LogicalDecodingContext;
+
+/* GUCs */
+extern PGDLLIMPORT int max_logical_slots;
+
+extern Size LogicalDecodingShmemSize(void);
+extern void LogicalDecodingShmemInit(void);
+
+extern void LogicalDecodingAcquireFreeSlot(const char *name, const char *plugin);
+extern void LogicalDecodingReleaseSlot(void);
+extern void LogicalDecodingReAcquireSlot(const char *name);
+extern void LogicalDecodingFreeSlot(const char *name);
+extern bool LogicalDecodingCountDBSlots(Oid dboid, int *nslots, int *nactive);
+
+extern void ComputeLogicalXmin(void);
+extern XLogRecPtr ComputeLogicalRestartLSN(void);
+
+/* change logical xmin */
+extern void IncreaseLogicalXminForSlot(XLogRecPtr lsn, TransactionId xmin);
+
+/* change recovery restart location */
+extern void IncreaseRestartDecodingForSlot(XLogRecPtr current_lsn, XLogRecPtr restart_lsn);
+
+extern void LogicalConfirmReceivedLocation(XLogRecPtr lsn);
+
+extern void CheckLogicalDecodingRequirements(void);
+
+extern void StartupLogicalDecoding(XLogRecPtr checkPointRedo);
+
+extern LogicalDecodingContext *CreateLogicalDecodingContext(
+ LogicalDecodingSlot *slot,
+ bool is_init,
+ XLogRecPtr start_lsn,
+ List *output_plugin_options,
+ XLogPageReadCB read_page,
+ LogicalOutputPluginWriterPrepareWrite prepare_write,
+ LogicalOutputPluginWriterWrite do_write);
+extern bool LogicalDecodingContextReady(LogicalDecodingContext *ctx);
+extern void FreeLogicalDecodingContext(LogicalDecodingContext *ctx);
+
+#endif
--- /dev/null
+/*-------------------------------------------------------------------------
+ * logicalfuncs.h
+ * PostgreSQL WAL to logical transformation support functions
+ *
+ * Copyright (c) 2012-2013, PostgreSQL Global Development Group
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef LOGICALFUNCS_H
+#define LOGICALFUNCS_H
+
+#include "replication/logical.h"
+
+extern int logical_read_local_xlog_page(XLogReaderState *state,
+ XLogRecPtr targetPagePtr,
+ int reqLen, XLogRecPtr targetRecPtr,
+ char *cur_page, TimeLineID *pageTLI);
+
+extern Datum pg_stat_get_logical_decoding_slots(PG_FUNCTION_ARGS);
+
+#endif
--- /dev/null
+/*-------------------------------------------------------------------------
+ * output_plugin.h
+ * PostgreSQL Logical Decode Plugin Interface
+ *
+ * Copyright (c) 2012-2013, PostgreSQL Global Development Group
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef OUTPUT_PLUGIN_H
+#define OUTPUT_PLUGIN_H
+
+#include "replication/reorderbuffer.h"
+
+struct LogicalDecodingContext;
+
+/*
+ * Callback that gets called in a user-defined plugin. ctx->private_data can
+ * be set to some private data.
+ *
+ * "is_init" will be set to "true" if the decoding slot just got defined. When
+ * the same slot is used from there one, it will be "false".
+ *
+ * Gets looked up via the library symbol pg_decode_init.
+ */
+typedef void (*LogicalDecodeInitCB) (
+ struct LogicalDecodingContext *ctx,
+ bool is_init
+);
+
+/*
+ * Callback called for every BEGIN of a successful transaction.
+ *
+ * Gets looked up via the library symbol pg_decode_begin_txn.
+ */
+typedef void (*LogicalDecodeBeginCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn);
+
+/*
+ * Callback for every individual change in a successful transaction.
+ *
+ * Gets looked up via the library symbol pg_decode_change.
+ */
+typedef void (*LogicalDecodeChangeCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ Relation relation,
+ ReorderBufferChange *change
+);
+
+/*
+ * Called for every COMMIT of a successful transaction.
+ *
+ * Gets looked up via the library symbol pg_decode_commit_txn.
+ */
+typedef void (*LogicalDecodeCommitCB) (
+ struct LogicalDecodingContext *,
+ ReorderBufferTXN *txn,
+ XLogRecPtr commit_lsn);
+
+/*
+ * Called to cleanup the state of an output plugin.
+ *
+ * Gets looked up via the library symbol pg_decode_cleanup.
+ */
+typedef void (*LogicalDecodeCleanupCB) (
+ struct LogicalDecodingContext *
+);
+
+#endif /* OUTPUT_PLUGIN_H */
--- /dev/null
+/*
+ * reorderbuffer.h
+ *
+ * PostgreSQL logical replay buffer management
+ *
+ * Copyright (c) 2012-2013, PostgreSQL Global Development Group
+ *
+ * src/include/replication/reorderbuffer.h
+ */
+#ifndef REORDERBUFFER_H
+#define REORDERBUFFER_H
+
+#include "access/htup_details.h"
+
+#include "lib/ilist.h"
+
+#include "storage/sinval.h"
+
+#include "utils/hsearch.h"
+#include "utils/rel.h"
+#include "utils/snapshot.h"
+#include "utils/timestamp.h"
+
+/* an individual tuple, stored in one chunk of memory */
+typedef struct ReorderBufferTupleBuf
+{
+ /* position in preallocated list */
+ slist_node node;
+
+ /* tuple, stored sequentially */
+ HeapTupleData tuple;
+ HeapTupleHeaderData header;
+ char data[MaxHeapTupleSize];
+} ReorderBufferTupleBuf;
+
+/* types of the change passed to a 'change' callback */
+enum ReorderBufferChangeType
+{
+ REORDER_BUFFER_CHANGE_INSERT,
+ REORDER_BUFFER_CHANGE_UPDATE,
+ REORDER_BUFFER_CHANGE_DELETE
+};
+
+/*
+ * a single 'change', can be an insert (with one tuple), an update (old, new),
+ * or a delete (old).
+ *
+ * The same struct is also used internally for other purposes but that should
+ * never be visible outside reorderbuffer.c.
+ */
+typedef struct ReorderBufferChange
+{
+ XLogRecPtr lsn;
+
+ /* type of change */
+ union
+ {
+ enum ReorderBufferChangeType action;
+ /* do not leak internal enum values to the outside */
+ int action_internal;
+ };
+
+ /*
+ * Context data for the change, which part of the union is valid depends
+ * on action/action_internal.
+ */
+ union
+ {
+ /* old, new tuples when action == *_INSERT|UPDATE|DELETE */
+ struct
+ {
+ /* relation that has been changed */
+ RelFileNode relnode;
+ /* valid for DELETE || UPDATE */
+ ReorderBufferTupleBuf *oldtuple;
+ /* valid for INSERT || UPDATE */
+ ReorderBufferTupleBuf *newtuple;
+ } tp;
+
+ /* new snapshot */
+ Snapshot snapshot;
+
+ /* new command id for existing snapshot in a catalog changing tx */
+ CommandId command_id;
+
+ /* new cid mapping for catalog changing transaction */
+ struct
+ {
+ RelFileNode node;
+ ItemPointerData tid;
+ CommandId cmin;
+ CommandId cmax;
+ CommandId combocid;
+ } tuplecid;
+ };
+
+ /*
+ * While in use this is how a change is linked into a transactions,
+ * otherwise it's the preallocated list.
+ */
+ dlist_node node;
+} ReorderBufferChange;
+
+typedef struct ReorderBufferTXN
+{
+ /*
+ * The transactions transaction id, can be a toplevel or sub xid.
+ */
+ TransactionId xid;
+
+ /* did the TX have catalog changes */
+ bool has_catalog_changes;
+
+ /*
+ * Do we know this is a subxact?
+ */
+ bool is_known_as_subxact;
+
+ /*
+ * LSN of the first data carrying, WAL record with knowledge about this
+ * xid. This is allowed to *not* be first record adorned with this xid, if
+ * the previous records aren't relevant for logical decoding.
+ */
+ XLogRecPtr first_lsn;
+
+ /* ----
+ * LSN of the record that lead to this xact to be committed or
+ * aborted. This can be a
+ * * plain commit record
+ * * plain commit record, of a parent transaction
+ * * prepared transaction commit
+ * * plain abort record
+ * * prepared transaction abort
+ * * error during decoding
+ * ----
+ */
+ XLogRecPtr final_lsn;
+
+ /*
+ * LSN pointing to the end of the commit record + 1.
+ */
+ XLogRecPtr end_lsn;
+
+ /*
+ * LSN of the last lsn at which snapshot information reside, so we can
+ * restart decoding from there and fully recover this transaction from
+ * WAL.
+ */
+ XLogRecPtr restart_decoding_lsn;
+
+ /*
+ * Commit time, only known when we read the actual commit record.
+ */
+ TimestampTz commit_time;
+
+ /*
+ * Base snapshot or NULL.
+ */
+ Snapshot base_snapshot;
+
+ /*
+ * How many ReorderBufferChange's do we have in this txn.
+ *
+ * Changes in subtransactions are *not* included but tracked separately.
+ */
+ uint64 nentries;
+
+ /*
+ * How many of the above entries are stored in memory in contrast to being
+ * spilled to disk.
+ */
+ uint64 nentries_mem;
+
+ /*
+ * List of ReorderBufferChange structs, including new Snapshots and new
+ * CommandIds
+ */
+ dlist_head changes;
+
+ /*
+ * List of (relation, ctid) => (cmin, cmax) mappings for catalog tuples.
+ * Those are always assigned to the toplevel transaction. (Keep track of
+ * #entries to create a hash of the right size)
+ */
+ dlist_head tuplecids;
+ uint64 ntuplecids;
+
+ /*
+ * On-demand built hash for looking up the above values.
+ */
+ HTAB *tuplecid_hash;
+
+ /*
+ * Hash containing (potentially partial) toast entries. NULL if no toast
+ * tuples have been found for the current change.
+ */
+ HTAB *toast_hash;
+
+ /*
+ * non-hierarchical list of subtransactions that are *not* aborted. Only
+ * used in toplevel transactions.
+ */
+ dlist_head subtxns;
+ uint32 nsubtxns;
+
+ /*
+ * Stored cache invalidations. This is not a linked list because we get
+ * all the invalidations at once.
+ */
+ uint32 ninvalidations;
+ SharedInvalidationMessage *invalidations;
+
+ /* ---
+ * Position in one of three lists:
+ * * list of subtransactions if we are *known* to be subxact
+ * * list of toplevel xacts (can be a as-yet unknown subxact)
+ * * list of preallocated ReorderBufferTXNs
+ * ---
+ */
+ dlist_node node;
+
+} ReorderBufferTXN;
+
+/* so we can define the callbacks used inside struct ReorderBuffer itself */
+typedef struct ReorderBuffer ReorderBuffer;
+
+/* change callback signature */
+typedef void (*ReorderBufferApplyChangeCB) (
+ ReorderBuffer *rb,
+ ReorderBufferTXN *txn,
+ Relation relation,
+ ReorderBufferChange *change);
+
+/* begin callback signature */
+typedef void (*ReorderBufferBeginCB) (
+ ReorderBuffer *rb,
+ ReorderBufferTXN *txn);
+
+/* commit callback signature */
+typedef void (*ReorderBufferCommitCB) (
+ ReorderBuffer *rb,
+ ReorderBufferTXN *txn,
+ XLogRecPtr commit_lsn);
+
+struct ReorderBuffer
+{
+ /*
+ * xid => ReorderBufferTXN lookup table
+ */
+ HTAB *by_txn;
+
+ /*
+ * Transactions that could be a toplevel xact, ordered by LSN of the first
+ * record bearing that xid..
+ */
+ dlist_head toplevel_by_lsn;
+
+ /*
+ * one-entry sized cache for by_txn. Very frequently the same txn gets
+ * looked up over and over again.
+ */
+ TransactionId by_txn_last_xid;
+ ReorderBufferTXN *by_txn_last_txn;
+
+ /*
+ * Callacks to be called when a transactions commits.
+ */
+ ReorderBufferBeginCB begin;
+ ReorderBufferApplyChangeCB apply_change;
+ ReorderBufferCommitCB commit;
+
+ /*
+ * Pointer that will be passed untouched to the callbacks.
+ */
+ void *private_data;
+
+ /*
+ * Private memory context.
+ */
+ MemoryContext context;
+
+ /*
+ * Data structure slab cache.
+ *
+ * We allocate/deallocate some structures very frequently, to avoid bigger
+ * overhead we cache some unused ones here.
+ *
+ * The maximum number of cached entries is controlled by const variables
+ * ontop of reorderbuffer.c
+ */
+
+ /* cached ReorderBufferTXNs */
+ dlist_head cached_transactions;
+ Size nr_cached_transactions;
+
+ /* cached ReorderBufferChanges */
+ dlist_head cached_changes;
+ Size nr_cached_changes;
+
+ /* cached ReorderBufferTupleBufs */
+ slist_head cached_tuplebufs;
+ Size nr_cached_tuplebufs;
+
+ XLogRecPtr current_restart_decoding_lsn;
+
+ /* buffer for disk<->memory conversions */
+ char *outbuf;
+ Size outbufsize;
+};
+
+
+ReorderBuffer *ReorderBufferAllocate(void);
+void ReorderBufferFree(ReorderBuffer *);
+
+ReorderBufferTupleBuf *ReorderBufferGetTupleBuf(ReorderBuffer *);
+void ReorderBufferReturnTupleBuf(ReorderBuffer *, ReorderBufferTupleBuf *tuple);
+ReorderBufferChange *ReorderBufferGetChange(ReorderBuffer *);
+void ReorderBufferReturnChange(ReorderBuffer *, ReorderBufferChange *);
+
+void ReorderBufferQueueChange(ReorderBuffer *, TransactionId, XLogRecPtr lsn, ReorderBufferChange *);
+void ReorderBufferCommit(ReorderBuffer *, TransactionId,
+ XLogRecPtr commit_lsn, XLogRecPtr end_lsn,
+ TimestampTz commit_time);
+void ReorderBufferAssignChild(ReorderBuffer *, TransactionId, TransactionId, XLogRecPtr commit_lsn);
+void ReorderBufferCommitChild(ReorderBuffer *, TransactionId, TransactionId,
+ XLogRecPtr commit_lsn, XLogRecPtr end_lsn);
+void ReorderBufferAbort(ReorderBuffer *, TransactionId, XLogRecPtr lsn);
+void ReorderBufferAbortOld(ReorderBuffer *, TransactionId xid);
+
+void ReorderBufferSetBaseSnapshot(ReorderBuffer *, TransactionId, XLogRecPtr lsn, struct SnapshotData *snap);
+void ReorderBufferAddSnapshot(ReorderBuffer *, TransactionId, XLogRecPtr lsn, struct SnapshotData *snap);
+void ReorderBufferAddNewCommandId(ReorderBuffer *, TransactionId, XLogRecPtr lsn,
+ CommandId cid);
+void ReorderBufferAddNewTupleCids(ReorderBuffer *, TransactionId, XLogRecPtr lsn,
+ RelFileNode node, ItemPointerData pt,
+ CommandId cmin, CommandId cmax, CommandId combocid);
+void ReorderBufferAddInvalidations(ReorderBuffer *, TransactionId, XLogRecPtr lsn,
+ Size nmsgs, SharedInvalidationMessage *msgs);
+bool ReorderBufferIsXidKnown(ReorderBuffer *, TransactionId xid);
+void ReorderBufferXidSetCatalogChanges(ReorderBuffer *, TransactionId xid, XLogRecPtr lsn);
+bool ReorderBufferXidHasCatalogChanges(ReorderBuffer *, TransactionId xid);
+bool ReorderBufferXidHasBaseSnapshot(ReorderBuffer *, TransactionId xid);
+
+ReorderBufferTXN *ReorderBufferGetOldestTXN(ReorderBuffer *);
+
+void ReorderBufferSetRestartPoint(ReorderBuffer *, XLogRecPtr ptr);
+
+void ReorderBufferStartup(void);
+
+#endif
--- /dev/null
+/*-------------------------------------------------------------------------
+ *
+ * snapbuild.h
+ * Exports from replication/logical/snapbuild.c.
+ *
+ * Copyright (c) 2012-2013, PostgreSQL Global Development Group
+ *
+ * src/include/replication/snapbuild.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef SNAPBUILD_H
+#define SNAPBUILD_H
+
+#include "access/xlogdefs.h"
+#include "utils/snapmgr.h"
+
+typedef enum
+{
+ /*
+ * Initial state, we can't do much yet.
+ */
+ SNAPBUILD_START,
+
+ /*
+ * We have collected enough information to decode tuples in transactions
+ * that started after this.
+ *
+ * Once we reached this we start to collect changes. We cannot apply them
+ * yet because the might be based on transactions that were still running
+ * when we reached them yet.
+ */
+ SNAPBUILD_FULL_SNAPSHOT,
+
+ /*
+ * Found a point after hitting built_full_snapshot where all transactions
+ * that were running at that point finished. Till we reach that we hold
+ * off calling any commit callbacks.
+ */
+ SNAPBUILD_CONSISTENT
+} SnapBuildState;
+
+/* forward declare so we don't have to expose the struct to the public */
+struct SnapBuild;
+typedef struct SnapBuild SnapBuild;
+
+/* forward declare so we don't have to include xlogreader.h */
+struct XLogRecordBuffer;
+struct ReorderBuffer;
+
+extern void CheckPointSnapBuild(void);
+
+extern SnapBuild *AllocateSnapshotBuilder(struct ReorderBuffer *cache,
+ TransactionId xmin_horizon, XLogRecPtr start_lsn);
+extern void FreeSnapshotBuilder(SnapBuild *cache);
+
+extern void SnapBuildSnapDecRefcount(Snapshot snap);
+
+extern const char *SnapBuildExportSnapshot(SnapBuild *snapstate);
+extern void SnapBuildClearExportedSnapshot(void);
+
+extern SnapBuildState SnapBuildCurrentState(SnapBuild *snapstate);
+
+extern bool SnapBuildXactNeedsSkip(SnapBuild *snapstate, XLogRecPtr ptr);
+
+/* don't want to include heapam_xlog.h */
+struct xl_heap_new_cid;
+struct xl_running_xacts;
+
+extern void SnapBuildCommitTxn(SnapBuild *builder, XLogRecPtr lsn,
+ TransactionId xid, int nsubxacts,
+ TransactionId *subxacts);
+extern void SnapBuildAbortTxn(SnapBuild *builder, XLogRecPtr lsn,
+ TransactionId xid, int nsubxacts,
+ TransactionId *subxacts);
+extern bool SnapBuildProcessChange(SnapBuild *builder, TransactionId xid,
+ XLogRecPtr lsn);
+extern void SnapBuildProcessNewCid(SnapBuild *builder, TransactionId xid,
+ XLogRecPtr lsn, struct xl_heap_new_cid *cid);
+extern void SnapBuildProcessRunningXacts(SnapBuild *builder, XLogRecPtr lsn,
+ struct xl_running_xacts *running);
+extern void SnapBuildSerializationPoint(SnapBuild *builder, XLogRecPtr lsn);
+
+#endif /* SNAPBUILD_H */
/*
* ItemPointerCopy
* Copies the contents of one disk item pointer to another.
+ *
+ * Should there ever be padding in an ItemPointer this would need to be handled
+ * differently as it's used as hash key.
*/
#define ItemPointerCopy(fromPointer, toPointer) \
( \
SyncRepLock,
BackgroundWorkerLock,
DynamicSharedMemoryControlLock,
+ LogicalDecodingSlotCtlLock,
/* Individual lock IDs end here */
FirstBufMappingLock,
FirstLockMgrLock = FirstBufMappingLock + NUM_BUFFER_PARTITIONS,
extern bool TransactionIdIsInProgress(TransactionId xid);
extern bool TransactionIdIsActive(TransactionId xid);
-extern TransactionId GetOldestXmin(bool allDbs, bool ignoreVacuum);
+extern TransactionId GetOldestXmin(bool allDbs, bool ignoreVacuum, bool alreadyLocked);
extern TransactionId GetOldestActiveTransactionId(void);
extern VirtualTransactionId *GetVirtualXIDsDelayingChkpt(int *nvxids);
int nmsgs, bool RelcacheInitFileInval,
Oid dbid, Oid tsid);
+extern void LocalExecuteInvalidationMessage(SharedInvalidationMessage *msg);
+
#endif /* SINVAL_H */
extern void CallSyscacheCallbacks(int cacheid, uint32 hashvalue);
+extern void InvalidateSystemCaches(void);
#endif /* INVAL_H */
extern void DeleteAllExportedSnapshotFiles(void);
extern bool ThereAreNoPriorRegisteredSnapshots(void);
+extern char *ExportSnapshot(Snapshot snapshot);
+
#endif /* SNAPMGR_H */
extern PGDLLIMPORT SnapshotData SnapshotSelfData;
extern PGDLLIMPORT SnapshotData SnapshotAnyData;
extern PGDLLIMPORT SnapshotData SnapshotToastData;
+extern PGDLLIMPORT SnapshotData CatalogSnapshotData;
#define SnapshotSelf (&SnapshotSelfData)
#define SnapshotAny (&SnapshotAnyData)
/* This macro encodes the knowledge of which snapshots are MVCC-safe */
#define IsMVCCSnapshot(snapshot) \
- ((snapshot)->satisfies == HeapTupleSatisfiesMVCC)
+ ((snapshot)->satisfies == HeapTupleSatisfiesMVCC || \
+ (snapshot)->satisfies == HeapTupleSatisfiesMVCCDuringDecoding)
/*
* HeapTupleSatisfiesVisibility
uint16 infomask, TransactionId xid);
extern bool HeapTupleHeaderIsOnlyLocked(HeapTupleHeader tuple);
+/* Support for catalog timetravel */
+struct HTAB;
+extern bool HeapTupleSatisfiesMVCCDuringDecoding(HeapTuple htup,
+ Snapshot snapshot, Buffer buffer);
+extern void SetupDecodingSnapshots(Snapshot snapshot_now, struct HTAB *tuplecids);
+extern void RevertFromDecodingSnapshots(bool is_error);
+extern void SuspendDecodingSnapshots(void);
+extern void UnSuspendDecodingSnapshots(void);
+extern bool DecodingSnapshotsActive(void);
+
+/*
+ * To avoid leaking to much knowledge about reorderbuffer implementation
+ * details this is implemented in reorderbuffer.c not tqual.c.
+ */
+extern bool ResolveCminCmaxDuringDecoding(struct HTAB *tuplecid_data,
+ Snapshot snapshot,
+ HeapTuple htup,
+ Buffer buffer,
+ CommandId *cmin, CommandId *cmax);
#endif /* TQUAL_H */
pg_stat_get_db_conflict_bufferpin(d.oid) AS confl_bufferpin,
pg_stat_get_db_conflict_startup_deadlock(d.oid) AS confl_deadlock
FROM pg_database d;
+pg_stat_logical_decoding| SELECT l.slot_name,
+ l.plugin,
+ l.datoid,
+ d.datname AS database,
+ l.active,
+ l.xmin,
+ l.restart_decoding_lsn
+ FROM (pg_stat_get_logical_decoding_slots() l(slot_name, plugin, datoid, active, xmin, restart_decoding_lsn)
+ LEFT JOIN pg_database d ON ((l.datoid = d.oid)));
pg_stat_replication| SELECT s.pid,
s.usesysid,
u.rolname AS usename,
LockingClause
LogOpts
LogStmtLevel
+LogicalDecodeBeginCB
+LogicalDecodeChangeCB
+LogicalDecodeCleanupCB
+LogicalDecodeCommitCB
+LogicalDecodeInitCB
+LogicalDecodingCheckpointData
+LogicalDecodingContext
+LogicalDecodingCtlData
+LogicalDecodingSlot
+LogicalOutputPluginWriterPrepareWrite
+LogicalOutputPluginWriterWrite
LogicalTape
LogicalTapeSet
MAGIC
OprProofCacheEntry
OprProofCacheKey
OutputContext
+OutputPluginCallbacks
OverrideSearchPath
OverrideStackEntry
PACE_HEADER
RelocationBufferInfo
RenameStmt
ReopenPtr
+ReorderBuffer
+ReorderBufferApplyChangeCB
+ReorderBufferBeginCB
+ReorderBufferChange
+ReorderBufferChangeTypeInternal
+ReorderBufferCommitCB
+ReorderBufferDiskChange
+ReorderBufferIterTXNEntry
+ReorderBufferIterTXNState
+ReorderBufferToastEnt
+ReorderBufferTupleBuf
+ReorderBufferTupleCidEnt
+ReorderBufferTupleCidKey
+ReorderBufferTXN
+ReorderBufferTXNByIdEnt
ReplaceVarsFromTargetList_context
ReplaceVarsNoMatchOption
ResTarget
SISeg
SMgrRelation
SMgrRelationData
+SnapBuildAction
+SnapBuildState
SOCKADDR
SOCKET
SPELL
Snapshot
SnapshotData
SnapshotSatisfiesFunc
+Snapstate
+SnapstateOnDisk
SockAddr
Sort
SortBy
XLogRecData
XLogRecPtr
XLogRecord
+XLogRecordBuffer
XLogSegNo
XLogSource
XLogwrtResult