CSNs
authorHeikki Linnakangas <[email protected]>
Mon, 22 Aug 2016 11:00:57 +0000 (14:00 +0300)
committerHeikki Linnakangas <[email protected]>
Mon, 22 Aug 2016 18:21:58 +0000 (21:21 +0300)
* latestCompletedXid is now updated in transam.c, atomically with setting
the csnlog. (used to be handled in procarray.c)

57 files changed:
doc/src/sgml/func.sgml
src/backend/access/heap/heapam.c
src/backend/access/nbtree/README
src/backend/access/rmgrdesc/standbydesc.c
src/backend/access/rmgrdesc/xactdesc.c
src/backend/access/transam/Makefile
src/backend/access/transam/README
src/backend/access/transam/clog.c
src/backend/access/transam/commit_ts.c
src/backend/access/transam/csnlog.c [new file with mode: 0644]
src/backend/access/transam/multixact.c
src/backend/access/transam/subtrans.c [deleted file]
src/backend/access/transam/transam.c
src/backend/access/transam/twophase.c
src/backend/access/transam/varsup.c
src/backend/access/transam/xact.c
src/backend/access/transam/xlog.c
src/backend/catalog/heap.c
src/backend/commands/async.c
src/backend/commands/matview.c
src/backend/commands/tablecmds.c
src/backend/replication/logical/decode.c
src/backend/replication/logical/logical.c
src/backend/replication/logical/reorderbuffer.c
src/backend/replication/logical/snapbuild.c
src/backend/storage/ipc/ipci.c
src/backend/storage/ipc/procarray.c
src/backend/storage/ipc/shmem.c
src/backend/storage/ipc/standby.c
src/backend/storage/lmgr/lmgr.c
src/backend/storage/lmgr/lwlocknames.txt
src/backend/storage/lmgr/predicate.c
src/backend/storage/lmgr/proc.c
src/backend/utils/adt/txid.c
src/backend/utils/probes.d
src/backend/utils/time/snapmgr.c
src/backend/utils/time/tqual.c
src/bin/initdb/initdb.c
src/include/access/clog.h
src/include/access/csnlog.h [new file with mode: 0644]
src/include/access/mvccvars.h [new file with mode: 0644]
src/include/access/subtrans.h
src/include/access/transam.h
src/include/access/xact.h
src/include/access/xlog.h
src/include/c.h
src/include/catalog/pg_proc.h
src/include/replication/snapbuild.h
src/include/storage/lwlock.h
src/include/storage/proc.h
src/include/storage/procarray.h
src/include/storage/standby.h
src/include/storage/standbydefs.h
src/include/utils/snapmgr.h
src/include/utils/snapshot.h
src/test/regress/expected/txid.out
src/test/regress/sql/txid.sql

index 169a385a9cc9decdfadfde16a86ec95eaea7e397..7d6927e0acf272af0e84024fe9f77bdbd603d722 100644 (file)
@@ -17123,10 +17123,6 @@ SELECT collation for ('foo' COLLATE "de_DE");
     <primary>txid_current_snapshot</primary>
    </indexterm>
 
-   <indexterm>
-    <primary>txid_snapshot_xip</primary>
-   </indexterm>
-
    <indexterm>
     <primary>txid_snapshot_xmax</primary>
    </indexterm>
@@ -17164,11 +17160,6 @@ SELECT collation for ('foo' COLLATE "de_DE");
        <entry><type>txid_snapshot</type></entry>
        <entry>get current snapshot</entry>
       </row>
-      <row>
-       <entry><literal><function>txid_snapshot_xip(<parameter>txid_snapshot</parameter>)</function></literal></entry>
-       <entry><type>setof bigint</type></entry>
-       <entry>get in-progress transaction IDs in snapshot</entry>
-      </row>
       <row>
        <entry><literal><function>txid_snapshot_xmax(<parameter>txid_snapshot</parameter>)</function></literal></entry>
        <entry><type>bigint</type></entry>
index c63dfa0bafc606ea3dc1ee9c7427d92c28ed09d4..a01edd75d9d1c64ed25c849d905e7164e49a6fc4 100644 (file)
@@ -3714,9 +3714,8 @@ l2:
                                update_xact = InvalidTransactionId;
 
                        /*
-                        * There was no UPDATE in the MultiXact; or it aborted. No
-                        * TransactionIdIsInProgress() call needed here, since we called
-                        * MultiXactIdWait() above.
+                        * There was no UPDATE in the MultiXact; or it aborted. It cannot
+                        * be in-progress anymore, since we called MultiXactIdWait() above.
                         */
                        if (!TransactionIdIsValid(update_xact) ||
                                TransactionIdDidAbort(update_xact))
@@ -5271,7 +5270,7 @@ heap_acquire_tuplock(Relation relation, ItemPointer tid, LockTupleMode mode,
  * either here, or within MultiXactIdExpand.
  *
  * There is a similar race condition possible when the old xmax was a regular
- * TransactionId.  We test TransactionIdIsInProgress again just to narrow the
+ * TransactionId.  We test TransactionIdGetStatus again just to narrow the
  * window, but it's still possible to end up creating an unnecessary
  * MultiXactId.  Fortunately this is harmless.
  */
@@ -5282,6 +5281,7 @@ compute_new_xmax_infomask(TransactionId xmax, uint16 old_infomask,
                                                  TransactionId *result_xmax, uint16 *result_infomask,
                                                  uint16 *result_infomask2)
 {
+       TransactionIdStatus xidstatus;
        TransactionId new_xmax;
        uint16          new_infomask,
                                new_infomask2;
@@ -5417,7 +5417,7 @@ l5:
                new_xmax = MultiXactIdCreate(xmax, status, add_to_xmax, new_status);
                GetMultiXactIdHintBits(new_xmax, &new_infomask, &new_infomask2);
        }
-       else if (TransactionIdIsInProgress(xmax))
+       else if ((xidstatus = TransactionIdGetStatus(xmax)) == XID_INPROGRESS)
        {
                /*
                 * If the XMAX is a valid, in-progress TransactionId, then we need to
@@ -5446,8 +5446,9 @@ l5:
                                /*
                                 * LOCK_ONLY can be present alone only when a page has been
                                 * upgraded by pg_upgrade.  But in that case,
-                                * TransactionIdIsInProgress() should have returned false.  We
-                                * assume it's no longer locked in this case.
+                                * TransactionIdGetStatus() should not have returned
+                                * XID_INPROGRESS.  We assume it's no longer locked in this
+                                * case.
                                 */
                                elog(WARNING, "LOCK_ONLY found for Xid in progress %u", xmax);
                                old_infomask |= HEAP_XMAX_INVALID;
@@ -5500,7 +5501,7 @@ l5:
                GetMultiXactIdHintBits(new_xmax, &new_infomask, &new_infomask2);
        }
        else if (!HEAP_XMAX_IS_LOCKED_ONLY(old_infomask) &&
-                        TransactionIdDidCommit(xmax))
+                        xidstatus == XID_COMMITTED)
        {
                /*
                 * It's a committed update, so we gotta preserve him as updater of the
@@ -5529,7 +5530,7 @@ l5:
                /*
                 * Can get here iff the locking/updating transaction was running when
                 * the infomask was extracted from the tuple, but finished before
-                * TransactionIdIsInProgress got to run.  Deal with it as if there was
+                * TransactionIdGetStatus got to run.  Deal with it as if there was
                 * no locker at all in the first place.
                 */
                old_infomask |= HEAP_XMAX_INVALID;
@@ -5560,15 +5561,11 @@ test_lockmode_for_conflict(MultiXactStatus status, TransactionId xid,
                                                   LockTupleMode mode, bool *needwait)
 {
        MultiXactStatus wantedstatus;
+       TransactionIdStatus xidstatus;
 
        *needwait = false;
        wantedstatus = get_mxact_status_for_lock(mode, false);
 
-       /*
-        * Note: we *must* check TransactionIdIsInProgress before
-        * TransactionIdDidAbort/Commit; see comment at top of tqual.c for an
-        * explanation.
-        */
        if (TransactionIdIsCurrentTransactionId(xid))
        {
                /*
@@ -5577,7 +5574,9 @@ test_lockmode_for_conflict(MultiXactStatus status, TransactionId xid,
                 */
                return HeapTupleSelfUpdated;
        }
-       else if (TransactionIdIsInProgress(xid))
+       xidstatus = TransactionIdGetStatus(xid);
+
+       if (xidstatus == XID_INPROGRESS)
        {
                /*
                 * If the locking transaction is running, what we do depends on
@@ -5597,9 +5596,9 @@ test_lockmode_for_conflict(MultiXactStatus status, TransactionId xid,
                 */
                return HeapTupleMayBeUpdated;
        }
-       else if (TransactionIdDidAbort(xid))
+       else if (xidstatus == XID_ABORTED)
                return HeapTupleMayBeUpdated;
-       else if (TransactionIdDidCommit(xid))
+       else if (xidstatus == XID_COMMITTED)
        {
                /*
                 * The other transaction committed.  If it was only a locker, then the
@@ -5612,7 +5611,7 @@ test_lockmode_for_conflict(MultiXactStatus status, TransactionId xid,
                 * Note: the reason we worry about ISUPDATE here is because as soon as
                 * a transaction ends, all its locks are gone and meaningless, and
                 * thus we can ignore them; whereas its updates persist.  In the
-                * TransactionIdIsInProgress case, above, we don't need to check
+                * XID_INPROGRESS case, above, we don't need to check
                 * because we know the lock is still "alive" and thus a conflict needs
                 * always be checked.
                 */
@@ -5626,9 +5625,7 @@ test_lockmode_for_conflict(MultiXactStatus status, TransactionId xid,
 
                return HeapTupleMayBeUpdated;
        }
-
-       /* Not in progress, not aborted, not committed -- must have crashed */
-       return HeapTupleMayBeUpdated;
+       return 0; /* not reached */
 }
 
 
@@ -6372,7 +6369,7 @@ FreezeMultiXactId(MultiXactId multi, uint16 t_infomask,
                         */
                        if (TransactionIdPrecedes(xid, cutoff_xid))
                        {
-                               Assert(!TransactionIdDidCommit(xid));
+                               Assert(TransactionIdGetStatus(xid) == XID_ABORTED);
                                *flags |= FRM_INVALIDATE_XMAX;
                                xid = InvalidTransactionId;             /* not strictly necessary */
                        }
@@ -6443,6 +6440,7 @@ FreezeMultiXactId(MultiXactId multi, uint16 t_infomask,
                if (ISUPDATE_from_mxstatus(members[i].status))
                {
                        TransactionId xid = members[i].xid;
+                       TransactionIdStatus xidstatus;
 
                        /*
                         * It's an update; should we keep it?  If the transaction is known
@@ -6450,18 +6448,14 @@ FreezeMultiXactId(MultiXactId multi, uint16 t_infomask,
                         * Note that an updater older than cutoff_xid cannot possibly be
                         * committed, because HeapTupleSatisfiesVacuum would have returned
                         * HEAPTUPLE_DEAD and we would not be trying to freeze the tuple.
-                        *
-                        * As with all tuple visibility routines, it's critical to test
-                        * TransactionIdIsInProgress before TransactionIdDidCommit,
-                        * because of race conditions explained in detail in tqual.c.
                         */
-                       if (TransactionIdIsCurrentTransactionId(xid) ||
-                               TransactionIdIsInProgress(xid))
+                       xidstatus = TransactionIdGetStatus(xid);
+                       if (xidstatus == XID_INPROGRESS)
                        {
                                Assert(!TransactionIdIsValid(update_xid));
                                update_xid = xid;
                        }
-                       else if (TransactionIdDidCommit(xid))
+                       else if (xidstatus == XID_COMMITTED)
                        {
                                /*
                                 * The transaction committed, so we can tell caller to set
@@ -6499,8 +6493,7 @@ FreezeMultiXactId(MultiXactId multi, uint16 t_infomask,
                else
                {
                        /* We only keep lockers if they are still running */
-                       if (TransactionIdIsCurrentTransactionId(members[i].xid) ||
-                               TransactionIdIsInProgress(members[i].xid))
+                       if (TransactionIdGetStatus(members[i].xid) == XID_INPROGRESS)
                        {
                                /* running locker cannot possibly be older than the cutoff */
                                Assert(!TransactionIdPrecedes(members[i].xid, cutoff_xid));
@@ -6974,6 +6967,7 @@ DoesMultiXactIdConflict(MultiXactId multi, uint16 infomask,
                {
                        TransactionId memxid;
                        LOCKMODE        memlockmode;
+                       TransactionIdStatus     xidstatus;
 
                        memlockmode = LOCKMODE_from_mxstatus(members[i].status);
 
@@ -6986,16 +6980,18 @@ DoesMultiXactIdConflict(MultiXactId multi, uint16 infomask,
                        if (TransactionIdIsCurrentTransactionId(memxid))
                                continue;
 
+                       xidstatus = TransactionIdGetStatus(memxid);
+
                        if (ISUPDATE_from_mxstatus(members[i].status))
                        {
                                /* ignore aborted updaters */
-                               if (TransactionIdDidAbort(memxid))
+                               if (xidstatus == XID_ABORTED)
                                        continue;
                        }
                        else
                        {
                                /* ignore lockers-only that are no longer in progress */
-                               if (!TransactionIdIsInProgress(memxid))
+                               if (xidstatus != XID_INPROGRESS)
                                        continue;
                        }
 
@@ -7075,7 +7071,7 @@ Do_MultiXactIdWait(MultiXactId multi, MultiXactStatus status,
                        if (!DoLockModesConflict(LOCKMODE_from_mxstatus(memstatus),
                                                                         LOCKMODE_from_mxstatus(status)))
                        {
-                               if (remaining && TransactionIdIsInProgress(memxid))
+                               if (remaining && TransactionIdGetStatus(memxid) == XID_INPROGRESS)
                                        remain++;
                                continue;
                        }
index 067d15c803916e3524e992dbbf4dbf2e40559fac..92b76aa8faacebdb66b23508b636bebc5581e2e7 100644 (file)
@@ -321,6 +321,9 @@ older than RecentGlobalXmin.  As collateral damage, this implementation
 also waits for running XIDs with no snapshots and for snapshots taken
 until the next transaction to allocate an XID commits.
 
+XXX: now that we use CSNs as snapshots, it would be more
+straightforward to use something based on CSNs instead of RecentGlobalXmin.
+
 Reclaiming a page doesn't actually change its state on disk --- we simply
 record it in the shared-memory free space map, from which it will be
 handed out the next time a new page is needed for a page split.  The
index 13797a3d2f49b94ded08e97c754611ff2805d1bc..267ac5e51954227b3b71a107f39cbc5c5335445a 100644 (file)
 static void
 standby_desc_running_xacts(StringInfo buf, xl_running_xacts *xlrec)
 {
-       int                     i;
-
        appendStringInfo(buf, "nextXid %u latestCompletedXid %u oldestRunningXid %u",
                                         xlrec->nextXid,
                                         xlrec->latestCompletedXid,
                                         xlrec->oldestRunningXid);
-       if (xlrec->xcnt > 0)
-       {
-               appendStringInfo(buf, "; %d xacts:", xlrec->xcnt);
-               for (i = 0; i < xlrec->xcnt; i++)
-                       appendStringInfo(buf, " %u", xlrec->xids[i]);
-       }
-
-       if (xlrec->subxid_overflow)
-               appendStringInfoString(buf, "; subxid ovf");
 }
 
 void
index 91d27d0654ea36816bdfba1f8e54f6da7a0afcf3..a9c7bc0fa0eafc3ed7cedfa17d8c902ce65bdd2c 100644 (file)
@@ -255,17 +255,6 @@ xact_desc_abort(StringInfo buf, uint8 info, xl_xact_abort *xlrec)
        }
 }
 
-static void
-xact_desc_assignment(StringInfo buf, xl_xact_assignment *xlrec)
-{
-       int                     i;
-
-       appendStringInfoString(buf, "subxacts:");
-
-       for (i = 0; i < xlrec->nsubxacts; i++)
-               appendStringInfo(buf, " %u", xlrec->xsub[i]);
-}
-
 void
 xact_desc(StringInfo buf, XLogReaderState *record)
 {
@@ -285,18 +274,6 @@ xact_desc(StringInfo buf, XLogReaderState *record)
 
                xact_desc_abort(buf, XLogRecGetInfo(record), xlrec);
        }
-       else if (info == XLOG_XACT_ASSIGNMENT)
-       {
-               xl_xact_assignment *xlrec = (xl_xact_assignment *) rec;
-
-               /*
-                * Note that we ignore the WAL record's xid, since we're more
-                * interested in the top-level xid that issued the record and which
-                * xids are being reported here.
-                */
-               appendStringInfo(buf, "xtop %u: ", xlrec->xtop);
-               xact_desc_assignment(buf, xlrec);
-       }
 }
 
 const char *
@@ -321,9 +298,6 @@ xact_identify(uint8 info)
                case XLOG_XACT_ABORT_PREPARED:
                        id = "ABORT_PREPARED";
                        break;
-               case XLOG_XACT_ASSIGNMENT:
-                       id = "ASSIGNMENT";
-                       break;
        }
 
        return id;
index 16fbe47269a9be93ac6a283b93da54aa4b82454e..fea6d28e3336cf354d18c742655a7916c0468584 100644 (file)
@@ -12,8 +12,8 @@ subdir = src/backend/access/transam
 top_builddir = ../../../..
 include $(top_builddir)/src/Makefile.global
 
-OBJS = clog.o commit_ts.o generic_xlog.o multixact.o parallel.o rmgr.o slru.o \
-       subtrans.o timeline.o transam.o twophase.o twophase_rmgr.o varsup.o \
+OBJS = clog.o commit_ts.o csnlog.o generic_xlog.o multixact.o parallel.o rmgr.o slru.o \
+       timeline.o transam.o twophase.o twophase_rmgr.o varsup.o \
        xact.o xlog.o xlogarchive.o xlogfuncs.o \
        xloginsert.o xlogreader.o xlogutils.o
 
index 4ae4715339e707ab5ed879a628aa96b73002ef43..51b0d166be1003e86697379302e41197912066fb 100644 (file)
@@ -244,44 +244,24 @@ transaction Y as committed, then snapshot A must consider transaction Y as
 committed".
 
 What we actually enforce is strict serialization of commits and rollbacks
-with snapshot-taking: we do not allow any transaction to exit the set of
-running transactions while a snapshot is being taken.  (This rule is
-stronger than necessary for consistency, but is relatively simple to
-enforce, and it assists with some other issues as explained below.)  The
-implementation of this is that GetSnapshotData takes the ProcArrayLock in
-shared mode (so that multiple backends can take snapshots in parallel),
-but ProcArrayEndTransaction must take the ProcArrayLock in exclusive mode
-while clearing MyPgXact->xid at transaction end (either commit or abort).
-(To reduce context switching, when multiple transactions commit nearly
-simultaneously, we have one backend take ProcArrayLock and clear the XIDs
-of multiple processes at once.)
-
-ProcArrayEndTransaction also holds the lock while advancing the shared
-latestCompletedXid variable.  This allows GetSnapshotData to use
-latestCompletedXid + 1 as xmax for its snapshot: there can be no
-transaction >= this xid value that the snapshot needs to consider as
-completed.
-
-In short, then, the rule is that no transaction may exit the set of
-currently-running transactions between the time we fetch latestCompletedXid
-and the time we finish building our snapshot.  However, this restriction
-only applies to transactions that have an XID --- read-only transactions
-can end without acquiring ProcArrayLock, since they don't affect anyone
-else's snapshot nor latestCompletedXid.
-
-Transaction start, per se, doesn't have any interlocking with these
-considerations, since we no longer assign an XID immediately at transaction
-start.  But when we do decide to allocate an XID, GetNewTransactionId must
-store the new XID into the shared ProcArray before releasing XidGenLock.
-This ensures that all top-level XIDs <= latestCompletedXid are either
-present in the ProcArray, or not running anymore.  (This guarantee doesn't
-apply to subtransaction XIDs, because of the possibility that there's not
-room for them in the subxid array; instead we guarantee that they are
-present or the overflow flag is set.)  If a backend released XidGenLock
-before storing its XID into MyPgXact, then it would be possible for another
-backend to allocate and commit a later XID, causing latestCompletedXid to
-pass the first backend's XID, before that value became visible in the
-ProcArray.  That would break GetOldestXmin, as discussed below.
+with snapshot-taking. Each commit is assigned a Commit Sequence Number, or
+CSN for short, using a monotonically increasing counter. A snapshot is
+represented by the value of the CSN counter, at the time the snapshot was
+taken. All (committed) transactions with a CSN <= the snapshot's CSN are
+considered as visible to the snapshot.
+
+When checking the visibility of a tuple, we need to look up the CSN
+of the xmin/xmax. For that purpose, we store the CSN of each
+transaction in the Commit Sequence Number log (csnlog).
+
+So, a snapshot is simply a CSN, such that all transactions that committed
+before that LSN are visible, and everything later is still considered as
+in-progress. However, to avoid consulting the csnlog every time the visibilty
+of a tuple is checked, we also record a lower and upper bound of the XIDs
+considered visible by the snapshot, in SnapshotData. When a snapshot is
+taken, xmax is set to the current nextXid value; any transaction that begins
+after the snapshot is surely still running. The xmin is tracked lazily in
+shared memory, by AdvanceRecentGlobalXmin().
 
 We allow GetNewTransactionId to store the XID into MyPgXact->xid (or the
 subxid array) without taking ProcArrayLock.  This was once necessary to
@@ -293,48 +273,34 @@ once, rather than assume they can read it multiple times and get the same
 answer each time.  (Use volatile-qualified pointers when doing this, to
 ensure that the C compiler does exactly what you tell it to.)
 
-Another important activity that uses the shared ProcArray is GetOldestXmin,
-which must determine a lower bound for the oldest xmin of any active MVCC
-snapshot, system-wide.  Each individual backend advertises the smallest
-xmin of its own snapshots in MyPgXact->xmin, or zero if it currently has no
+Another important activity that uses the shared ProcArray is GetOldestSnapshot
+which must determine a lower bound for the oldest of any active MVCC
+snapshots, system-wide.  Each individual backend advertises the earliest
+of its own snapshots in MyPgXact->snapshotcsn, or zero if it currently has no
 live snapshots (eg, if it's between transactions or hasn't yet set a
-snapshot for a new transaction).  GetOldestXmin takes the MIN() of the
-valid xmin fields.  It does this with only shared lock on ProcArrayLock,
-which means there is a potential race condition against other backends
-doing GetSnapshotData concurrently: we must be certain that a concurrent
-backend that is about to set its xmin does not compute an xmin less than
-what GetOldestXmin returns.  We ensure that by including all the active
-XIDs into the MIN() calculation, along with the valid xmins.  The rule that
-transactions can't exit without taking exclusive ProcArrayLock ensures that
-concurrent holders of shared ProcArrayLock will compute the same minimum of
-currently-active XIDs: no xact, in particular not the oldest, can exit
-while we hold shared ProcArrayLock.  So GetOldestXmin's view of the minimum
-active XID will be the same as that of any concurrent GetSnapshotData, and
-so it can't produce an overestimate.  If there is no active transaction at
-all, GetOldestXmin returns latestCompletedXid + 1, which is a lower bound
-for the xmin that might be computed by concurrent or later GetSnapshotData
-calls.  (We know that no XID less than this could be about to appear in
-the ProcArray, because of the XidGenLock interlock discussed above.)
-
-GetSnapshotData also performs an oldest-xmin calculation (which had better
-match GetOldestXmin's) and stores that into RecentGlobalXmin, which is used
-for some tuple age cutoff checks where a fresh call of GetOldestXmin seems
-too expensive.  Note that while it is certain that two concurrent
-executions of GetSnapshotData will compute the same xmin for their own
-snapshots, as argued above, it is not certain that they will arrive at the
-same estimate of RecentGlobalXmin.  This is because we allow XID-less
-transactions to clear their MyPgXact->xmin asynchronously (without taking
-ProcArrayLock), so one execution might see what had been the oldest xmin,
-and another not.  This is OK since RecentGlobalXmin need only be a valid
-lower bound.  As noted above, we are already assuming that fetch/store
-of the xid fields is atomic, so assuming it for xmin as well is no extra
-risk.
-
-
-pg_clog and pg_subtrans
+snapshot for a new transaction).  GetOldestSnapshot takes the MIN() of the
+snapshots.
+
+For freezing tuples, vacuum needs to know the oldest XID that is still
+considered running by any active transaction. That is, the oldest XID still
+considered running by the oldest active snapshot, as returned by
+GetOldestSnapshotCSN(). This value is somewhat expensive to calculate, so
+the most recently calculated value is kept in shared memory
+(SharedVariableCache->recentXmin), and is recalculated lazily by
+AdvanceRecentGlobalXmin() function. AdvanceRecentGlobalXmin() first scans
+the proc array, and makes note of the oldest active XID. That XID - 1 will
+become the new xmin. It then waits until all currently active snapshots have
+finished. Any snapshot that begins later will see the xmin as finished, so
+after all the active snapshots have finished, xmin will be visible to
+everyone. However, AdvanceRecentGlobalXmin() does not actually block waiting
+for anything; instead it contains a state machine that advances if possible,
+when AdvanceRecentGlobalXmin() is called. AdvanceRecentGlobalXmin() is
+called periodically by the WAL writer, so that it doesn't get very stale.
+
+pg_clog and pg_csnlog
 -----------------------
 
-pg_clog and pg_subtrans are permanent (on-disk) storage of transaction related
+pg_clog and pg_csnlog are permanent (on-disk) storage of transaction related
 information.  There is a limited number of pages of each kept in memory, so
 in many cases there is no need to actually read from disk.  However, if
 there's a long running transaction or a backend sitting idle with an open
@@ -343,21 +309,10 @@ from disk.  They also allow information to be permanent across server restarts.
 
 pg_clog records the commit status for each transaction that has been assigned
 an XID.  A transaction can be in progress, committed, aborted, or
-"sub-committed".  This last state means that it's a subtransaction that's no
-longer running, but its parent has not updated its state yet.  It is not
-necessary to update a subtransaction's transaction status to subcommit, so we
-can just defer it until main transaction commit.  The main role of marking
-transactions as sub-committed is to provide an atomic commit protocol when
-transaction status is spread across multiple clog pages. As a result, whenever
-transaction status spreads across multiple pages we must use a two-phase commit
-protocol: the first phase is to mark the subtransactions as sub-committed, then
-we mark the top level transaction and all its subtransactions committed (in
-that order).  Thus, subtransactions that have not aborted appear as in-progress
-even when they have already finished, and the subcommit status appears as a
-very short transitory state during main transaction commit.  Subtransaction
-abort is always marked in clog as soon as it occurs.  When the transaction
-status all fit in a single CLOG page, we atomically mark them all as committed
-without bothering with the intermediate sub-commit state.
+"committing". For committed transactions, the clog stores the commit WAL
+record's LSN. This last state means that the transaction is just about to
+write its commit WAL record, or just did so, but it hasn't yet updated the
+clog with the record's LSN.
 
 Savepoints are implemented using subtransactions.  A subtransaction is a
 transaction inside a transaction; its commit or abort status is not only
@@ -370,7 +325,7 @@ transaction.
 The "subtransaction parent" (pg_subtrans) mechanism records, for each
 transaction with an XID, the TransactionId of its parent transaction.  This
 information is stored as soon as the subtransaction is assigned an XID.
-Top-level transactions do not have a parent, so they leave their pg_subtrans
+Top-level transactions do not have a parent, so they leave their pg_csnlog
 entries set to the default value of zero (InvalidTransactionId).
 
 pg_subtrans is used to check whether the transaction in question is still
index 263447679b8991ff35796d421bb790e6d2f5dd28..0c382d15dd689572bda91280dd22d25059aaa736 100644 (file)
@@ -33,6 +33,7 @@
 #include "postgres.h"
 
 #include "access/clog.h"
+#include "access/mvccvars.h"
 #include "access/slru.h"
 #include "access/transam.h"
 #include "access/xlog.h"
@@ -84,17 +85,15 @@ static int  ZeroCLOGPage(int pageno, bool writeXlog);
 static bool CLOGPagePrecedes(int page1, int page2);
 static void WriteZeroPageXlogRec(int pageno);
 static void WriteTruncateXlogRec(int pageno);
-static void TransactionIdSetPageStatus(TransactionId xid, int nsubxids,
-                                                  TransactionId *subxids, XidStatus status,
+static void CLogSetPageStatus(TransactionId xid, int nsubxids,
+                                                  TransactionId *subxids, CLogXidStatus status,
                                                   XLogRecPtr lsn, int pageno);
-static void TransactionIdSetStatusBit(TransactionId xid, XidStatus status,
+static void CLogSetStatusBit(TransactionId xid, CLogXidStatus status,
                                                  XLogRecPtr lsn, int slotno);
-static void set_status_by_pages(int nsubxids, TransactionId *subxids,
-                                       XidStatus status, XLogRecPtr lsn);
 
 
 /*
- * TransactionIdSetTreeStatus
+ * CLogSetTreeStatus
  *
  * Record the final state of transaction entries in the commit log for
  * a transaction and its subtransaction tree. Take care to ensure this is
@@ -112,30 +111,13 @@ static void set_status_by_pages(int nsubxids, TransactionId *subxids,
  * caller guarantees the commit record is already flushed in that case.  It
  * should be InvalidXLogRecPtr for abort cases, too.
  *
- * In the commit case, atomicity is limited by whether all the subxids are in
- * the same CLOG page as xid.  If they all are, then the lock will be grabbed
- * only once, and the status will be set to committed directly.  Otherwise
- * we must
- *      1. set sub-committed all subxids that are not on the same page as the
- *             main xid
- *      2. atomically set committed the main xid and the subxids on the same page
- *      3. go over the first bunch again and set them committed
- * Note that as far as concurrent checkers are concerned, main transaction
- * commit as a whole is still atomic.
- *
- * Example:
- *             TransactionId t commits and has subxids t1, t2, t3, t4
- *             t is on page p1, t1 is also on p1, t2 and t3 are on p2, t4 is on p3
- *             1. update pages2-3:
- *                                     page2: set t2,t3 as sub-committed
- *                                     page3: set t4 as sub-committed
- *             2. update page1:
- *                                     set t1 as sub-committed,
- *                                     then set t as committed,
-                                       then set t1 as committed
- *             3. update pages2-3:
- *                                     page2: set t2,t3 as committed
- *                                     page3: set t4 as committed
+ * The atomicity is limited by whether all the subxids are in the same CLOG
+ * page as xid.  If they all are, then the lock will be grabbed only once,
+ * and the status will be set to committed directly.  Otherwise there is
+ * a window that the parent will be seen as committed, while (some of) the
+ * children are still seen as in-progress. That's OK with the current use,
+ * as visibility checking code will not rely on the CLOG for recent
+ * transactions (CSNLOG will be used instead).
  *
  * NB: this is a low-level routine and is NOT the preferred entry point
  * for most uses; functions in transam.c are the intended callers.
@@ -145,102 +127,45 @@ static void set_status_by_pages(int nsubxids, TransactionId *subxids,
  * cache yet.
  */
 void
-TransactionIdSetTreeStatus(TransactionId xid, int nsubxids,
-                                       TransactionId *subxids, XidStatus status, XLogRecPtr lsn)
+CLogSetTreeStatus(TransactionId xid, int nsubxids,
+                                 TransactionId *subxids, CLogXidStatus status, XLogRecPtr lsn)
 {
-       int                     pageno = TransactionIdToPage(xid);              /* get page of parent */
+       TransactionId topXid;
+       int                     pageno;
        int                     i;
+       int                     offset;
 
-       Assert(status == TRANSACTION_STATUS_COMMITTED ||
-                  status == TRANSACTION_STATUS_ABORTED);
-
-       /*
-        * See how many subxids, if any, are on the same page as the parent, if
-        * any.
-        */
-       for (i = 0; i < nsubxids; i++)
-       {
-               if (TransactionIdToPage(subxids[i]) != pageno)
-                       break;
-       }
+       Assert(status == CLOG_XID_STATUS_COMMITTED ||
+                  status == CLOG_XID_STATUS_ABORTED);
 
        /*
-        * Do all items fit on a single page?
+        * Update the clog page-by-page. On first iteration, we will set the
+        * status of the top-XID, and any subtransactions on the same page.
         */
-       if (i == nsubxids)
-       {
-               /*
-                * Set the parent and all subtransactions in a single call
-                */
-               TransactionIdSetPageStatus(xid, nsubxids, subxids, status, lsn,
-                                                                  pageno);
-       }
-       else
-       {
-               int                     nsubxids_on_first_page = i;
-
-               /*
-                * If this is a commit then we care about doing this correctly (i.e.
-                * using the subcommitted intermediate status).  By here, we know
-                * we're updating more than one page of clog, so we must mark entries
-                * that are *not* on the first page so that they show as subcommitted
-                * before we then return to update the status to fully committed.
-                *
-                * To avoid touching the first page twice, skip marking subcommitted
-                * for the subxids on that first page.
-                */
-               if (status == TRANSACTION_STATUS_COMMITTED)
-                       set_status_by_pages(nsubxids - nsubxids_on_first_page,
-                                                               subxids + nsubxids_on_first_page,
-                                                               TRANSACTION_STATUS_SUB_COMMITTED, lsn);
-
-               /*
-                * Now set the parent and subtransactions on same page as the parent,
-                * if any
-                */
-               pageno = TransactionIdToPage(xid);
-               TransactionIdSetPageStatus(xid, nsubxids_on_first_page, subxids, status,
-                                                                  lsn, pageno);
-
-               /*
-                * Now work through the rest of the subxids one clog page at a time,
-                * starting from the second page onwards, like we did above.
-                */
-               set_status_by_pages(nsubxids - nsubxids_on_first_page,
-                                                       subxids + nsubxids_on_first_page,
-                                                       status, lsn);
-       }
-}
-
-/*
- * Helper for TransactionIdSetTreeStatus: set the status for a bunch of
- * transactions, chunking in the separate CLOG pages involved. We never
- * pass the whole transaction tree to this function, only subtransactions
- * that are on different pages to the top level transaction id.
- */
-static void
-set_status_by_pages(int nsubxids, TransactionId *subxids,
-                                       XidStatus status, XLogRecPtr lsn)
-{
-       int                     pageno = TransactionIdToPage(subxids[0]);
-       int                     offset = 0;
-       int                     i = 0;
-
-       while (i < nsubxids)
+       pageno = TransactionIdToPage(xid);              /* get page of parent */
+       topXid = xid;
+       offset = 0;
+       i = 0;
+       for (;;)
        {
                int                     num_on_page = 0;
 
-               while (TransactionIdToPage(subxids[i]) == pageno && i < nsubxids)
+               while (i < nsubxids && TransactionIdToPage(subxids[i]) == pageno)
                {
                        num_on_page++;
                        i++;
                }
 
-               TransactionIdSetPageStatus(InvalidTransactionId,
-                                                                  num_on_page, subxids + offset,
-                                                                  status, lsn, pageno);
+               CLogSetPageStatus(topXid,
+                                                 num_on_page, subxids + offset,
+                                                 status, lsn, pageno);
+
+               if (i == nsubxids)
+                       break;
+
                offset = i;
                pageno = TransactionIdToPage(subxids[offset]);
+               topXid = InvalidTransactionId;
        }
 }
 
@@ -248,19 +173,18 @@ set_status_by_pages(int nsubxids, TransactionId *subxids,
  * Record the final state of transaction entries in the commit log for
  * all entries on a single page.  Atomic only on this page.
  *
- * Otherwise API is same as TransactionIdSetTreeStatus()
+ * Otherwise API is same as CLogSetTreeStatus()
  */
 static void
-TransactionIdSetPageStatus(TransactionId xid, int nsubxids,
-                                                  TransactionId *subxids, XidStatus status,
-                                                  XLogRecPtr lsn, int pageno)
+CLogSetPageStatus(TransactionId xid, int nsubxids,
+                                 TransactionId *subxids, CLogXidStatus status,
+                                 XLogRecPtr lsn, int pageno)
 {
        int                     slotno;
        int                     i;
 
-       Assert(status == TRANSACTION_STATUS_COMMITTED ||
-                  status == TRANSACTION_STATUS_ABORTED ||
-                  (status == TRANSACTION_STATUS_SUB_COMMITTED && !TransactionIdIsValid(xid)));
+       Assert(status == CLOG_XID_STATUS_COMMITTED ||
+                  status == CLOG_XID_STATUS_ABORTED);
 
        LWLockAcquire(CLogControlLock, LW_EXCLUSIVE);
 
@@ -275,38 +199,15 @@ TransactionIdSetPageStatus(TransactionId xid, int nsubxids,
         */
        slotno = SimpleLruReadPage(ClogCtl, pageno, XLogRecPtrIsInvalid(lsn), xid);
 
-       /*
-        * Set the main transaction id, if any.
-        *
-        * If we update more than one xid on this page while it is being written
-        * out, we might find that some of the bits go to disk and others don't.
-        * If we are updating commits on the page with the top-level xid that
-        * could break atomicity, so we subcommit the subxids first before we mark
-        * the top-level commit.
-        */
+       /* Set the main transaction id, if any. */
        if (TransactionIdIsValid(xid))
-       {
-               /* Subtransactions first, if needed ... */
-               if (status == TRANSACTION_STATUS_COMMITTED)
-               {
-                       for (i = 0; i < nsubxids; i++)
-                       {
-                               Assert(ClogCtl->shared->page_number[slotno] == TransactionIdToPage(subxids[i]));
-                               TransactionIdSetStatusBit(subxids[i],
-                                                                                 TRANSACTION_STATUS_SUB_COMMITTED,
-                                                                                 lsn, slotno);
-                       }
-               }
-
-               /* ... then the main transaction */
-               TransactionIdSetStatusBit(xid, status, lsn, slotno);
-       }
+               CLogSetStatusBit(xid, status, lsn, slotno);
 
        /* Set the subtransactions */
        for (i = 0; i < nsubxids; i++)
        {
                Assert(ClogCtl->shared->page_number[slotno] == TransactionIdToPage(subxids[i]));
-               TransactionIdSetStatusBit(subxids[i], status, lsn, slotno);
+               CLogSetStatusBit(subxids[i], status, lsn, slotno);
        }
 
        ClogCtl->shared->page_dirty[slotno] = true;
@@ -320,7 +221,7 @@ TransactionIdSetPageStatus(TransactionId xid, int nsubxids,
  * Must be called with CLogControlLock held
  */
 static void
-TransactionIdSetStatusBit(TransactionId xid, XidStatus status, XLogRecPtr lsn, int slotno)
+CLogSetStatusBit(TransactionId xid, CLogXidStatus status, XLogRecPtr lsn, int slotno)
 {
        int                     byteno = TransactionIdToByte(xid);
        int                     bshift = TransactionIdToBIndex(xid) * CLOG_BITS_PER_XACT;
@@ -331,23 +232,13 @@ TransactionIdSetStatusBit(TransactionId xid, XidStatus status, XLogRecPtr lsn, i
        byteptr = ClogCtl->shared->page_buffer[slotno] + byteno;
        curval = (*byteptr >> bshift) & CLOG_XACT_BITMASK;
 
-       /*
-        * When replaying transactions during recovery we still need to perform
-        * the two phases of subcommit and then commit. However, some transactions
-        * are already correctly marked, so we just treat those as a no-op which
-        * allows us to keep the following Assert as restrictive as possible.
-        */
-       if (InRecovery && status == TRANSACTION_STATUS_SUB_COMMITTED &&
-               curval == TRANSACTION_STATUS_COMMITTED)
-               return;
-
        /*
         * Current state change should be from 0 or subcommitted to target state
         * or we should already be there when replaying changes during recovery.
         */
        Assert(curval == 0 ||
-                  (curval == TRANSACTION_STATUS_SUB_COMMITTED &&
-                       status != TRANSACTION_STATUS_IN_PROGRESS) ||
+                  (curval == CLOG_XID_STATUS_SUB_COMMITTED &&
+                       status != CLOG_XID_STATUS_IN_PROGRESS) ||
                   curval == status);
 
        /* note this assumes exclusive access to the clog page */
@@ -388,8 +279,8 @@ TransactionIdSetStatusBit(TransactionId xid, XidStatus status, XLogRecPtr lsn, i
  * NB: this is a low-level routine and is NOT the preferred entry point
  * for most uses; TransactionLogFetch() in transam.c is the intended caller.
  */
-XidStatus
-TransactionIdGetStatus(TransactionId xid, XLogRecPtr *lsn)
+CLogXidStatus
+CLogGetStatus(TransactionId xid, XLogRecPtr *lsn)
 {
        int                     pageno = TransactionIdToPage(xid);
        int                     byteno = TransactionIdToByte(xid);
@@ -397,7 +288,7 @@ TransactionIdGetStatus(TransactionId xid, XLogRecPtr *lsn)
        int                     slotno;
        int                     lsnindex;
        char       *byteptr;
-       XidStatus       status;
+       CLogXidStatus   status;
 
        /* lock is acquired by SimpleLruReadPage_ReadOnly */
 
index e330105217d83c4bb7075cccb207a07d8d91d252..0e7aba12fa696248c8139ff57dd7d0c370217dd4 100644 (file)
@@ -26,6 +26,7 @@
 
 #include "access/commit_ts.h"
 #include "access/htup_details.h"
+#include "access/mvccvars.h"
 #include "access/slru.h"
 #include "access/transam.h"
 #include "catalog/pg_type.h"
diff --git a/src/backend/access/transam/csnlog.c b/src/backend/access/transam/csnlog.c
new file mode 100644 (file)
index 0000000..23ad93c
--- /dev/null
@@ -0,0 +1,599 @@
+/*-------------------------------------------------------------------------
+ *
+ * csnlog.c
+ *             Tracking Commit-Sequence-Numbers and in-progress subtransactions
+ *
+ * The pg_csnlog manager is a pg_clog-like manager that stores the commit
+ * sequence number, or parent transaction Id, for each transaction.  It is
+ * a fundamental part of MVCC.
+ *
+ * The csnlog serves two purposes:
+ *
+ * 1. While a transaction is in progress, it stores the parent transaction
+ * Id for each in-progress subtransaction. A main transaction has a parent
+ * of InvalidTransactionId, and each subtransaction has its immediate
+ * parent. The tree can easily be walked from child to parent, but not in
+ * the opposite direction.
+ *
+ * 2. After a transaction has committed, it stores the Commit Sequence
+ * Number of the commit.
+ *
+ * We can use the same structure for both, because we don't care about the
+ * parent-child relationships subtransaction after commit.
+ *
+ * This code is based on clog.c, but the robustness requirements
+ * are completely different from pg_clog, because we only need to remember
+ * pg_csnlog information for currently-open and recently committed
+ * transactions.  Thus, there is no need to preserve data over a crash and
+ * restart.
+ *
+ * There are no XLOG interactions since we do not care about preserving
+ * data across crashes.  During database startup, we simply force the
+ * currently-active page of CSNLOG to zeroes.
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/backend/access/transam/csnlog.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/csnlog.h"
+#include "access/mvccvars.h"
+#include "access/slru.h"
+#include "access/subtrans.h"
+#include "access/transam.h"
+#include "miscadmin.h"
+#include "pg_trace.h"
+#include "utils/snapmgr.h"
+
+/*
+ * Defines for CSNLOG page sizes.  A page is the same BLCKSZ as is used
+ * everywhere else in Postgres.
+ *
+ * Note: because TransactionIds are 32 bits and wrap around at 0xFFFFFFFF,
+ * CSNLOG page numbering also wraps around at 0xFFFFFFFF/CSNLOG_XACTS_PER_PAGE,
+ * and CSNLOG segment numbering at
+ * 0xFFFFFFFF/CLOG_XACTS_PER_PAGE/SLRU_PAGES_PER_SEGMENT.  We need take no
+ * explicit notice of that fact in this module, except when comparing segment
+ * and page numbers in TruncateCSNLOG (see CSNLOGPagePrecedes).
+ */
+
+/* We store the commit LSN for each xid */
+#define CSNLOG_XACTS_PER_PAGE (BLCKSZ / sizeof(CommitSeqNo))
+
+#define TransactionIdToPage(xid)       ((xid) / (TransactionId) CSNLOG_XACTS_PER_PAGE)
+#define TransactionIdToPgIndex(xid) ((xid) % (TransactionId) CSNLOG_XACTS_PER_PAGE)
+
+/*
+ * Link to shared-memory data structures for CLOG control
+ */
+static SlruCtlData CsnlogCtlData;
+
+#define CsnlogCtl (&CsnlogCtlData)
+
+
+static int     ZeroCSNLOGPage(int pageno);
+static bool CSNLOGPagePrecedes(int page1, int page2);
+static void CSNLogSetPageStatus(TransactionId xid, int nsubxids,
+                                                  TransactionId *subxids,
+                                                  CommitSeqNo csn, int pageno);
+static void CSNLogSetCSN(TransactionId xid, CommitSeqNo csn, int slotno);
+
+/*
+ * CSNLogSetCommitSeqNo
+ *
+ * Record the status and CSN of transaction entries in the commit log for a
+ * transaction and its subtransaction tree. Take care to ensure this is
+ * efficient, and as atomic as possible.
+ *
+ * xid is a single xid to set status for. This will typically be the
+ * top level transactionid for a top level commit or abort. It can
+ * also be a subtransaction when we record transaction aborts.
+ *
+ * subxids is an array of xids of length nsubxids, representing subtransactions
+ * in the tree of xid. In various cases nsubxids may be zero.
+ *
+ * csn is the commit sequence number of the transaction. It should be
+ * InvalidCommitSeqNo for abort cases.
+ *
+ * Note: This doesn't guarantee atomicity. The caller can use the
+ * COMMITSEQNO_COMMITTING special value for that.
+ */
+void
+CSNLogSetCommitSeqNo(TransactionId xid, int nsubxids,
+                                        TransactionId *subxids, CommitSeqNo csn)
+{
+       int                     pageno;
+       int                     i = 0;
+       int                     offset = 0;
+
+       if (csn == InvalidCommitSeqNo || xid == BootstrapTransactionId)
+       {
+               if (IsBootstrapProcessingMode())
+                       csn = COMMITSEQNO_FROZEN;
+               else
+                       elog(ERROR, "cannot mark transaction committed without CSN");
+       }
+
+       pageno = TransactionIdToPage(xid);              /* get page of parent */
+       for (;;)
+       {
+               int                     num_on_page = 0;
+
+               while (i < nsubxids && TransactionIdToPage(subxids[i]) == pageno)
+               {
+                       num_on_page++;
+                       i++;
+               }
+
+               CSNLogSetPageStatus(xid,
+                                                       num_on_page, subxids + offset,
+                                                       csn, pageno);
+               if (i >= nsubxids)
+                       break;
+
+               offset = i;
+               pageno = TransactionIdToPage(subxids[offset]);
+               xid = InvalidTransactionId;
+       }
+}
+
+/*
+ * Record the final state of transaction entries in the csn log for
+ * all entries on a single page.  Atomic only on this page.
+ *
+ * Otherwise API is same as TransactionIdSetTreeStatus()
+ */
+static void
+CSNLogSetPageStatus(TransactionId xid, int nsubxids,
+                                                  TransactionId *subxids,
+                                                  CommitSeqNo csn, int pageno)
+{
+       int                     slotno;
+       int                     i;
+
+       LWLockAcquire(CSNLogControlLock, LW_EXCLUSIVE);
+
+       slotno = SimpleLruReadPage(CsnlogCtl, pageno, true, xid);
+
+       /* Subtransactions first, if needed ... */
+       for (i = 0; i < nsubxids; i++)
+       {
+               Assert(CsnlogCtl->shared->page_number[slotno] == TransactionIdToPage(subxids[i]));
+               CSNLogSetCSN(subxids[i],        csn, slotno);
+       }
+
+       /* ... then the main transaction */
+       if (TransactionIdIsValid(xid))
+               CSNLogSetCSN(xid, csn, slotno);
+
+       CsnlogCtl->shared->page_dirty[slotno] = true;
+
+       LWLockRelease(CSNLogControlLock);
+}
+
+
+
+/*
+ * Record the parent of a subtransaction in the subtrans log.
+ *
+ * In some cases we may need to overwrite an existing value.
+ */
+void
+SubTransSetParent(TransactionId xid, TransactionId parent, bool overwriteOK)
+{
+       int                     pageno = TransactionIdToPage(xid);
+       int                     entryno = TransactionIdToPgIndex(xid);
+       int                     slotno;
+       CommitSeqNo *ptr;
+       CommitSeqNo newcsn;
+
+       Assert(TransactionIdIsValid(parent));
+
+       newcsn = CSN_SUBTRANS_BIT | (uint64) parent;
+
+       LWLockAcquire(CSNLogControlLock, LW_EXCLUSIVE);
+
+       slotno = SimpleLruReadPage(CsnlogCtl, pageno, true, xid);
+       ptr = (CommitSeqNo *) CsnlogCtl->shared->page_buffer[slotno];
+       ptr += entryno;
+
+       /* Current state should be 0 */
+       Assert(*ptr == COMMITSEQNO_INPROGRESS ||
+                  (*ptr == newcsn && overwriteOK));
+
+       *ptr = newcsn;
+
+       CsnlogCtl->shared->page_dirty[slotno] = true;
+
+       LWLockRelease(CSNLogControlLock);
+}
+
+/*
+ * Interrogate the parent of a transaction in the csnlog.
+ */
+TransactionId
+SubTransGetParent(TransactionId xid)
+{
+       CommitSeqNo csn;
+
+       csn = CSNLogGetCommitSeqNo(xid);
+
+       if (COMMITSEQNO_IS_SUBTRANS(csn))
+               return (TransactionId) (csn & 0xFFFFFFFF);
+       else
+               return InvalidTransactionId;
+}
+
+/*
+ * SubTransGetTopmostTransaction
+ *
+ * Returns the topmost transaction of the given transaction id.
+ *
+ * Because we cannot look back further than TransactionXmin, it is possible
+ * that this function will lie and return an intermediate subtransaction ID
+ * instead of the true topmost parent ID.  This is OK, because in practice
+ * we only care about detecting whether the topmost parent is still running
+ * or is part of a current snapshot's list of still-running transactions.
+ * Therefore, any XID before TransactionXmin is as good as any other.
+ */
+TransactionId
+SubTransGetTopmostTransaction(TransactionId xid)
+{
+       TransactionId parentXid = xid,
+                               previousXid = xid;
+
+       /* Can't ask about stuff that might not be around anymore */
+       Assert(TransactionIdFollowsOrEquals(xid, TransactionXmin));
+
+       while (TransactionIdIsValid(parentXid))
+       {
+               previousXid = parentXid;
+               if (TransactionIdPrecedes(parentXid, TransactionXmin))
+                       break;
+               parentXid = SubTransGetParent(parentXid);
+       }
+
+       Assert(TransactionIdIsValid(previousXid));
+
+       return previousXid;
+}
+
+
+
+
+/*
+ * Sets the commit status of a single transaction.
+ *
+ * Must be called with CSNLogControlLock held
+ */
+static void
+CSNLogSetCSN(TransactionId xid, CommitSeqNo csn, int slotno)
+{
+       int                     entryno = TransactionIdToPgIndex(xid);
+       CommitSeqNo *ptr;
+
+       ptr = (CommitSeqNo *) (CsnlogCtl->shared->page_buffer[slotno] + entryno * sizeof(XLogRecPtr));
+
+       /*
+        * Current state change should be from 0 to target state. (Allow
+        * setting it again to same value.)
+        */
+       Assert(COMMITSEQNO_IS_INPROGRESS(*ptr) ||
+                  COMMITSEQNO_IS_COMMITTING(*ptr) ||
+                  COMMITSEQNO_IS_SUBTRANS(*ptr) ||
+                  *ptr == csn);
+
+       *ptr = csn;
+}
+
+/*
+ * Interrogate the state of a transaction in the commit log.
+ *
+ * Aside from the actual commit status, this function returns (into *lsn)
+ * an LSN that is late enough to be able to guarantee that if we flush up to
+ * that LSN then we will have flushed the transaction's commit record to disk.
+ * The result is not necessarily the exact LSN of the transaction's commit
+ * record!     For example, for long-past transactions (those whose clog pages
+ * already migrated to disk), we'll return InvalidXLogRecPtr.  Also, because
+ * we group transactions on the same clog page to conserve storage, we might
+ * return the LSN of a later transaction that falls into the same group.
+ *
+ * NB: this is a low-level routine and is NOT the preferred entry point
+ * for most uses; TransactionLogFetch() in transam.c is the intended caller.
+ */
+CommitSeqNo
+CSNLogGetCommitSeqNo(TransactionId xid)
+{
+       int                     pageno = TransactionIdToPage(xid);
+       int                     entryno = TransactionIdToPgIndex(xid);
+       int                     slotno;
+       XLogRecPtr *ptr;
+       XLogRecPtr      commitlsn;
+
+       /* Can't ask about stuff that might not be around anymore */
+       Assert(TransactionIdFollowsOrEquals(xid, TransactionXmin));
+
+       if (!TransactionIdIsNormal(xid))
+       {
+               if (xid == InvalidTransactionId)
+                       return COMMITSEQNO_ABORTED;
+               if (xid == FrozenTransactionId || xid == BootstrapTransactionId)
+                       return COMMITSEQNO_FROZEN;
+       }
+
+       /* lock is acquired by SimpleLruReadPage_ReadOnly */
+
+       slotno = SimpleLruReadPage_ReadOnly(CsnlogCtl, pageno, xid);
+       ptr = (XLogRecPtr *) (CsnlogCtl->shared->page_buffer[slotno] + entryno * sizeof(XLogRecPtr));
+
+       commitlsn = *ptr;
+
+       LWLockRelease(CSNLogControlLock);
+
+       return commitlsn;
+}
+
+/*
+ * Number of shared CSNLOG buffers.
+ */
+Size
+CSNLOGShmemBuffers(void)
+{
+       return Min(32, Max(4, NBuffers / 512));
+}
+
+/*
+ * Initialization of shared memory for CSNLOG
+ */
+Size
+CSNLOGShmemSize(void)
+{
+       return SimpleLruShmemSize(CSNLOGShmemBuffers(), 0);
+}
+
+void
+CSNLOGShmemInit(void)
+{
+       CsnlogCtl->PagePrecedes = CSNLOGPagePrecedes;
+       SimpleLruInit(CsnlogCtl, "CSNLOG Ctl", CSNLOGShmemBuffers(), 0,
+                                 CSNLogControlLock, "pg_csnlog", LWTRANCHE_CSNLOG_BUFFERS);
+}
+
+/*
+ * This func must be called ONCE on system install.  It creates
+ * the initial CSNLOG segment.  (The pg_csnlog directory is assumed to
+ * have been created by initdb, and CSNLOGShmemInit must have been
+ * called already.)
+ */
+void
+BootStrapCSNLOG(void)
+{
+       int                     slotno;
+
+       LWLockAcquire(CSNLogControlLock, LW_EXCLUSIVE);
+
+       /* Create and zero the first page of the commit log */
+       slotno = ZeroCSNLOGPage(0);
+
+       /* Make sure it's written out */
+       SimpleLruWritePage(CsnlogCtl, slotno);
+       Assert(!CsnlogCtl->shared->page_dirty[slotno]);
+
+       LWLockRelease(CSNLogControlLock);
+}
+
+/*
+ * Initialize (or reinitialize) a page of CLOG to zeroes.
+ * If writeXlog is TRUE, also emit an XLOG record saying we did this.
+ *
+ * The page is not actually written, just set up in shared memory.
+ * The slot number of the new page is returned.
+ *
+ * Control lock must be held at entry, and will be held at exit.
+ */
+static int
+ZeroCSNLOGPage(int pageno)
+{
+       return SimpleLruZeroPage(CsnlogCtl, pageno);
+}
+
+/*
+ * This must be called ONCE during postmaster or standalone-backend startup,
+ * after StartupXLOG has initialized ShmemVariableCache->nextXid.
+ *
+ * oldestActiveXID is the oldest XID of any prepared transaction, or nextXid
+ * if there are none.
+ */
+void
+StartupCSNLOG(TransactionId oldestActiveXID)
+{
+       int                     startPage;
+       int                     endPage;
+
+       /*
+        * Since we don't expect pg_csnlog to be valid across crashes, we
+        * initialize the currently-active page(s) to zeroes during startup.
+        * Whenever we advance into a new page, ExtendCSNLOG will likewise zero
+        * the new page without regard to whatever was previously on disk.
+        */
+       LWLockAcquire(CSNLogControlLock, LW_EXCLUSIVE);
+
+       startPage = TransactionIdToPage(oldestActiveXID);
+       endPage = TransactionIdToPage(ShmemVariableCache->nextXid);
+
+       while (startPage != endPage)
+       {
+               (void) ZeroCSNLOGPage(startPage);
+               startPage++;
+               /* must account for wraparound */
+               if (startPage > TransactionIdToPage(MaxTransactionId))
+                       startPage = 0;
+       }
+       (void) ZeroCSNLOGPage(startPage);
+
+       LWLockRelease(CSNLogControlLock);
+}
+
+/*
+ * This must be called ONCE during postmaster or standalone-backend shutdown
+ */
+void
+ShutdownCSNLOG(void)
+{
+       /*
+        * Flush dirty CLOG pages to disk
+        *
+        * This is not actually necessary from a correctness point of view. We do
+        * it merely as a debugging aid.
+        */
+       TRACE_POSTGRESQL_CSNLOG_CHECKPOINT_START(false);
+       SimpleLruFlush(CsnlogCtl, false);
+       TRACE_POSTGRESQL_CSNLOG_CHECKPOINT_DONE(false);
+}
+
+/*
+ * This must be called ONCE at the end of startup/recovery.
+ */
+void
+TrimCSNLOG(void)
+{
+       TransactionId xid = ShmemVariableCache->nextXid;
+       int                     pageno = TransactionIdToPage(xid);
+
+       LWLockAcquire(CSNLogControlLock, LW_EXCLUSIVE);
+
+       /*
+        * Re-Initialize our idea of the latest page number.
+        */
+       CsnlogCtl->shared->latest_page_number = pageno;
+
+       /*
+        * Zero out the remainder of the current clog page.  Under normal
+        * circumstances it should be zeroes already, but it seems at least
+        * theoretically possible that XLOG replay will have settled on a nextXID
+        * value that is less than the last XID actually used and marked by the
+        * previous database lifecycle (since subtransaction commit writes clog
+        * but makes no WAL entry).  Let's just be safe. (We need not worry about
+        * pages beyond the current one, since those will be zeroed when first
+        * used.  For the same reason, there is no need to do anything when
+        * nextXid is exactly at a page boundary; and it's likely that the
+        * "current" page doesn't exist yet in that case.)
+        */
+       if (TransactionIdToPgIndex(xid) != 0)
+       {
+               int                     entryno = TransactionIdToPgIndex(xid);
+               int                     byteno = entryno * sizeof(XLogRecPtr);
+               int                     slotno;
+               char       *byteptr;
+
+               slotno = SimpleLruReadPage(CsnlogCtl, pageno, false, xid);
+
+               byteptr = CsnlogCtl->shared->page_buffer[slotno] + byteno;
+
+               /* Zero the rest of the page */
+               MemSet(byteptr, 0, BLCKSZ - byteno);
+
+               CsnlogCtl->shared->page_dirty[slotno] = true;
+       }
+
+       LWLockRelease(CSNLogControlLock);
+}
+
+/*
+ * Perform a checkpoint --- either during shutdown, or on-the-fly
+ */
+void
+CheckPointCSNLOG(void)
+{
+       /*
+        * Flush dirty CLOG pages to disk
+        *
+        * This is not actually necessary from a correctness point of view. We do
+        * it merely to improve the odds that writing of dirty pages is done by
+        * the checkpoint process and not by backends.
+        */
+       TRACE_POSTGRESQL_CSNLOG_CHECKPOINT_START(true);
+       SimpleLruFlush(CsnlogCtl, true);
+       TRACE_POSTGRESQL_CSNLOG_CHECKPOINT_DONE(true);
+}
+
+
+/*
+ * Make sure that CSNLOG has room for a newly-allocated XID.
+ *
+ * NB: this is called while holding XidGenLock.  We want it to be very fast
+ * most of the time; even when it's not so fast, no actual I/O need happen
+ * unless we're forced to write out a dirty clog or xlog page to make room
+ * in shared memory.
+ */
+void
+ExtendCSNLOG(TransactionId newestXact)
+{
+       int                     pageno;
+
+       /*
+        * No work except at first XID of a page.  But beware: just after
+        * wraparound, the first XID of page zero is FirstNormalTransactionId.
+        */
+       if (TransactionIdToPgIndex(newestXact) != 0 &&
+               !TransactionIdEquals(newestXact, FirstNormalTransactionId))
+               return;
+
+       pageno = TransactionIdToPage(newestXact);
+
+       LWLockAcquire(CSNLogControlLock, LW_EXCLUSIVE);
+
+       /* Zero the page and make an XLOG entry about it */
+       ZeroCSNLOGPage(pageno);
+
+       LWLockRelease(CSNLogControlLock);
+}
+
+
+/*
+ * Remove all CSNLOG segments before the one holding the passed transaction ID
+ *
+ * This is normally called during checkpoint, with oldestXact being the
+ * oldest TransactionXmin of any running transaction.
+ */
+void
+TruncateCSNLOG(TransactionId oldestXact)
+{
+       int                     cutoffPage;
+
+       /*
+        * The cutoff point is the start of the segment containing oldestXact. We
+        * pass the *page* containing oldestXact to SimpleLruTruncate.
+        */
+       cutoffPage = TransactionIdToPage(oldestXact);
+
+       SimpleLruTruncate(CsnlogCtl, cutoffPage);
+}
+
+
+/*
+ * Decide which of two CLOG page numbers is "older" for truncation purposes.
+ *
+ * We need to use comparison of TransactionIds here in order to do the right
+ * thing with wraparound XID arithmetic.  However, if we are asked about
+ * page number zero, we don't want to hand InvalidTransactionId to
+ * TransactionIdPrecedes: it'll get weird about permanent xact IDs.  So,
+ * offset both xids by FirstNormalTransactionId to avoid that.
+ */
+static bool
+CSNLOGPagePrecedes(int page1, int page2)
+{
+       TransactionId xid1;
+       TransactionId xid2;
+
+       xid1 = ((TransactionId) page1) * CSNLOG_XACTS_PER_PAGE;
+       xid1 += FirstNormalTransactionId;
+       xid2 = ((TransactionId) page2) * CSNLOG_XACTS_PER_PAGE;
+       xid2 += FirstNormalTransactionId;
+
+       return TransactionIdPrecedes(xid1, xid2);
+}
index 0c8c17af33c5b8321ccdea64367bae55923a317b..9cc6d3dffda36041c5b819be56d25da0e06064d7 100644 (file)
@@ -69,6 +69,7 @@
 #include "postgres.h"
 
 #include "access/multixact.h"
+#include "access/mvccvars.h"
 #include "access/slru.h"
 #include "access/transam.h"
 #include "access/twophase.h"
@@ -513,9 +514,11 @@ MultiXactIdExpand(MultiXactId multi, TransactionId xid, MultiXactStatus status)
 
        for (i = 0, j = 0; i < nmembers; i++)
        {
-               if (TransactionIdIsInProgress(members[i].xid) ||
+               TransactionIdStatus xidstatus = TransactionIdGetStatus(members[i].xid);
+
+               if (xidstatus == XID_INPROGRESS ||
                        (ISUPDATE_from_mxstatus(members[i].status) &&
-                        TransactionIdDidCommit(members[i].xid)))
+                        xidstatus == XID_COMMITTED))
                {
                        newMembers[j].xid = members[i].xid;
                        newMembers[j++].status = members[i].status;
@@ -590,7 +593,7 @@ MultiXactIdIsRunning(MultiXactId multi, bool isLockOnly)
         */
        for (i = 0; i < nmembers; i++)
        {
-               if (TransactionIdIsInProgress(members[i].xid))
+               if (TransactionIdGetStatus(members[i].xid) == XID_INPROGRESS)
                {
                        debug_elog4(DEBUG2, "IsRunning: member %d (%u) is running",
                                                i, members[i].xid);
diff --git a/src/backend/access/transam/subtrans.c b/src/backend/access/transam/subtrans.c
deleted file mode 100644 (file)
index 908fe2d..0000000
+++ /dev/null
@@ -1,382 +0,0 @@
-/*-------------------------------------------------------------------------
- *
- * subtrans.c
- *             PostgreSQL subtransaction-log manager
- *
- * The pg_subtrans manager is a pg_clog-like manager that stores the parent
- * transaction Id for each transaction.  It is a fundamental part of the
- * nested transactions implementation.  A main transaction has a parent
- * of InvalidTransactionId, and each subtransaction has its immediate parent.
- * The tree can easily be walked from child to parent, but not in the
- * opposite direction.
- *
- * This code is based on clog.c, but the robustness requirements
- * are completely different from pg_clog, because we only need to remember
- * pg_subtrans information for currently-open transactions.  Thus, there is
- * no need to preserve data over a crash and restart.
- *
- * There are no XLOG interactions since we do not care about preserving
- * data across crashes.  During database startup, we simply force the
- * currently-active page of SUBTRANS to zeroes.
- *
- * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group
- * Portions Copyright (c) 1994, Regents of the University of California
- *
- * src/backend/access/transam/subtrans.c
- *
- *-------------------------------------------------------------------------
- */
-#include "postgres.h"
-
-#include "access/slru.h"
-#include "access/subtrans.h"
-#include "access/transam.h"
-#include "pg_trace.h"
-#include "utils/snapmgr.h"
-
-
-/*
- * Defines for SubTrans page sizes.  A page is the same BLCKSZ as is used
- * everywhere else in Postgres.
- *
- * Note: because TransactionIds are 32 bits and wrap around at 0xFFFFFFFF,
- * SubTrans page numbering also wraps around at
- * 0xFFFFFFFF/SUBTRANS_XACTS_PER_PAGE, and segment numbering at
- * 0xFFFFFFFF/SUBTRANS_XACTS_PER_PAGE/SLRU_PAGES_PER_SEGMENT.  We need take no
- * explicit notice of that fact in this module, except when comparing segment
- * and page numbers in TruncateSUBTRANS (see SubTransPagePrecedes) and zeroing
- * them in StartupSUBTRANS.
- */
-
-/* We need four bytes per xact */
-#define SUBTRANS_XACTS_PER_PAGE (BLCKSZ / sizeof(TransactionId))
-
-#define TransactionIdToPage(xid) ((xid) / (TransactionId) SUBTRANS_XACTS_PER_PAGE)
-#define TransactionIdToEntry(xid) ((xid) % (TransactionId) SUBTRANS_XACTS_PER_PAGE)
-
-
-/*
- * Link to shared-memory data structures for SUBTRANS control
- */
-static SlruCtlData SubTransCtlData;
-
-#define SubTransCtl  (&SubTransCtlData)
-
-
-static int     ZeroSUBTRANSPage(int pageno);
-static bool SubTransPagePrecedes(int page1, int page2);
-
-
-/*
- * Record the parent of a subtransaction in the subtrans log.
- *
- * In some cases we may need to overwrite an existing value.
- */
-void
-SubTransSetParent(TransactionId xid, TransactionId parent, bool overwriteOK)
-{
-       int                     pageno = TransactionIdToPage(xid);
-       int                     entryno = TransactionIdToEntry(xid);
-       int                     slotno;
-       TransactionId *ptr;
-
-       Assert(TransactionIdIsValid(parent));
-
-       LWLockAcquire(SubtransControlLock, LW_EXCLUSIVE);
-
-       slotno = SimpleLruReadPage(SubTransCtl, pageno, true, xid);
-       ptr = (TransactionId *) SubTransCtl->shared->page_buffer[slotno];
-       ptr += entryno;
-
-       /* Current state should be 0 */
-       Assert(*ptr == InvalidTransactionId ||
-                  (*ptr == parent && overwriteOK));
-
-       *ptr = parent;
-
-       SubTransCtl->shared->page_dirty[slotno] = true;
-
-       LWLockRelease(SubtransControlLock);
-}
-
-/*
- * Interrogate the parent of a transaction in the subtrans log.
- */
-TransactionId
-SubTransGetParent(TransactionId xid)
-{
-       int                     pageno = TransactionIdToPage(xid);
-       int                     entryno = TransactionIdToEntry(xid);
-       int                     slotno;
-       TransactionId *ptr;
-       TransactionId parent;
-
-       /* Can't ask about stuff that might not be around anymore */
-       Assert(TransactionIdFollowsOrEquals(xid, TransactionXmin));
-
-       /* Bootstrap and frozen XIDs have no parent */
-       if (!TransactionIdIsNormal(xid))
-               return InvalidTransactionId;
-
-       /* lock is acquired by SimpleLruReadPage_ReadOnly */
-
-       slotno = SimpleLruReadPage_ReadOnly(SubTransCtl, pageno, xid);
-       ptr = (TransactionId *) SubTransCtl->shared->page_buffer[slotno];
-       ptr += entryno;
-
-       parent = *ptr;
-
-       LWLockRelease(SubtransControlLock);
-
-       return parent;
-}
-
-/*
- * SubTransGetTopmostTransaction
- *
- * Returns the topmost transaction of the given transaction id.
- *
- * Because we cannot look back further than TransactionXmin, it is possible
- * that this function will lie and return an intermediate subtransaction ID
- * instead of the true topmost parent ID.  This is OK, because in practice
- * we only care about detecting whether the topmost parent is still running
- * or is part of a current snapshot's list of still-running transactions.
- * Therefore, any XID before TransactionXmin is as good as any other.
- */
-TransactionId
-SubTransGetTopmostTransaction(TransactionId xid)
-{
-       TransactionId parentXid = xid,
-                               previousXid = xid;
-
-       /* Can't ask about stuff that might not be around anymore */
-       Assert(TransactionIdFollowsOrEquals(xid, TransactionXmin));
-
-       while (TransactionIdIsValid(parentXid))
-       {
-               previousXid = parentXid;
-               if (TransactionIdPrecedes(parentXid, TransactionXmin))
-                       break;
-               parentXid = SubTransGetParent(parentXid);
-       }
-
-       Assert(TransactionIdIsValid(previousXid));
-
-       return previousXid;
-}
-
-
-/*
- * Initialization of shared memory for SUBTRANS
- */
-Size
-SUBTRANSShmemSize(void)
-{
-       return SimpleLruShmemSize(NUM_SUBTRANS_BUFFERS, 0);
-}
-
-void
-SUBTRANSShmemInit(void)
-{
-       SubTransCtl->PagePrecedes = SubTransPagePrecedes;
-       SimpleLruInit(SubTransCtl, "subtrans", NUM_SUBTRANS_BUFFERS, 0,
-                                 SubtransControlLock, "pg_subtrans",
-                                 LWTRANCHE_SUBTRANS_BUFFERS);
-       /* Override default assumption that writes should be fsync'd */
-       SubTransCtl->do_fsync = false;
-}
-
-/*
- * This func must be called ONCE on system install.  It creates
- * the initial SUBTRANS segment.  (The SUBTRANS directory is assumed to
- * have been created by the initdb shell script, and SUBTRANSShmemInit
- * must have been called already.)
- *
- * Note: it's not really necessary to create the initial segment now,
- * since slru.c would create it on first write anyway.  But we may as well
- * do it to be sure the directory is set up correctly.
- */
-void
-BootStrapSUBTRANS(void)
-{
-       int                     slotno;
-
-       LWLockAcquire(SubtransControlLock, LW_EXCLUSIVE);
-
-       /* Create and zero the first page of the subtrans log */
-       slotno = ZeroSUBTRANSPage(0);
-
-       /* Make sure it's written out */
-       SimpleLruWritePage(SubTransCtl, slotno);
-       Assert(!SubTransCtl->shared->page_dirty[slotno]);
-
-       LWLockRelease(SubtransControlLock);
-}
-
-/*
- * Initialize (or reinitialize) a page of SUBTRANS to zeroes.
- *
- * The page is not actually written, just set up in shared memory.
- * The slot number of the new page is returned.
- *
- * Control lock must be held at entry, and will be held at exit.
- */
-static int
-ZeroSUBTRANSPage(int pageno)
-{
-       return SimpleLruZeroPage(SubTransCtl, pageno);
-}
-
-/*
- * This must be called ONCE during postmaster or standalone-backend startup,
- * after StartupXLOG has initialized ShmemVariableCache->nextXid.
- *
- * oldestActiveXID is the oldest XID of any prepared transaction, or nextXid
- * if there are none.
- */
-void
-StartupSUBTRANS(TransactionId oldestActiveXID)
-{
-       int                     startPage;
-       int                     endPage;
-
-       /*
-        * Since we don't expect pg_subtrans to be valid across crashes, we
-        * initialize the currently-active page(s) to zeroes during startup.
-        * Whenever we advance into a new page, ExtendSUBTRANS will likewise zero
-        * the new page without regard to whatever was previously on disk.
-        */
-       LWLockAcquire(SubtransControlLock, LW_EXCLUSIVE);
-
-       startPage = TransactionIdToPage(oldestActiveXID);
-       endPage = TransactionIdToPage(ShmemVariableCache->nextXid);
-
-       while (startPage != endPage)
-       {
-               (void) ZeroSUBTRANSPage(startPage);
-               startPage++;
-               /* must account for wraparound */
-               if (startPage > TransactionIdToPage(MaxTransactionId))
-                       startPage = 0;
-       }
-       (void) ZeroSUBTRANSPage(startPage);
-
-       LWLockRelease(SubtransControlLock);
-}
-
-/*
- * This must be called ONCE during postmaster or standalone-backend shutdown
- */
-void
-ShutdownSUBTRANS(void)
-{
-       /*
-        * Flush dirty SUBTRANS pages to disk
-        *
-        * This is not actually necessary from a correctness point of view. We do
-        * it merely as a debugging aid.
-        */
-       TRACE_POSTGRESQL_SUBTRANS_CHECKPOINT_START(false);
-       SimpleLruFlush(SubTransCtl, false);
-       TRACE_POSTGRESQL_SUBTRANS_CHECKPOINT_DONE(false);
-}
-
-/*
- * Perform a checkpoint --- either during shutdown, or on-the-fly
- */
-void
-CheckPointSUBTRANS(void)
-{
-       /*
-        * Flush dirty SUBTRANS pages to disk
-        *
-        * This is not actually necessary from a correctness point of view. We do
-        * it merely to improve the odds that writing of dirty pages is done by
-        * the checkpoint process and not by backends.
-        */
-       TRACE_POSTGRESQL_SUBTRANS_CHECKPOINT_START(true);
-       SimpleLruFlush(SubTransCtl, true);
-       TRACE_POSTGRESQL_SUBTRANS_CHECKPOINT_DONE(true);
-}
-
-
-/*
- * Make sure that SUBTRANS has room for a newly-allocated XID.
- *
- * NB: this is called while holding XidGenLock.  We want it to be very fast
- * most of the time; even when it's not so fast, no actual I/O need happen
- * unless we're forced to write out a dirty subtrans page to make room
- * in shared memory.
- */
-void
-ExtendSUBTRANS(TransactionId newestXact)
-{
-       int                     pageno;
-
-       /*
-        * No work except at first XID of a page.  But beware: just after
-        * wraparound, the first XID of page zero is FirstNormalTransactionId.
-        */
-       if (TransactionIdToEntry(newestXact) != 0 &&
-               !TransactionIdEquals(newestXact, FirstNormalTransactionId))
-               return;
-
-       pageno = TransactionIdToPage(newestXact);
-
-       LWLockAcquire(SubtransControlLock, LW_EXCLUSIVE);
-
-       /* Zero the page */
-       ZeroSUBTRANSPage(pageno);
-
-       LWLockRelease(SubtransControlLock);
-}
-
-
-/*
- * Remove all SUBTRANS segments before the one holding the passed transaction ID
- *
- * This is normally called during checkpoint, with oldestXact being the
- * oldest TransactionXmin of any running transaction.
- */
-void
-TruncateSUBTRANS(TransactionId oldestXact)
-{
-       int                     cutoffPage;
-
-       /*
-        * The cutoff point is the start of the segment containing oldestXact. We
-        * pass the *page* containing oldestXact to SimpleLruTruncate.  We step
-        * back one transaction to avoid passing a cutoff page that hasn't been
-        * created yet in the rare case that oldestXact would be the first item on
-        * a page and oldestXact == next XID.  In that case, if we didn't subtract
-        * one, we'd trigger SimpleLruTruncate's wraparound detection.
-        */
-       TransactionIdRetreat(oldestXact);
-       cutoffPage = TransactionIdToPage(oldestXact);
-
-       SimpleLruTruncate(SubTransCtl, cutoffPage);
-}
-
-
-/*
- * Decide which of two SUBTRANS page numbers is "older" for truncation purposes.
- *
- * We need to use comparison of TransactionIds here in order to do the right
- * thing with wraparound XID arithmetic.  However, if we are asked about
- * page number zero, we don't want to hand InvalidTransactionId to
- * TransactionIdPrecedes: it'll get weird about permanent xact IDs.  So,
- * offset both xids by FirstNormalTransactionId to avoid that.
- */
-static bool
-SubTransPagePrecedes(int page1, int page2)
-{
-       TransactionId xid1;
-       TransactionId xid2;
-
-       xid1 = ((TransactionId) page1) * SUBTRANS_XACTS_PER_PAGE;
-       xid1 += FirstNormalTransactionId;
-       xid2 = ((TransactionId) page2) * SUBTRANS_XACTS_PER_PAGE;
-       xid2 += FirstNormalTransactionId;
-
-       return TransactionIdPrecedes(xid1, xid2);
-}
index 1eba49a94b321b83185f0a6f9ac4ab1914ad6c63..e4ce0d51490e1992a68861bc4a488b20309aa78c 100644 (file)
@@ -3,6 +3,15 @@
  * transam.c
  *       postgres transaction log interface routines
  *
+ * This module contains high level functions for managing the status
+ * of transactions. It sits on top of two lower level structures: the
+ * CLOG, and the CSNLOG. The CLOG is a permanent on-disk structure that
+ * tracks the committed/aborted status for each transaction ID. The CSNLOG
+ * tracks *when* each transaction ID committed (or aborted). The CSNLOG
+ * is used when checking the status of recent transactions that might still
+ * be in-progress, and it is reset at server startup. The CLOG is used for
+ * older transactions that are known to have completed (or crashed).
+ *
  * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
  *
  * IDENTIFICATION
  *       src/backend/access/transam/transam.c
  *
- * NOTES
- *       This file contains the high level access-method interface to the
- *       transaction system.
- *
  *-------------------------------------------------------------------------
  */
 
 #include "postgres.h"
 
 #include "access/clog.h"
+#include "access/csnlog.h"
+#include "access/mvccvars.h"
 #include "access/subtrans.h"
 #include "access/transam.h"
+#include "storage/lmgr.h"
 #include "utils/snapmgr.h"
 
 /*
- * Single-item cache for results of TransactionLogFetch.  It's worth having
+ * Single-item cache for results of TransactionIdGetCommitSeqNo.  It's worth
+ * having
  * such a cache because we frequently find ourselves repeatedly checking the
  * same XID, for example when scanning a table just after a bulk insert,
  * update, or delete.
  */
 static TransactionId cachedFetchXid = InvalidTransactionId;
-static XidStatus cachedFetchXidStatus;
-static XLogRecPtr cachedCommitLSN;
+static CommitSeqNo cachedCSN;
 
-/* Local functions */
-static XidStatus TransactionLogFetch(TransactionId transactionId);
-
-
-/* ----------------------------------------------------------------
- *             Postgres log access method interface
- *
- *             TransactionLogFetch
- * ----------------------------------------------------------------
+/*
+ * Also have a (separate) cache for CLogGetCommitLSN()
  */
+static TransactionId cachedLSNFetchXid = InvalidTransactionId;
+static XLogRecPtr cachedCommitLSN;
 
 /*
- * TransactionLogFetch --- fetch commit status of specified transaction id
+ * TransactionIdGetCommitSeqNo --- fetch CSN of specified transaction id
  */
-static XidStatus
-TransactionLogFetch(TransactionId transactionId)
+CommitSeqNo
+TransactionIdGetCommitSeqNo(TransactionId transactionId)
 {
-       XidStatus       xidstatus;
-       XLogRecPtr      xidlsn;
+       CommitSeqNo     csn;
 
        /*
         * Before going to the commit log manager, check our single item cache to
         * see if we didn't just check the transaction status a moment ago.
         */
        if (TransactionIdEquals(transactionId, cachedFetchXid))
-               return cachedFetchXidStatus;
+               return cachedCSN;
 
        /*
         * Also, check to see if the transaction ID is a permanent one.
@@ -67,53 +69,63 @@ TransactionLogFetch(TransactionId transactionId)
        if (!TransactionIdIsNormal(transactionId))
        {
                if (TransactionIdEquals(transactionId, BootstrapTransactionId))
-                       return TRANSACTION_STATUS_COMMITTED;
+                       return COMMITSEQNO_FROZEN;
                if (TransactionIdEquals(transactionId, FrozenTransactionId))
-                       return TRANSACTION_STATUS_COMMITTED;
-               return TRANSACTION_STATUS_ABORTED;
+                       return COMMITSEQNO_FROZEN;
+               return COMMITSEQNO_ABORTED;
        }
 
        /*
-        * Get the transaction status.
+        * If the XID is older than TransactionXmin, check the clog. Otherwise
+        * check the csnlog.
         */
-       xidstatus = TransactionIdGetStatus(transactionId, &xidlsn);
+       Assert(TransactionIdIsValid(TransactionXmin));
+       if (TransactionIdPrecedes(transactionId, TransactionXmin))
+       {
+               XLogRecPtr lsn;
+
+               if (CLogGetStatus(transactionId, &lsn) == CLOG_XID_STATUS_COMMITTED)
+                       csn = COMMITSEQNO_FROZEN;
+               else
+                       csn = COMMITSEQNO_ABORTED;
+       }
+       else
+       {
+               csn = CSNLogGetCommitSeqNo(transactionId);
+
+               if (csn == COMMITSEQNO_COMMITTING)
+               {
+                       /*
+                        * If the transaction is committing at this very instant, and
+                        * hasn't set its CSN yet, wait for it to finish doing so.
+                        *
+                        * XXX: Alternatively, we could wait on the heavy-weight lock on
+                        * the XID. that'd make TransactionIdCommitTree() slightly
+                        * cheaper, as it wouldn't need to acquire CommitSeqNoLock (even
+                        * in shared mode).
+                        */
+                       LWLockAcquire(CommitSeqNoLock, LW_EXCLUSIVE);
+                       LWLockRelease(CommitSeqNoLock);
+
+                       csn = CSNLogGetCommitSeqNo(transactionId);
+                       Assert(csn != COMMITSEQNO_COMMITTING);
+               }
+       }
 
        /*
-        * Cache it, but DO NOT cache status for unfinished or sub-committed
-        * transactions!  We only cache status that is guaranteed not to change.
+        * Cache it, but DO NOT cache status for unfinished transactions!
+        * We only cache status that is guaranteed not to change.
         */
-       if (xidstatus != TRANSACTION_STATUS_IN_PROGRESS &&
-               xidstatus != TRANSACTION_STATUS_SUB_COMMITTED)
+       if (COMMITSEQNO_IS_COMMITTED(csn) ||
+               COMMITSEQNO_IS_ABORTED(csn))
        {
                cachedFetchXid = transactionId;
-               cachedFetchXidStatus = xidstatus;
-               cachedCommitLSN = xidlsn;
+               cachedCSN = csn;
        }
 
-       return xidstatus;
+       return csn;
 }
 
-/* ----------------------------------------------------------------
- *                                             Interface functions
- *
- *             TransactionIdDidCommit
- *             TransactionIdDidAbort
- *             ========
- *                these functions test the transaction status of
- *                a specified transaction id.
- *
- *             TransactionIdCommitTree
- *             TransactionIdAsyncCommitTree
- *             TransactionIdAbortTree
- *             ========
- *                these functions set the transaction status of the specified
- *                transaction tree.
- *
- * See also TransactionIdIsInProgress, which once was in this module
- * but now lives in procarray.c.
- * ----------------------------------------------------------------
- */
-
 /*
  * TransactionIdDidCommit
  *             True iff transaction associated with the identifier did commit.
@@ -124,50 +136,14 @@ TransactionLogFetch(TransactionId transactionId)
 bool                                                   /* true if given transaction committed */
 TransactionIdDidCommit(TransactionId transactionId)
 {
-       XidStatus       xidstatus;
+       CommitSeqNo csn;
 
-       xidstatus = TransactionLogFetch(transactionId);
+       csn = TransactionIdGetCommitSeqNo(transactionId);
 
-       /*
-        * If it's marked committed, it's committed.
-        */
-       if (xidstatus == TRANSACTION_STATUS_COMMITTED)
+       if (COMMITSEQNO_IS_COMMITTED(csn))
                return true;
-
-       /*
-        * If it's marked subcommitted, we have to check the parent recursively.
-        * However, if it's older than TransactionXmin, we can't look at
-        * pg_subtrans; instead assume that the parent crashed without cleaning up
-        * its children.
-        *
-        * Originally we Assert'ed that the result of SubTransGetParent was not
-        * zero. However with the introduction of prepared transactions, there can
-        * be a window just after database startup where we do not have complete
-        * knowledge in pg_subtrans of the transactions after TransactionXmin.
-        * StartupSUBTRANS() has ensured that any missing information will be
-        * zeroed.  Since this case should not happen under normal conditions, it
-        * seems reasonable to emit a WARNING for it.
-        */
-       if (xidstatus == TRANSACTION_STATUS_SUB_COMMITTED)
-       {
-               TransactionId parentXid;
-
-               if (TransactionIdPrecedes(transactionId, TransactionXmin))
-                       return false;
-               parentXid = SubTransGetParent(transactionId);
-               if (!TransactionIdIsValid(parentXid))
-               {
-                       elog(WARNING, "no pg_subtrans entry for subcommitted XID %u",
-                                transactionId);
-                       return false;
-               }
-               return TransactionIdDidCommit(parentXid);
-       }
-
-       /*
-        * It's not committed.
-        */
-       return false;
+       else
+               return false;
 }
 
 /*
@@ -180,70 +156,35 @@ TransactionIdDidCommit(TransactionId transactionId)
 bool                                                   /* true if given transaction aborted */
 TransactionIdDidAbort(TransactionId transactionId)
 {
-       XidStatus       xidstatus;
+       CommitSeqNo csn;
 
-       xidstatus = TransactionLogFetch(transactionId);
+       csn = TransactionIdGetCommitSeqNo(transactionId);
 
-       /*
-        * If it's marked aborted, it's aborted.
-        */
-       if (xidstatus == TRANSACTION_STATUS_ABORTED)
+       if (COMMITSEQNO_IS_ABORTED(csn))
                return true;
-
-       /*
-        * If it's marked subcommitted, we have to check the parent recursively.
-        * However, if it's older than TransactionXmin, we can't look at
-        * pg_subtrans; instead assume that the parent crashed without cleaning up
-        * its children.
-        */
-       if (xidstatus == TRANSACTION_STATUS_SUB_COMMITTED)
-       {
-               TransactionId parentXid;
-
-               if (TransactionIdPrecedes(transactionId, TransactionXmin))
-                       return true;
-               parentXid = SubTransGetParent(transactionId);
-               if (!TransactionIdIsValid(parentXid))
-               {
-                       /* see notes in TransactionIdDidCommit */
-                       elog(WARNING, "no pg_subtrans entry for subcommitted XID %u",
-                                transactionId);
-                       return true;
-               }
-               return TransactionIdDidAbort(parentXid);
-       }
-
-       /*
-        * It's not aborted.
-        */
-       return false;
+       else
+               return false;
 }
 
 /*
- * TransactionIdIsKnownCompleted
- *             True iff transaction associated with the identifier is currently
- *             known to have either committed or aborted.
- *
- * This does NOT look into pg_clog but merely probes our local cache
- * (and so it's not named TransactionIdDidComplete, which would be the
- * appropriate name for a function that worked that way).  The intended
- * use is just to short-circuit TransactionIdIsInProgress calls when doing
- * repeated tqual.c checks for the same XID.  If this isn't extremely fast
- * then it will be counterproductive.
+ * Returns the status of the tranaction.
  *
- * Note:
- *             Assumes transaction identifier is valid.
+ * Note that this treats a a crashed transaction as still in-progress,
+ * until it falls off the xmin horizon.
  */
-bool
-TransactionIdIsKnownCompleted(TransactionId transactionId)
+TransactionIdStatus
+TransactionIdGetStatus(TransactionId xid)
 {
-       if (TransactionIdEquals(transactionId, cachedFetchXid))
-       {
-               /* If it's in the cache at all, it must be completed. */
-               return true;
-       }
+       CommitSeqNo csn;
+
+       csn = TransactionIdGetCommitSeqNo(xid);
 
-       return false;
+       if (COMMITSEQNO_IS_COMMITTED(csn))
+               return XID_COMMITTED;
+       else if (COMMITSEQNO_IS_ABORTED(csn))
+               return XID_ABORTED;
+       else
+               return XID_INPROGRESS;
 }
 
 /*
@@ -252,28 +193,80 @@ TransactionIdIsKnownCompleted(TransactionId transactionId)
  *
  * "xid" is a toplevel transaction commit, and the xids array contains its
  * committed subtransactions.
- *
- * This commit operation is not guaranteed to be atomic, but if not, subxids
- * are correctly marked subcommit first.
  */
 void
 TransactionIdCommitTree(TransactionId xid, int nxids, TransactionId *xids)
 {
-       TransactionIdSetTreeStatus(xid, nxids, xids,
-                                                          TRANSACTION_STATUS_COMMITTED,
-                                                          InvalidXLogRecPtr);
+       TransactionIdAsyncCommitTree(xid, nxids, xids, InvalidXLogRecPtr);
 }
 
 /*
  * TransactionIdAsyncCommitTree
- *             Same as above, but for async commits.  The commit record LSN is needed.
+ *             Same as above, but for async commits.
+ *
+ * "xid" is a toplevel transaction commit, and the xids array contains its
+ * committed subtransactions.
  */
 void
 TransactionIdAsyncCommitTree(TransactionId xid, int nxids, TransactionId *xids,
                                                         XLogRecPtr lsn)
 {
-       TransactionIdSetTreeStatus(xid, nxids, xids,
-                                                          TRANSACTION_STATUS_COMMITTED, lsn);
+       CommitSeqNo csn;
+       TransactionId latestXid;
+       TransactionId currentLatestCompletedXid;
+
+       latestXid = TransactionIdLatest(xid, nxids, xids);
+
+       /*
+        * Grab the CommitSeqNoLock, in shared mode. This is only used to
+        * provide a way for a concurrent transaction to wait for us to
+        * complete (see TransactionIdGetCommitSeqNo()).
+        *
+        * XXX: We could reduce the time the lock is held, by only setting
+        * the CSN on the top-XID while holding the lock, and updating the
+        * sub-XIDs later. But it doesn't matter much, because we're only
+        * holding it in shared mode, and it's rare for it to be acquired
+        * in exclusive mode.
+        */
+       LWLockAcquire(CommitSeqNoLock, LW_SHARED);
+
+       /*
+        * First update latestCompletedXid to cover this xid. We do this before
+        * assigning a CSN, so that if someone acquires a new snapshot at the same
+        * time, the xmax it computes is sure to cover our XID.
+        */
+       currentLatestCompletedXid = pg_atomic_read_u32(&ShmemVariableCache->latestCompletedXid);
+       while (TransactionIdFollows(latestXid, currentLatestCompletedXid))
+       {
+               if (pg_atomic_compare_exchange_u32(&ShmemVariableCache->latestCompletedXid,
+                                                                                  &currentLatestCompletedXid,
+                                                                                  latestXid))
+                       break;
+       }
+
+       /*
+        * Mark our top transaction id as commit-in-progress.
+        */
+       CSNLogSetCommitSeqNo(xid, 0, NULL, COMMITSEQNO_COMMITTING);
+
+       /* Get our CSN and increment */
+       csn = pg_atomic_fetch_add_u64(&ShmemVariableCache->nextCommitSeqNo, 1);
+       Assert(csn >= COMMITSEQNO_FIRST_NORMAL);
+
+       /* Stamp this XID (and sub-XIDs) with the CSN */
+       CSNLogSetCommitSeqNo(xid, nxids, xids, csn);
+
+       LWLockRelease(CommitSeqNoLock);
+
+       /*
+        * Also update the CLOG. This doesn't need to happen atomically with
+        * updating the CSN log, because no-one will look at the CLOG until
+        * GlobalXmin has advanced past our XID, and that can't happen until
+        * we clear the XID from the proc array.
+        */
+       CLogSetTreeStatus(xid, nxids, xids,
+                                         CLOG_XID_STATUS_COMMITTED,
+                                         lsn);
 }
 
 /*
@@ -289,8 +282,23 @@ TransactionIdAsyncCommitTree(TransactionId xid, int nxids, TransactionId *xids,
 void
 TransactionIdAbortTree(TransactionId xid, int nxids, TransactionId *xids)
 {
-       TransactionIdSetTreeStatus(xid, nxids, xids,
-                                                          TRANSACTION_STATUS_ABORTED, InvalidXLogRecPtr);
+       TransactionId latestXid;
+       TransactionId currentLatestCompletedXid;
+
+       latestXid = TransactionIdLatest(xid, nxids, xids);
+
+       currentLatestCompletedXid = pg_atomic_read_u32(&ShmemVariableCache->latestCompletedXid);
+       while (TransactionIdFollows(latestXid, currentLatestCompletedXid))
+       {
+               if (pg_atomic_compare_exchange_u32(&ShmemVariableCache->latestCompletedXid,
+                                                                                  &currentLatestCompletedXid,
+                                                                                  latestXid))
+                       break;
+       }
+
+       CSNLogSetCommitSeqNo(xid, nxids, xids, COMMITSEQNO_ABORTED);
+       CLogSetTreeStatus(xid, nxids, xids,
+                                         CLOG_XID_STATUS_ABORTED, InvalidCommitSeqNo);
 }
 
 /*
@@ -409,7 +417,7 @@ TransactionIdGetCommitLSN(TransactionId xid)
         * checking TransactionLogFetch's cache will usually succeed and avoid an
         * extra trip to shared memory.
         */
-       if (TransactionIdEquals(xid, cachedFetchXid))
+       if (TransactionIdEquals(xid, cachedLSNFetchXid))
                return cachedCommitLSN;
 
        /* Special XIDs are always known committed */
@@ -419,7 +427,10 @@ TransactionIdGetCommitLSN(TransactionId xid)
        /*
         * Get the transaction status.
         */
-       (void) TransactionIdGetStatus(xid, &result);
+       (void) CLogGetStatus(xid, &result);
+
+       cachedLSNFetchXid = xid;
+       cachedCommitLSN = result;
 
        return result;
 }
index 9f55adcaf5ef0e50fe1ea93e4e725c5c7b794215..3aa91572d52b8172e9e3dd3eec1c30b750b011bb 100644 (file)
@@ -22,7 +22,7 @@
  *             transaction in prepared state with the same GID.
  *
  *             A global transaction (gxact) also has dummy PGXACT and PGPROC; this is
- *             what keeps the XID considered running by TransactionIdIsInProgress.
+ *             what keeps the XID considered running by the functions in procarray.c.
  *             It is also convenient as a PGPROC to hook the gxact's locks to.
  *
  *             Information to recover prepared transactions in case of crash is
@@ -60,6 +60,7 @@
 
 #include "access/commit_ts.h"
 #include "access/htup_details.h"
+#include "access/mvccvars.h"
 #include "access/subtrans.h"
 #include "access/transam.h"
 #include "access/twophase.h"
@@ -414,6 +415,7 @@ MarkAsPreparing(TransactionId xid, const char *gid,
        proc->lxid = (LocalTransactionId) xid;
        pgxact->xid = xid;
        pgxact->xmin = InvalidTransactionId;
+       pgxact->snapshotcsn = InvalidCommitSeqNo;
        pgxact->delayChkpt = false;
        pgxact->vacuumFlags = 0;
        proc->pid = 0;
@@ -426,9 +428,6 @@ MarkAsPreparing(TransactionId xid, const char *gid,
        proc->waitProcLock = NULL;
        for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
                SHMQueueInit(&(proc->myProcLocks[i]));
-       /* subxid data must be filled later by GXactLoadSubxactData */
-       pgxact->overflowed = false;
-       pgxact->nxids = 0;
 
        gxact->prepared_at = prepared_at;
        /* initialize LSN to InvalidXLogRecPtr */
@@ -455,34 +454,6 @@ MarkAsPreparing(TransactionId xid, const char *gid,
        return gxact;
 }
 
-/*
- * GXactLoadSubxactData
- *
- * If the transaction being persisted had any subtransactions, this must
- * be called before MarkAsPrepared() to load information into the dummy
- * PGPROC.
- */
-static void
-GXactLoadSubxactData(GlobalTransaction gxact, int nsubxacts,
-                                        TransactionId *children)
-{
-       PGPROC     *proc = &ProcGlobal->allProcs[gxact->pgprocno];
-       PGXACT     *pgxact = &ProcGlobal->allPgXact[gxact->pgprocno];
-
-       /* We need no extra lock since the GXACT isn't valid yet */
-       if (nsubxacts > PGPROC_MAX_CACHED_SUBXIDS)
-       {
-               pgxact->overflowed = true;
-               nsubxacts = PGPROC_MAX_CACHED_SUBXIDS;
-       }
-       if (nsubxacts > 0)
-       {
-               memcpy(proc->subxids.xids, children,
-                          nsubxacts * sizeof(TransactionId));
-               pgxact->nxids = nsubxacts;
-       }
-}
-
 /*
  * MarkAsPrepared
  *             Mark the GXACT as fully valid, and enter it into the global ProcArray.
@@ -497,7 +468,7 @@ MarkAsPrepared(GlobalTransaction gxact)
        LWLockRelease(TwoPhaseStateLock);
 
        /*
-        * Put it into the global ProcArray so TransactionIdIsInProgress considers
+        * Put it into the global ProcArray so GetOldestActiveTransactionId() considers
         * the XID as still running.
         */
        ProcArrayAdd(&ProcGlobal->allProcs[gxact->pgprocno]);
@@ -992,8 +963,6 @@ StartPrepare(GlobalTransaction gxact)
        if (hdr.nsubxacts > 0)
        {
                save_state_data(children, hdr.nsubxacts * sizeof(TransactionId));
-               /* While we have the child-xact data, stuff it in the gxact too */
-               GXactLoadSubxactData(gxact, hdr.nsubxacts, children);
        }
        if (hdr.ncommitrels > 0)
        {
@@ -1079,7 +1048,7 @@ EndPrepare(GlobalTransaction gxact)
         * NB: a side effect of this is to make a dummy ProcArray entry for the
         * prepared XID.  This must happen before we clear the XID from MyPgXact,
         * else there is a window where the XID is not running according to
-        * TransactionIdIsInProgress, and onlookers would be entitled to assume
+        * GetOldestActiveTransactionId, and onlookers would be entitled to assume
         * the xact crashed.  Instead we have a window where the same XID appears
         * twice in ProcArray, which is OK.
         */
@@ -1328,7 +1297,6 @@ FinishPreparedTransaction(const char *gid, bool isCommit)
        char       *buf;
        char       *bufptr;
        TwoPhaseFileHeader *hdr;
-       TransactionId latestXid;
        TransactionId *children;
        RelFileNode *commitrels;
        RelFileNode *abortrels;
@@ -1373,14 +1341,11 @@ FinishPreparedTransaction(const char *gid, bool isCommit)
        invalmsgs = (SharedInvalidationMessage *) bufptr;
        bufptr += MAXALIGN(hdr->ninvalmsgs * sizeof(SharedInvalidationMessage));
 
-       /* compute latestXid among all children */
-       latestXid = TransactionIdLatest(xid, hdr->nsubxacts, children);
-
        /*
         * The order of operations here is critical: make the XLOG entry for
         * commit or abort, then mark the transaction committed or aborted in
         * pg_clog, then remove its PGPROC from the global ProcArray (which means
-        * TransactionIdIsInProgress will stop saying the prepared xact is in
+        * GetOldestActiveTransactionId() will stop saying the prepared xact is in
         * progress), then run the post-commit or post-abort callbacks. The
         * callbacks will release the locks the transaction held.
         */
@@ -1395,7 +1360,7 @@ FinishPreparedTransaction(const char *gid, bool isCommit)
                                                                           hdr->nsubxacts, children,
                                                                           hdr->nabortrels, abortrels);
 
-       ProcArrayRemove(proc, latestXid);
+       ProcArrayRemove(proc);
 
        /*
         * In case we fail while running the callbacks, mark the gxact invalid so
@@ -1841,7 +1806,7 @@ StandbyRecoverPreparedTransactions(bool overwriteOK)
                        xid = (TransactionId) strtoul(clde->d_name, NULL, 16);
 
                        /* Already processed? */
-                       if (TransactionIdDidCommit(xid) || TransactionIdDidAbort(xid))
+                       if (TransactionIdGetStatus(xid) != XID_INPROGRESS)
                        {
                                ereport(WARNING,
                                                (errmsg("removing stale two-phase state file \"%s\"",
@@ -1926,7 +1891,7 @@ RecoverPreparedTransactions(void)
                        xid = (TransactionId) strtoul(clde->d_name, NULL, 16);
 
                        /* Already processed? */
-                       if (TransactionIdDidCommit(xid) || TransactionIdDidAbort(xid))
+                       if (TransactionIdGetStatus(xid) != XID_INPROGRESS)
                        {
                                ereport(WARNING,
                                                (errmsg("removing stale two-phase state file \"%s\"",
@@ -1965,9 +1930,13 @@ RecoverPreparedTransactions(void)
                         * It's possible that SubTransSetParent has been set before, if
                         * the prepared transaction generated xid assignment records. Test
                         * here must match one used in AssignTransactionId().
+                        *
+                        * FIXME: I think this now always needs to be true. Or false?
                         */
+#ifdef FIXME
                        if (InHotStandby && (hdr->nsubxacts >= PGPROC_MAX_CACHED_SUBXIDS ||
                                                                 XLogLogicalInfoActive()))
+#endif
                                overwriteOK = true;
 
                        /*
@@ -1987,7 +1956,6 @@ RecoverPreparedTransactions(void)
                                                                        hdr->prepared_at,
                                                                        hdr->owner, hdr->database);
                        gxact->ondisk = true;
-                       GXactLoadSubxactData(gxact, hdr->nsubxacts, subxids);
                        MarkAsPrepared(gxact);
 
                        /*
@@ -2089,7 +2057,7 @@ RecordTransactionCommitPrepared(TransactionId xid,
        /* Flush XLOG to disk */
        XLogFlush(recptr);
 
-       /* Mark the transaction committed in pg_clog */
+       /* Mark the transaction committed in pg_clog and pg_csnlog */
        TransactionIdCommitTree(xid, nchildren, children);
 
        /* Checkpoint can proceed now */
@@ -2127,7 +2095,7 @@ RecordTransactionAbortPrepared(TransactionId xid,
         * Catch the scenario where we aborted partway through
         * RecordTransactionCommitPrepared ...
         */
-       if (TransactionIdDidCommit(xid))
+       if (TransactionIdGetStatus(xid) == XID_COMMITTED)
                elog(PANIC, "cannot abort transaction %u, it was already committed",
                         xid);
 
index 2f7e645ace8d85c28489024af956fc194bd18bf7..d6a91254877700187c904776d3d67ede60a38528 100644 (file)
@@ -15,6 +15,8 @@
 
 #include "access/clog.h"
 #include "access/commit_ts.h"
+#include "access/csnlog.h"
+#include "access/mvccvars.h"
 #include "access/subtrans.h"
 #include "access/transam.h"
 #include "access/xact.h"
@@ -169,8 +171,8 @@ GetNewTransactionId(bool isSubXact)
         * Extend pg_subtrans and pg_commit_ts too.
         */
        ExtendCLOG(xid);
+       ExtendCSNLOG(xid);
        ExtendCommitTs(xid);
-       ExtendSUBTRANS(xid);
 
        /*
         * Now advance the nextXid counter.  This must not happen until after we
@@ -200,17 +202,8 @@ GetNewTransactionId(bool isSubXact)
         * A solution to the atomic-store problem would be to give each PGXACT its
         * own spinlock used only for fetching/storing that PGXACT's xid and
         * related fields.
-        *
-        * If there's no room to fit a subtransaction XID into PGPROC, set the
-        * cache-overflowed flag instead.  This forces readers to look in
-        * pg_subtrans to map subtransaction XIDs up to top-level XIDs. There is a
-        * race-condition window, in that the new XID will not appear as running
-        * until its parent link has been placed into pg_subtrans. However, that
-        * will happen before anyone could possibly have a reason to inquire about
-        * the status of the XID, so it seems OK.  (Snapshots taken during this
-        * window *will* include the parent XID, so they will deliver the correct
-        * answer later on when someone does have a reason to inquire.)
         */
+       if (!isSubXact)
        {
                /*
                 * Use volatile pointer to prevent code rearrangement; other backends
@@ -219,23 +212,9 @@ GetNewTransactionId(bool isSubXact)
                 * nxids before filling the array entry.  Note we are assuming that
                 * TransactionId and int fetch/store are atomic.
                 */
-               volatile PGPROC *myproc = MyProc;
                volatile PGXACT *mypgxact = MyPgXact;
 
-               if (!isSubXact)
-                       mypgxact->xid = xid;
-               else
-               {
-                       int                     nxids = mypgxact->nxids;
-
-                       if (nxids < PGPROC_MAX_CACHED_SUBXIDS)
-                       {
-                               myproc->subxids.xids[nxids] = xid;
-                               mypgxact->nxids = nxids + 1;
-                       }
-                       else
-                               mypgxact->overflowed = true;
-               }
+               mypgxact->xid = xid;
        }
 
        LWLockRelease(XidGenLock);
index 23f36ead7e54e86d0ef1e33a63aa60ade10f8b8f..c33e5d37db424f1b2f7d1119b52045ee509b5d3d 100644 (file)
 #include <time.h>
 #include <unistd.h>
 
+#include "access/clog.h"
 #include "access/commit_ts.h"
 #include "access/multixact.h"
+#include "access/mvccvars.h"
 #include "access/parallel.h"
 #include "access/subtrans.h"
 #include "access/transam.h"
@@ -183,7 +185,6 @@ typedef struct TransactionStateData
        int                     prevSecContext; /* previous SecurityRestrictionContext */
        bool            prevXactReadOnly;               /* entry-time xact r/o state */
        bool            startedInRecovery;              /* did we start in recovery? */
-       bool            didLogXid;              /* has xid been included in WAL record? */
        int                     parallelModeLevel;              /* Enter/ExitParallelMode counter */
        struct TransactionStateData *parent;            /* back link to parent */
 } TransactionStateData;
@@ -214,18 +215,10 @@ static TransactionStateData TopTransactionStateData = {
        0,                                                      /* previous SecurityRestrictionContext */
        false,                                          /* entry-time xact r/o state */
        false,                                          /* startedInRecovery */
-       false,                                          /* didLogXid */
        0,                                                      /* parallelMode */
        NULL                                            /* link to parent state block */
 };
 
-/*
- * unreportedXids holds XIDs of all subtransactions that have not yet been
- * reported in an XLOG_XACT_ASSIGNMENT record.
- */
-static int     nUnreportedXids;
-static TransactionId unreportedXids[PGPROC_MAX_CACHED_SUBXIDS];
-
 static TransactionState CurrentTransactionState = &TopTransactionStateData;
 
 /*
@@ -309,7 +302,7 @@ static void CleanupTransaction(void);
 static void CheckTransactionChain(bool isTopLevel, bool throwError,
                                          const char *stmtType);
 static void CommitTransaction(void);
-static TransactionId RecordTransactionAbort(bool isSubXact);
+static void RecordTransactionAbort(bool isSubXact);
 static void StartTransaction(void);
 
 static void StartSubTransaction(void);
@@ -433,19 +426,6 @@ GetCurrentTransactionIdIfAny(void)
        return CurrentTransactionState->transactionId;
 }
 
-/*
- *     MarkCurrentTransactionIdLoggedIfAny
- *
- * Remember that the current xid - if it is assigned - now has been wal logged.
- */
-void
-MarkCurrentTransactionIdLoggedIfAny(void)
-{
-       if (TransactionIdIsValid(CurrentTransactionState->transactionId))
-               CurrentTransactionState->didLogXid = true;
-}
-
-
 /*
  *     GetStableLatestTransactionId
  *
@@ -487,7 +467,6 @@ AssignTransactionId(TransactionState s)
 {
        bool            isSubXact = (s->parent != NULL);
        ResourceOwner currentOwner;
-       bool            log_unknown_top = false;
 
        /* Assert that caller didn't screw up */
        Assert(!TransactionIdIsValid(s->transactionId));
@@ -538,18 +517,14 @@ AssignTransactionId(TransactionState s)
         * superfluously log something. That can happen when an xid is included
         * somewhere inside a wal record, but not in XLogRecord->xl_xid, like in
         * xl_standby_locks.
+        *
+        * FIXME: didLogXid and the whole xact_assignment stuff is no more. We
+        * no longer need it for subtransactions. Do we still need it for this
+        * logical stuff?
         */
-       if (isSubXact && XLogLogicalInfoActive() &&
-               !TopTransactionStateData.didLogXid)
-               log_unknown_top = true;
 
        /*
         * Generate a new Xid and record it in PG_PROC and pg_subtrans.
-        *
-        * NB: we must make the subtrans entry BEFORE the Xid appears anywhere in
-        * shared storage other than PG_PROC; because if there's no room for it in
-        * PG_PROC, the subtrans entry is needed to ensure that other backends see
-        * the Xid as "running".  See GetNewTransactionId.
         */
        s->transactionId = GetNewTransactionId(isSubXact);
        if (!isSubXact)
@@ -584,59 +559,6 @@ AssignTransactionId(TransactionState s)
        }
        PG_END_TRY();
        CurrentResourceOwner = currentOwner;
-
-       /*
-        * Every PGPROC_MAX_CACHED_SUBXIDS assigned transaction ids within each
-        * top-level transaction we issue a WAL record for the assignment. We
-        * include the top-level xid and all the subxids that have not yet been
-        * reported using XLOG_XACT_ASSIGNMENT records.
-        *
-        * This is required to limit the amount of shared memory required in a hot
-        * standby server to keep track of in-progress XIDs. See notes for
-        * RecordKnownAssignedTransactionIds().
-        *
-        * We don't keep track of the immediate parent of each subxid, only the
-        * top-level transaction that each subxact belongs to. This is correct in
-        * recovery only because aborted subtransactions are separately WAL
-        * logged.
-        *
-        * This is correct even for the case where several levels above us didn't
-        * have an xid assigned as we recursed up to them beforehand.
-        */
-       if (isSubXact && XLogStandbyInfoActive())
-       {
-               unreportedXids[nUnreportedXids] = s->transactionId;
-               nUnreportedXids++;
-
-               /*
-                * ensure this test matches similar one in
-                * RecoverPreparedTransactions()
-                */
-               if (nUnreportedXids >= PGPROC_MAX_CACHED_SUBXIDS ||
-                       log_unknown_top)
-               {
-                       xl_xact_assignment xlrec;
-
-                       /*
-                        * xtop is always set by now because we recurse up transaction
-                        * stack to the highest unassigned xid and then come back down
-                        */
-                       xlrec.xtop = GetTopTransactionId();
-                       Assert(TransactionIdIsValid(xlrec.xtop));
-                       xlrec.nsubxacts = nUnreportedXids;
-
-                       XLogBeginInsert();
-                       XLogRegisterData((char *) &xlrec, MinSizeOfXactAssignment);
-                       XLogRegisterData((char *) unreportedXids,
-                                                        nUnreportedXids * sizeof(TransactionId));
-
-                       (void) XLogInsert(RM_XACT_ID, XLOG_XACT_ASSIGNMENT);
-
-                       nUnreportedXids = 0;
-                       /* mark top, not current xact as having been logged */
-                       TopTransactionStateData.didLogXid = true;
-               }
-       }
 }
 
 /*
@@ -1117,17 +1039,13 @@ AtSubStart_ResourceOwner(void)
 /*
  *     RecordTransactionCommit
  *
- * Returns latest XID among xact and its children, or InvalidTransactionId
- * if the xact has no XID.  (We compute that here just because it's easier.)
- *
  * If you change this function, see RecordTransactionCommitPrepared also.
  */
-static TransactionId
+static void
 RecordTransactionCommit(void)
 {
        TransactionId xid = GetTopTransactionIdIfAny();
        bool            markXidCommitted = TransactionIdIsValid(xid);
-       TransactionId latestXid = InvalidTransactionId;
        int                     nrels;
        RelFileNode *rels;
        int                     nchildren;
@@ -1290,7 +1208,7 @@ RecordTransactionCommit(void)
                XLogFlush(XactLastRecEnd);
 
                /*
-                * Now we may update the CLOG, if we wrote a COMMIT record above
+                * Now we may update the CLOG and CSNLOG, if we wrote a COMMIT record above
                 */
                if (markXidCommitted)
                        TransactionIdCommitTree(xid, nchildren, children);
@@ -1316,7 +1234,8 @@ RecordTransactionCommit(void)
                 * flushed before the CLOG may be updated.
                 */
                if (markXidCommitted)
-                       TransactionIdAsyncCommitTree(xid, nchildren, children, XactLastRecEnd);
+                       TransactionIdAsyncCommitTree(xid, nchildren, children,
+                                                                                XactLastRecEnd);
        }
 
        /*
@@ -1329,9 +1248,6 @@ RecordTransactionCommit(void)
                END_CRIT_SECTION();
        }
 
-       /* Compute latestXid while we have the child XIDs handy */
-       latestXid = TransactionIdLatest(xid, nchildren, children);
-
        /*
         * Wait for synchronous replication, if required. Similar to the decision
         * above about using committing asynchronously we only want to wait if
@@ -1353,8 +1269,6 @@ cleanup:
        /* Clean up local data */
        if (rels)
                pfree(rels);
-
-       return latestXid;
 }
 
 
@@ -1522,15 +1436,11 @@ AtSubCommit_childXids(void)
 
 /*
  *     RecordTransactionAbort
- *
- * Returns latest XID among xact and its children, or InvalidTransactionId
- * if the xact has no XID.  (We compute that here just because it's easier.)
  */
-static TransactionId
+static void
 RecordTransactionAbort(bool isSubXact)
 {
        TransactionId xid = GetCurrentTransactionIdIfAny();
-       TransactionId latestXid;
        int                     nrels;
        RelFileNode *rels;
        int                     nchildren;
@@ -1548,7 +1458,7 @@ RecordTransactionAbort(bool isSubXact)
                /* Reset XactLastRecEnd until the next transaction writes something */
                if (!isSubXact)
                        XactLastRecEnd = 0;
-               return InvalidTransactionId;
+               return;
        }
 
        /*
@@ -1611,18 +1521,6 @@ RecordTransactionAbort(bool isSubXact)
 
        END_CRIT_SECTION();
 
-       /* Compute latestXid while we have the child XIDs handy */
-       latestXid = TransactionIdLatest(xid, nchildren, children);
-
-       /*
-        * If we're aborting a subtransaction, we can immediately remove failed
-        * XIDs from PGPROC's cache of running child XIDs.  We do that here for
-        * subxacts, because we already have the child XID array at hand.  For
-        * main xacts, the equivalent happens just after this function returns.
-        */
-       if (isSubXact)
-               XidCacheRemoveRunningXids(xid, nchildren, children, latestXid);
-
        /* Reset XactLastRecEnd until the next transaction writes something */
        if (!isSubXact)
                XactLastRecEnd = 0;
@@ -1630,8 +1528,6 @@ RecordTransactionAbort(bool isSubXact)
        /* And clean up local data */
        if (rels)
                pfree(rels);
-
-       return latestXid;
 }
 
 /*
@@ -1857,12 +1753,6 @@ StartTransaction(void)
        currentCommandId = FirstCommandId;
        currentCommandIdUsed = false;
 
-       /*
-        * initialize reported xid accounting
-        */
-       nUnreportedXids = 0;
-       s->didLogXid = false;
-
        /*
         * must initialize resource-management stuff first
         */
@@ -1940,7 +1830,6 @@ static void
 CommitTransaction(void)
 {
        TransactionState s = CurrentTransactionState;
-       TransactionId latestXid;
        bool            is_parallel_worker;
 
        is_parallel_worker = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
@@ -2040,16 +1929,10 @@ CommitTransaction(void)
                 * We need to mark our XIDs as committed in pg_clog.  This is where we
                 * durably commit.
                 */
-               latestXid = RecordTransactionCommit();
+               RecordTransactionCommit();
        }
        else
        {
-               /*
-                * We must not mark our XID committed; the parallel master is
-                * responsible for that.
-                */
-               latestXid = InvalidTransactionId;
-
                /*
                 * Make sure the master will know about any WAL we wrote before it
                 * commits.
@@ -2064,7 +1947,7 @@ CommitTransaction(void)
         * must be done _before_ releasing locks we hold and _after_
         * RecordTransactionCommit.
         */
-       ProcArrayEndTransaction(MyProc, latestXid);
+       ProcArrayEndTransaction(MyProc);
 
        /*
         * This is all post-commit cleanup.  Note that if an error is raised here,
@@ -2447,7 +2330,6 @@ static void
 AbortTransaction(void)
 {
        TransactionState s = CurrentTransactionState;
-       TransactionId latestXid;
        bool            is_parallel_worker;
 
        /* Prevent cancel/die interrupt while cleaning up */
@@ -2549,11 +2431,9 @@ AbortTransaction(void)
         * record.
         */
        if (!is_parallel_worker)
-               latestXid = RecordTransactionAbort(false);
+               RecordTransactionAbort(false);
        else
        {
-               latestXid = InvalidTransactionId;
-
                /*
                 * Since the parallel master won't get our value of XactLastRecEnd in
                 * this case, we nudge WAL-writer ourselves in this case.  See related
@@ -2569,7 +2449,7 @@ AbortTransaction(void)
         * must be done _before_ releasing locks we hold and _after_
         * RecordTransactionAbort.
         */
-       ProcArrayEndTransaction(MyProc, latestXid);
+       ProcArrayEndTransaction(MyProc);
 
        /*
         * Post-abort cleanup.  See notes in CommitTransaction() concerning
@@ -5375,9 +5255,12 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
        if (standbyState == STANDBY_DISABLED)
        {
                /*
-                * Mark the transaction committed in pg_clog.
+                * Mark the transaction committed in pg_clog. We don't bother updating
+                * pg_csnlog during replay.
                 */
-               TransactionIdCommitTree(xid, parsed->nsubxacts, parsed->subxacts);
+               CLogSetTreeStatus(xid, parsed->nsubxacts, parsed->subxacts,
+                                                 CLOG_XID_STATUS_COMMITTED,
+                                                 InvalidXLogRecPtr);
        }
        else
        {
@@ -5401,14 +5284,7 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
                 * bits set on changes made by transactions that haven't yet
                 * recovered. It's unlikely but it's good to be safe.
                 */
-               TransactionIdAsyncCommitTree(
-                                                         xid, parsed->nsubxacts, parsed->subxacts, lsn);
-
-               /*
-                * We must mark clog before we update the ProcArray.
-                */
-               ExpireTreeKnownAssignedTransactionIds(
-                                                 xid, parsed->nsubxacts, parsed->subxacts, max_xid);
+               TransactionIdAsyncCommitTree(xid, parsed->nsubxacts, parsed->subxacts, lsn);
 
                /*
                 * Send any cache invalidations attached to the commit. We must
@@ -5530,8 +5406,13 @@ xact_redo_abort(xl_xact_parsed_abort *parsed, TransactionId xid)
 
        if (standbyState == STANDBY_DISABLED)
        {
-               /* Mark the transaction aborted in pg_clog, no need for async stuff */
-               TransactionIdAbortTree(xid, parsed->nsubxacts, parsed->subxacts);
+               /*
+                * Mark the transaction aborted in pg_clog, no need for async stuff or
+                * to update pg_csnlog.
+                */
+               CLogSetTreeStatus(xid, parsed->nsubxacts, parsed->subxacts,
+                                                 CLOG_XID_STATUS_ABORTED,
+                                                 InvalidXLogRecPtr);
        }
        else
        {
@@ -5549,12 +5430,6 @@ xact_redo_abort(xl_xact_parsed_abort *parsed, TransactionId xid)
                /* Mark the transaction aborted in pg_clog, no need for async stuff */
                TransactionIdAbortTree(xid, parsed->nsubxacts, parsed->subxacts);
 
-               /*
-                * We must update the ProcArray after we have marked clog.
-                */
-               ExpireTreeKnownAssignedTransactionIds(
-                                                 xid, parsed->nsubxacts, parsed->subxacts, max_xid);
-
                /*
                 * There are no flat files that need updating, nor invalidation
                 * messages to send or undo.
@@ -5635,14 +5510,6 @@ xact_redo(XLogReaderState *record)
                RecreateTwoPhaseFile(XLogRecGetXid(record),
                                                  XLogRecGetData(record), XLogRecGetDataLen(record));
        }
-       else if (info == XLOG_XACT_ASSIGNMENT)
-       {
-               xl_xact_assignment *xlrec = (xl_xact_assignment *) XLogRecGetData(record);
-
-               if (standbyState >= STANDBY_INITIALIZED)
-                       ProcArrayApplyXidAssignment(xlrec->xtop,
-                                                                               xlrec->nsubxacts, xlrec->xsub);
-       }
        else
                elog(PANIC, "xact_redo: unknown op code %u", info);
 }
index f13f9c1fa5e271709a137d517533284c84253afd..308398154c7fa44f1d8db19e3cd8e5c22982df40 100644 (file)
@@ -23,7 +23,9 @@
 
 #include "access/clog.h"
 #include "access/commit_ts.h"
+#include "access/csnlog.h"
 #include "access/multixact.h"
+#include "access/mvccvars.h"
 #include "access/rewriteheap.h"
 #include "access/subtrans.h"
 #include "access/timeline.h"
@@ -1022,8 +1024,6 @@ XLogInsertRecord(XLogRecData *rdata, XLogRecPtr fpw_lsn)
         */
        WALInsertLockRelease();
 
-       MarkCurrentTransactionIdLoggedIfAny();
-
        END_CRIT_SECTION();
 
        /*
@@ -4766,6 +4766,7 @@ BootStrapXLOG(void)
        uint64          sysidentifier;
        struct timeval tv;
        pg_crc32c       crc;
+       TransactionId latestCompletedXid;
 
        /*
         * Select a hopefully-unique system identifier code for this installation.
@@ -4820,6 +4821,14 @@ BootStrapXLOG(void)
        ShmemVariableCache->nextXid = checkPoint.nextXid;
        ShmemVariableCache->nextOid = checkPoint.nextOid;
        ShmemVariableCache->oidCount = 0;
+
+       pg_atomic_write_u64(&ShmemVariableCache->nextCommitSeqNo, COMMITSEQNO_FIRST_NORMAL);
+       latestCompletedXid = checkPoint.nextXid;
+       TransactionIdRetreat(latestCompletedXid);
+       pg_atomic_write_u32(&ShmemVariableCache->latestCompletedXid, latestCompletedXid);
+       pg_atomic_write_u32(&ShmemVariableCache->oldestActiveXid, checkPoint.nextXid);
+       pg_atomic_write_u32(&ShmemVariableCache->globalXmin, checkPoint.nextXid);
+
        MultiXactSetNextMXact(checkPoint.nextMulti, checkPoint.nextMultiOffset);
        SetTransactionIdLimit(checkPoint.oldestXid, checkPoint.oldestXidDB);
        SetMultiXactIdLimit(checkPoint.oldestMulti, checkPoint.oldestMultiDB);
@@ -4912,8 +4921,8 @@ BootStrapXLOG(void)
 
        /* Bootstrap the commit log, too */
        BootStrapCLOG();
+       BootStrapCSNLOG();
        BootStrapCommitTs();
-       BootStrapSUBTRANS();
        BootStrapMultiXact();
 
        pfree(buffer);
@@ -5930,6 +5939,7 @@ StartupXLOG(void)
        XLogPageReadPrivate private;
        bool            fast_promoted = false;
        struct stat st;
+       TransactionId latestCompletedXid;
 
        /*
         * Read control file and check XLOG status looks valid.
@@ -6346,6 +6356,13 @@ StartupXLOG(void)
        XLogCtl->ckptXidEpoch = checkPoint.nextXidEpoch;
        XLogCtl->ckptXid = checkPoint.nextXid;
 
+       pg_atomic_write_u64(&ShmemVariableCache->nextCommitSeqNo, COMMITSEQNO_FIRST_NORMAL);
+       latestCompletedXid = checkPoint.nextXid;
+       TransactionIdRetreat(latestCompletedXid);
+       pg_atomic_write_u32(&ShmemVariableCache->latestCompletedXid, latestCompletedXid);
+       pg_atomic_write_u32(&ShmemVariableCache->oldestActiveXid, checkPoint.nextXid);
+       pg_atomic_write_u32(&ShmemVariableCache->globalXmin, checkPoint.nextXid);
+
        /*
         * Initialize replication slots, before there's a chance to remove
         * required resources.
@@ -6588,15 +6605,15 @@ StartupXLOG(void)
                        Assert(TransactionIdIsValid(oldestActiveXID));
 
                        /* Tell procarray about the range of xids it has to deal with */
-                       ProcArrayInitRecovery(ShmemVariableCache->nextXid);
+                       ProcArrayInitRecovery(oldestActiveXID, ShmemVariableCache->nextXid);
 
                        /*
-                        * Startup commit log and subtrans only.  MultiXact and commit
+                        * Startup commit log and csnlog only.  MultiXact and commit
                         * timestamp have already been started up and other SLRUs are not
                         * maintained during recovery and need not be started yet.
                         */
                        StartupCLOG();
-                       StartupSUBTRANS(oldestActiveXID);
+                       StartupCSNLOG(oldestActiveXID);
 
                        /*
                         * If we're beginning at a shutdown checkpoint, we know that
@@ -6607,7 +6624,6 @@ StartupXLOG(void)
                        if (wasShutdown)
                        {
                                RunningTransactionsData running;
-                               TransactionId latestCompletedXid;
 
                                /*
                                 * Construct a RunningTransactions snapshot representing a
@@ -6615,16 +6631,8 @@ StartupXLOG(void)
                                 * alive. We're never overflowed at this point because all
                                 * subxids are listed with their parent prepared transactions.
                                 */
-                               running.xcnt = nxids;
-                               running.subxcnt = 0;
-                               running.subxid_overflow = false;
                                running.nextXid = checkPoint.nextXid;
                                running.oldestRunningXid = oldestActiveXID;
-                               latestCompletedXid = checkPoint.nextXid;
-                               TransactionIdRetreat(latestCompletedXid);
-                               Assert(TransactionIdIsNormal(latestCompletedXid));
-                               running.latestCompletedXid = latestCompletedXid;
-                               running.xids = xids;
 
                                ProcArrayApplyRecoveryInfo(&running);
 
@@ -7358,20 +7366,22 @@ StartupXLOG(void)
        /* start the archive_timeout timer running */
        XLogCtl->lastSegSwitchTime = (pg_time_t) time(NULL);
 
-       /* also initialize latestCompletedXid, to nextXid - 1 */
-       LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
-       ShmemVariableCache->latestCompletedXid = ShmemVariableCache->nextXid;
-       TransactionIdRetreat(ShmemVariableCache->latestCompletedXid);
-       LWLockRelease(ProcArrayLock);
+       /* also initialize latestCompletedXid, to nextXid - 1, and oldestActiveXid */
+       latestCompletedXid = ShmemVariableCache->nextXid;
+       TransactionIdRetreat(latestCompletedXid);
+       pg_atomic_write_u32(&ShmemVariableCache->latestCompletedXid,
+                                               latestCompletedXid);
+       pg_atomic_write_u32(&ShmemVariableCache->oldestActiveXid,
+                                               oldestActiveXID);
 
        /*
-        * Start up the commit log and subtrans, if not already done for hot
+        * Start up the commit log and csnlog, if not already done for hot
         * standby.  (commit timestamps are started below, if necessary.)
         */
        if (standbyState == STANDBY_DISABLED)
        {
                StartupCLOG();
-               StartupSUBTRANS(oldestActiveXID);
+               StartupCSNLOG(oldestActiveXID);
        }
 
        /*
@@ -7975,8 +7985,8 @@ ShutdownXLOG(int code, Datum arg)
                CreateCheckPoint(CHECKPOINT_IS_SHUTDOWN | CHECKPOINT_IMMEDIATE);
        }
        ShutdownCLOG();
+       ShutdownCSNLOG();
        ShutdownCommitTs();
-       ShutdownSUBTRANS();
        ShutdownMultiXact();
 }
 
@@ -8546,14 +8556,14 @@ CreateCheckPoint(int flags)
                PreallocXlogFiles(recptr);
 
        /*
-        * Truncate pg_subtrans if possible.  We can throw away all data before
+        * Truncate pg_csnlog if possible.  We can throw away all data before
         * the oldest XMIN of any running transaction.  No future transaction will
-        * attempt to reference any pg_subtrans entry older than that (see Asserts
-        * in subtrans.c).  During recovery, though, we mustn't do this because
-        * StartupSUBTRANS hasn't been called yet.
+        * attempt to reference any pg_csnlog entry older than that (see Asserts
+        * in csnlog.c).  During recovery, though, we mustn't do this because
+        * StartupCSNLOG hasn't been called yet.
         */
        if (!RecoveryInProgress())
-               TruncateSUBTRANS(GetOldestXmin(NULL, false));
+               TruncateCSNLOG(GetOldestXmin(NULL, false));
 
        /* Real work is done, but log and update stats before releasing lock. */
        LogCheckpointEnd(false);
@@ -8629,13 +8639,12 @@ static void
 CheckPointGuts(XLogRecPtr checkPointRedo, int flags)
 {
        CheckPointCLOG();
+       CheckPointCSNLOG();
        CheckPointCommitTs();
-       CheckPointSUBTRANS();
        CheckPointMultiXact();
        CheckPointPredicate();
        CheckPointRelationMap();
        CheckPointReplicationSlots();
-       CheckPointSnapBuild();
        CheckPointLogicalRewriteHeap();
        CheckPointBuffers(flags);       /* performs all required fsyncs */
        CheckPointReplicationOrigin();
@@ -8885,14 +8894,14 @@ CreateRestartPoint(int flags)
        }
 
        /*
-        * Truncate pg_subtrans if possible.  We can throw away all data before
+        * Truncate pg_csnlog if possible.  We can throw away all data before
         * the oldest XMIN of any running transaction.  No future transaction will
-        * attempt to reference any pg_subtrans entry older than that (see Asserts
-        * in subtrans.c).  When hot standby is disabled, though, we mustn't do
-        * this because StartupSUBTRANS hasn't been called yet.
+        * attempt to reference any pg_csnlog entry older than that (see Asserts
+        * in csnlog.c).  When hot standby is disabled, though, we mustn't do
+        * this because StartupCSNLOG hasn't been called yet.
         */
        if (EnableHotStandby)
-               TruncateSUBTRANS(GetOldestXmin(NULL, false));
+               TruncateCSNLOG(GetOldestXmin(NULL, false));
 
        /* Real work is done, but log and update before releasing lock. */
        LogCheckpointEnd(true);
@@ -9271,7 +9280,6 @@ xlog_redo(XLogReaderState *record)
                        TransactionId *xids;
                        int                     nxids;
                        TransactionId oldestActiveXID;
-                       TransactionId latestCompletedXid;
                        RunningTransactionsData running;
 
                        oldestActiveXID = PrescanPreparedTransactions(&xids, &nxids);
@@ -9282,16 +9290,8 @@ xlog_redo(XLogReaderState *record)
                         * never overflowed at this point because all subxids are listed
                         * with their parent prepared transactions.
                         */
-                       running.xcnt = nxids;
-                       running.subxcnt = 0;
-                       running.subxid_overflow = false;
                        running.nextXid = checkPoint.nextXid;
                        running.oldestRunningXid = oldestActiveXID;
-                       latestCompletedXid = checkPoint.nextXid;
-                       TransactionIdRetreat(latestCompletedXid);
-                       Assert(TransactionIdIsNormal(latestCompletedXid));
-                       running.latestCompletedXid = latestCompletedXid;
-                       running.xids = xids;
 
                        ProcArrayApplyRecoveryInfo(&running);
 
index e997b574ca9eaf93514817464b3a77467d1a5371..d47bdd07e34ba1ce13473a24fd4a9e5b1e550b76 100644 (file)
@@ -64,6 +64,7 @@
 #include "parser/parse_expr.h"
 #include "parser/parse_relation.h"
 #include "storage/predicate.h"
+#include "storage/procarray.h"
 #include "storage/smgr.h"
 #include "utils/acl.h"
 #include "utils/builtins.h"
@@ -895,7 +896,7 @@ AddNewRelationTuple(Relation pg_class_desc,
                 * We know that no xacts older than RecentXmin are still running, so
                 * that will do.
                 */
-               new_rel_reltup->relfrozenxid = RecentXmin;
+               new_rel_reltup->relfrozenxid = GetOldestActiveTransactionId();
 
                /*
                 * Similarly, initialize the minimum Multixact to the first value that
index 716f1c33183da6e2cc0d50bf463098bfcbf213a6..fb77e5f85df2d84453690e8e8c9d2ab27000e291 100644 (file)
@@ -1928,27 +1928,21 @@ asyncQueueProcessPageEntries(volatile QueuePosition *current,
                /* Ignore messages destined for other databases */
                if (qe->dboid == MyDatabaseId)
                {
-                       if (TransactionIdIsInProgress(qe->xid))
+                       TransactionIdStatus xidstatus = TransactionIdGetStatus(qe->xid);
+
+                       if (xidstatus == XID_INPROGRESS)
                        {
                                /*
                                 * The source transaction is still in progress, so we can't
                                 * process this message yet.  Break out of the loop, but first
                                 * back up *current so we will reprocess the message next
-                                * time.  (Note: it is unlikely but not impossible for
-                                * TransactionIdDidCommit to fail, so we can't really avoid
-                                * this advance-then-back-up behavior when dealing with an
-                                * uncommitted message.)
-                                *
-                                * Note that we must test TransactionIdIsInProgress before we
-                                * test TransactionIdDidCommit, else we might return a message
-                                * from a transaction that is not yet visible to snapshots;
-                                * compare the comments at the head of tqual.c.
+                                * time.
                                 */
                                *current = thisentry;
                                reachedStop = true;
                                break;
                        }
-                       else if (TransactionIdDidCommit(qe->xid))
+                       else if (xidstatus == XID_COMMITTED)
                        {
                                /* qe->data is the null-terminated channel name */
                                char       *channel = qe->data;
index 6cddcbd02c380f7243cbf97ffeb41347d5a15e5b..4bcacfbe747a6a8ad49dd95fe3e41b7eb80acd02 100644 (file)
@@ -32,6 +32,7 @@
 #include "parser/parse_relation.h"
 #include "rewrite/rewriteHandler.h"
 #include "storage/lmgr.h"
+#include "storage/procarray.h"
 #include "storage/smgr.h"
 #include "tcop/tcopprot.h"
 #include "utils/builtins.h"
@@ -820,7 +821,8 @@ static void
 refresh_by_heap_swap(Oid matviewOid, Oid OIDNewHeap, char relpersistence)
 {
        finish_heap_swap(matviewOid, OIDNewHeap, false, false, true, true,
-                                        RecentXmin, ReadNextMultiXactId(), relpersistence);
+                                        GetOldestActiveTransactionId(), ReadNextMultiXactId(),
+                                        relpersistence);
 }
 
 
index 86e98148c1667e1b5cf04146e4945f1f5b5c8b42..38ca2d37c5a09823e3bee0223b75bb4bb985f7b9 100644 (file)
@@ -82,6 +82,7 @@
 #include "storage/lmgr.h"
 #include "storage/lock.h"
 #include "storage/predicate.h"
+#include "storage/procarray.h"
 #include "storage/smgr.h"
 #include "utils/acl.h"
 #include "utils/builtins.h"
@@ -1219,7 +1220,7 @@ ExecuteTruncate(TruncateStmt *stmt)
                         * deletion at commit.
                         */
                        RelationSetNewRelfilenode(rel, rel->rd_rel->relpersistence,
-                                                                         RecentXmin, minmulti);
+                                                                         GetOldestActiveTransactionId(), minmulti);
                        if (rel->rd_rel->relpersistence == RELPERSISTENCE_UNLOGGED)
                                heap_create_init_fork(rel);
 
@@ -1233,7 +1234,7 @@ ExecuteTruncate(TruncateStmt *stmt)
                        {
                                rel = relation_open(toast_relid, AccessExclusiveLock);
                                RelationSetNewRelfilenode(rel, rel->rd_rel->relpersistence,
-                                                                                 RecentXmin, minmulti);
+                                                                                 GetOldestActiveTransactionId(), minmulti);
                                if (rel->rd_rel->relpersistence == RELPERSISTENCE_UNLOGGED)
                                        heap_create_init_fork(rel);
                                heap_close(rel, NoLock);
@@ -3868,7 +3869,7 @@ ATRewriteTables(AlterTableStmt *parsetree, List **wqueue, LOCKMODE lockmode)
                        finish_heap_swap(tab->relid, OIDNewHeap,
                                                         false, false, true,
                                                         !OidIsValid(tab->newTableSpace),
-                                                        RecentXmin,
+                                                        GetOldestActiveTransactionId(),
                                                         ReadNextMultiXactId(),
                                                         persistence);
                }
index 46cd5ba1f2ded5b68ce5246384d19f4c53ecca58..ff1a2427d912404da5a5dfd5aa363f1276efe489 100644 (file)
@@ -165,7 +165,6 @@ LogicalDecodingProcessRecord(LogicalDecodingContext *ctx, XLogReaderState *recor
 static void
 DecodeXLogOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 {
-       SnapBuild  *builder = ctx->snapshot_builder;
        uint8           info = XLogRecGetInfo(buf->record) & ~XLR_INFO_MASK;
 
        ReorderBufferProcessXid(ctx->reorder, XLogRecGetXid(buf->record),
@@ -176,8 +175,6 @@ DecodeXLogOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
                        /* 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:
 
@@ -217,8 +214,11 @@ DecodeXactOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
         * ok not to call ReorderBufferProcessXid() in that case, except in the
         * assignment case there'll not be any later records with the same xid;
         * and in the assignment case we'll not decode those xacts.
+        *
+        * FIXME: the assignment record is no more. I don't understand the above
+        * comment. Can it be just removed?
         */
-       if (SnapBuildCurrentState(builder) < SNAPBUILD_FULL_SNAPSHOT)
+       if (SnapBuildCurrentState(builder) < SNAPBUILD_CONSISTENT)
                return;
 
        switch (info)
@@ -259,23 +259,6 @@ DecodeXactOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
                                DecodeAbort(ctx, buf, &parsed, xid);
                                break;
                        }
-               case XLOG_XACT_ASSIGNMENT:
-                       {
-                               xl_xact_assignment *xlrec;
-                               int                     i;
-                               TransactionId *sub_xid;
-
-                               xlrec = (xl_xact_assignment *) XLogRecGetData(r);
-
-                               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:
 
                        /*
@@ -354,7 +337,7 @@ DecodeHeap2Op(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
        ReorderBufferProcessXid(ctx->reorder, xid, buf->origptr);
 
        /* no point in doing anything yet */
-       if (SnapBuildCurrentState(builder) < SNAPBUILD_FULL_SNAPSHOT)
+       if (SnapBuildCurrentState(builder) < SNAPBUILD_CONSISTENT)
                return;
 
        switch (info)
@@ -409,7 +392,7 @@ DecodeHeapOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
        ReorderBufferProcessXid(ctx->reorder, xid, buf->origptr);
 
        /* no point in doing anything yet */
-       if (SnapBuildCurrentState(builder) < SNAPBUILD_FULL_SNAPSHOT)
+       if (SnapBuildCurrentState(builder) < SNAPBUILD_CONSISTENT)
                return;
 
        switch (info)
@@ -502,7 +485,7 @@ DecodeLogicalMsgOp(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
        ReorderBufferProcessXid(ctx->reorder, XLogRecGetXid(r), buf->origptr);
 
        /* No point in doing anything yet. */
-       if (SnapBuildCurrentState(builder) < SNAPBUILD_FULL_SNAPSHOT)
+       if (SnapBuildCurrentState(builder) < SNAPBUILD_CONSISTENT)
                return;
 
        message = (xl_logical_message *) XLogRecGetData(r);
index ecf9a03318044a438b6549a7555faf1b0c723603..9dd658cc33fc7ed3b913b289ec49156e5957797d 100644 (file)
@@ -113,7 +113,6 @@ CheckLogicalDecodingRequirements(void)
 static LogicalDecodingContext *
 StartupDecodingContext(List *output_plugin_options,
                                           XLogRecPtr start_lsn,
-                                          TransactionId xmin_horizon,
                                           XLogPageReadCB read_page,
                                           LogicalOutputPluginWriterPrepareWrite prepare_write,
                                           LogicalOutputPluginWriterWrite do_write)
@@ -173,7 +172,7 @@ StartupDecodingContext(List *output_plugin_options,
 
        ctx->reorder = ReorderBufferAllocate();
        ctx->snapshot_builder =
-               AllocateSnapshotBuilder(ctx->reorder, xmin_horizon, start_lsn);
+               AllocateSnapshotBuilder(ctx->reorder, start_lsn);
 
        ctx->reorder->private_data = ctx;
 
@@ -216,7 +215,6 @@ CreateInitDecodingContext(char *plugin,
                                                  LogicalOutputPluginWriterPrepareWrite prepare_write,
                                                  LogicalOutputPluginWriterWrite do_write)
 {
-       TransactionId xmin_horizon = InvalidTransactionId;
        ReplicationSlot *slot;
        LogicalDecodingContext *ctx;
        MemoryContext old_context;
@@ -280,16 +278,10 @@ CreateInitDecodingContext(char *plugin,
 
        LWLockRelease(ProcArrayLock);
 
-       /*
-        * tell the snapshot builder to only assemble snapshot once reaching the
-        * running_xact's record with the respective xmin.
-        */
-       xmin_horizon = slot->data.catalog_xmin;
-
        ReplicationSlotMarkDirty();
        ReplicationSlotSave();
 
-       ctx = StartupDecodingContext(NIL, InvalidXLogRecPtr, xmin_horizon,
+       ctx = StartupDecodingContext(NIL, InvalidXLogRecPtr,
                                                                 read_page, prepare_write, do_write);
 
        /* call output plugin initialization callback */
@@ -379,7 +371,7 @@ CreateDecodingContext(XLogRecPtr start_lsn,
        }
 
        ctx = StartupDecodingContext(output_plugin_options,
-                                                                start_lsn, InvalidTransactionId,
+                                                                start_lsn,
                                                                 read_page, prepare_write, do_write);
 
        /* call output plugin initialization callback */
@@ -749,12 +741,12 @@ message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 }
 
 /*
- * Set the required catalog xmin horizon for historic snapshots in the current
- * replication slot.
+ * Set the oldest snapshot required for historic catalog lookups in the
+ * current replication slot.
  *
- * Note that in the most cases, we won't be able to immediately use the xmin
- * to increase the xmin horizon: we need to wait till the client has confirmed
- * receiving current_lsn with LogicalConfirmReceivedLocation().
+ * Note that in the most cases, we won't be able to immediately use the
+ * snapshot to increase the oldest snapshot, we need to wait till the client
+ * has confirmed receiving current_lsn with LogicalConfirmReceivedLocation().
  */
 void
 LogicalIncreaseXminForSlot(XLogRecPtr current_lsn, TransactionId xmin)
index 213ce34674ced87e1c010ed617e4458cee975668..bc744d215661d212828bedd447143f0b2d95d6b7 100644 (file)
@@ -1275,7 +1275,6 @@ ReorderBufferCopySnap(ReorderBuffer *rb, Snapshot orig_snap,
        Size            size;
 
        size = sizeof(SnapshotData) +
-               sizeof(TransactionId) * orig_snap->xcnt +
                sizeof(TransactionId) * (txn->nsubtxns + 1);
 
        snap = MemoryContextAllocZero(rb->context, size);
@@ -1284,36 +1283,33 @@ ReorderBufferCopySnap(ReorderBuffer *rb, Snapshot orig_snap,
        snap->copied = true;
        snap->active_count = 1;         /* mark as active so nobody frees it */
        snap->regd_count = 0;
-       snap->xip = (TransactionId *) (snap + 1);
-
-       memcpy(snap->xip, orig_snap->xip, sizeof(TransactionId) * snap->xcnt);
 
        /*
         * snap->subxip contains all txids that belong to our transaction which we
         * need to check via cmin/cmax. Thats why we store the toplevel
         * transaction in there as well.
         */
-       snap->subxip = snap->xip + snap->xcnt;
-       snap->subxip[i++] = txn->xid;
+       snap->this_xip = (TransactionId *) (snap + 1);
+       snap->this_xip[i++] = txn->xid;
 
        /*
         * nsubxcnt isn't decreased when subtransactions abort, so count manually.
         * Since it's an upper boundary it is safe to use it for the allocation
         * above.
         */
-       snap->subxcnt = 1;
+       snap->this_xcnt = 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++;
+               snap->this_xip[i++] = sub_txn->xid;
+               snap->this_xcnt++;
        }
 
        /* sort so we can bsearch() later */
-       qsort(snap->subxip, snap->subxcnt, sizeof(TransactionId), xidComparator);
+       qsort(snap->this_xip, snap->this_xcnt, sizeof(TransactionId), xidComparator);
 
        /* store the specified current CommandId */
        snap->curcid = cid;
@@ -1389,6 +1385,7 @@ ReorderBufferCommit(ReorderBuffer *rb, TransactionId xid,
        }
 
        snapshot_now = txn->base_snapshot;
+       Assert(snapshot_now->snapshotcsn != InvalidCommitSeqNo);
 
        /* build data to be able to lookup the CommandIds of catalog tuples */
        ReorderBufferBuildTupleCidHash(rb, txn);
@@ -2277,10 +2274,7 @@ ReorderBufferSerializeChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
 
                                snap = change->data.snapshot;
 
-                               sz += sizeof(SnapshotData) +
-                                       sizeof(TransactionId) * snap->xcnt +
-                                       sizeof(TransactionId) * snap->subxcnt
-                                       ;
+                               sz += sizeof(SnapshotData);
 
                                /* make sure we have enough space */
                                ReorderBufferSerializeReserve(rb, sz);
@@ -2290,20 +2284,6 @@ ReorderBufferSerializeChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
 
                                memcpy(data, snap, sizeof(SnapshotData));
                                data += sizeof(SnapshotData);
-
-                               if (snap->xcnt)
-                               {
-                                       memcpy(data, snap->xip,
-                                                  sizeof(TransactionId) * snap->xcnt);
-                                       data += sizeof(TransactionId) * snap->xcnt;
-                               }
-
-                               if (snap->subxcnt)
-                               {
-                                       memcpy(data, snap->subxip,
-                                                  sizeof(TransactionId) * snap->subxcnt);
-                                       data += sizeof(TransactionId) * snap->subxcnt;
-                               }
                                break;
                        }
                case REORDER_BUFFER_CHANGE_INTERNAL_SPEC_CONFIRM:
@@ -2563,24 +2543,16 @@ ReorderBufferRestoreChange(ReorderBuffer *rb, ReorderBufferTXN *txn,
                        }
                case REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT:
                        {
-                               Snapshot        oldsnap;
                                Snapshot        newsnap;
                                Size            size;
 
-                               oldsnap = (Snapshot) data;
-
-                               size = sizeof(SnapshotData) +
-                                       sizeof(TransactionId) * oldsnap->xcnt +
-                                       sizeof(TransactionId) * (oldsnap->subxcnt + 0);
+                               size = sizeof(SnapshotData);
 
                                change->data.snapshot = MemoryContextAllocZero(rb->context, size);
 
                                newsnap = change->data.snapshot;
 
                                memcpy(newsnap, data, size);
-                               newsnap->xip = (TransactionId *)
-                                       (((char *) newsnap) + sizeof(SnapshotData));
-                               newsnap->subxip = newsnap->xip + newsnap->xcnt;
                                newsnap->copied = true;
                                break;
                        }
@@ -3230,7 +3202,7 @@ UpdateLogicalMappings(HTAB *tuplecid_data, Oid relid, Snapshot snapshot)
                        continue;
 
                /* not for our transaction */
-               if (!TransactionIdInArray(f_mapped_xid, snapshot->subxip, snapshot->subxcnt))
+               if (!TransactionIdInArray(f_mapped_xid, snapshot->this_xip, snapshot->this_xcnt))
                        continue;
 
                /* ok, relevant, queue for apply */
@@ -3258,7 +3230,7 @@ UpdateLogicalMappings(HTAB *tuplecid_data, Oid relid, Snapshot snapshot)
                RewriteMappingFile *f = files_a[off];
 
                elog(DEBUG1, "applying mapping: \"%s\" in %u", f->fname,
-                        snapshot->subxip[0]);
+                        snapshot->this_xip[0]);
                ApplyLogicalMappingFile(tuplecid_data, relid, f->fname);
                pfree(f);
        }
index b5fa3dbbc0bb7a4be872087933fb386348a88477..3ce4e0e375a50dfa96acc7c660b026ba9b32b8bc 100644 (file)
@@ -152,90 +152,24 @@ struct SnapBuild
        /* all transactions >= than this are uncommitted */
        TransactionId xmax;
 
+       /* this determines the state of transactions between xmin and xmax */
+       CommitSeqNo snapshotcsn;
+
        /*
         * 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      start_decoding_at;
 
-       /*
-        * Don't start decoding WAL until the "xl_running_xacts" information
-        * indicates there are no running xids with an xid smaller than this.
-        */
-       TransactionId initial_xmin_horizon;
-
        /*
         * Snapshot that's valid to see the catalog state seen at this moment.
         */
        Snapshot        snapshot;
 
-       /*
-        * LSN of the last location we are sure a snapshot has been serialized to.
-        */
-       XLogRecPtr      last_serialized_snapshot;
-
        /*
         * The reorderbuffer we need to update with usable snapshots et al.
         */
        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.
-                *
-                * TODO: It's unclear whether that reasoning has much merit. Every
-                * time we add something here after becoming consistent will also
-                * require distributing a snapshot. Storing them sorted would
-                * potentially also make it easier to purge (but more complicated wrt
-                * wraparound?). Should be improved if sorting while building the
-                * snapshot shows up in profiles.
-                */
-               TransactionId *xip;
-       }                       committed;
 };
 
 /*
@@ -245,15 +179,6 @@ struct SnapBuild
 static ResourceOwner SavedResourceOwnerDuringExport = NULL;
 static 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);
 
@@ -263,13 +188,6 @@ 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.
@@ -279,7 +197,6 @@ static bool SnapBuildRestore(SnapBuild *builder, XLogRecPtr lsn);
  */
 SnapBuild *
 AllocateSnapshotBuilder(ReorderBuffer *reorder,
-                                               TransactionId xmin_horizon,
                                                XLogRecPtr start_lsn)
 {
        MemoryContext context;
@@ -301,13 +218,6 @@ AllocateSnapshotBuilder(ReorderBuffer *reorder,
        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->initial_xmin_horizon = xmin_horizon;
        builder->start_decoding_at = start_lsn;
 
        MemoryContextSwitchTo(oldcontext);
@@ -345,7 +255,6 @@ SnapBuildFreeSnapshot(Snapshot snap)
 
        /* make sure nobody modified our snapshot */
        Assert(snap->curcid == FirstCommandId);
-       Assert(!snap->suboverflowed);
        Assert(!snap->takenDuringRecovery);
        Assert(snap->regd_count == 0);
 
@@ -403,7 +312,6 @@ SnapBuildSnapDecRefcount(Snapshot snap)
 
        /* make sure nobody modified our snapshot */
        Assert(snap->curcid == FirstCommandId);
-       Assert(!snap->suboverflowed);
        Assert(!snap->takenDuringRecovery);
 
        Assert(snap->regd_count == 0);
@@ -433,10 +341,9 @@ SnapBuildBuildSnapshot(SnapBuild *builder, TransactionId xid)
        Snapshot        snapshot;
        Size            ssize;
 
-       Assert(builder->state >= SNAPBUILD_FULL_SNAPSHOT);
+       Assert(builder->state >= SNAPBUILD_CONSISTENT);
 
        ssize = sizeof(SnapshotData)
-               + sizeof(TransactionId) * builder->committed.xcnt
                + sizeof(TransactionId) * 1 /* toplevel xid */ ;
 
        snapshot = MemoryContextAllocZero(builder->context, ssize);
@@ -444,52 +351,34 @@ SnapBuildBuildSnapshot(SnapBuild *builder, TransactionId xid)
        snapshot->satisfies = HeapTupleSatisfiesHistoricMVCC;
 
        /*
-        * 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 it's 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
+        * use the 'this_xip' 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
+        * visible that would not normally be visible by the CSN test. this_xip 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.
+        * this_xip array is qsort'ed so that we can use bsearch() on them.
         */
        Assert(TransactionIdIsNormal(builder->xmin));
        Assert(TransactionIdIsNormal(builder->xmax));
+       Assert(builder->snapshotcsn != InvalidCommitSeqNo);
 
        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);
+       snapshot->snapshotcsn = builder->snapshotcsn;
 
        /*
-        * Initially, subxip is empty, i.e. it's a snapshot to be used by
+        * Initially, this_xip 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->this_xcnt = 0;
+       snapshot->this_xip = NULL;
 
-       snapshot->suboverflowed = false;
        snapshot->takenDuringRecovery = false;
        snapshot->copied = false;
        snapshot->curcid = FirstCommandId;
@@ -515,19 +404,13 @@ 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 (TransactionIdIsValid(MyPgXact->snapshotcsn))
+               elog(ERROR, "cannot export a snapshot when MyPgXact->snapshotcsn already is valid");
 
        if (IsTransactionOrTransactionBlock())
                elog(ERROR, "cannot export a snapshot from within a transaction");
@@ -553,42 +436,7 @@ SnapBuildExportSnapshot(SnapBuild *builder)
         * 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;
+       MyPgXact->snapshotcsn = snap->snapshotcsn;
 
        /*
         * now that we've built a plain snapshot, use the normal mechanisms for
@@ -597,10 +445,10 @@ SnapBuildExportSnapshot(SnapBuild *builder)
        snapname = ExportSnapshot(snap);
 
        ereport(LOG,
-                       (errmsg_plural("exported logical decoding snapshot: \"%s\" with %u transaction ID",
-               "exported logical decoding snapshot: \"%s\" with %u transaction IDs",
-                                                  snap->xcnt,
-                                                  snapname, snap->xcnt)));
+                       (errmsg("exported logical decoding snapshot: \"%s\" at %X/%X",
+                                       snapname,
+                                       (uint32) (snap->snapshotcsn >> 32),
+                                       (uint32) snap->snapshotcsn)));
        return snapname;
 }
 
@@ -658,16 +506,7 @@ SnapBuildProcessChange(SnapBuild *builder, TransactionId xid, XLogRecPtr lsn)
         * 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. This means that they started before
-        * we got into the SNAPBUILD_FULL_SNAPSHOT state.
-        */
-       if (builder->state < SNAPBUILD_CONSISTENT &&
-               SnapBuildTxnIsRunning(builder, xid))
+       if (builder->state < SNAPBUILD_CONSISTENT)
                return false;
 
        /*
@@ -735,38 +574,6 @@ SnapBuildProcessNewCid(SnapBuild *builder, TransactionId xid,
        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 their contents. They
- * only exist after we freshly started from an < CONSISTENT snapshot.
- */
-static bool
-SnapBuildTxnIsRunning(SnapBuild *builder, TransactionId xid)
-{
-       Assert(builder->state < SNAPBUILD_CONSISTENT);
-       Assert(TransactionIdIsNormal(builder->running.xmin));
-       Assert(TransactionIdIsNormal(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
@@ -818,133 +625,6 @@ SnapBuildDistributeNewCatalogSnapshot(SnapBuild *builder, XLogRecPtr lsn)
        }
 }
 
-/*
- * 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;
-
-               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));
-       }
-
-       /*
-        * TODO: It might make sense to keep the array sorted here instead of
-        * doing it every time 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 ->committed 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;
-
-       /* TODO: Neater algorithm than just copying and iterating? */
-       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;
-
-       /*
-        * NB: This handles subtransactions correctly even if we started from
-        * suboverflowed xl_running_xacts because we only keep track of toplevel
-        * transactions. Since the latter are always allocated before their
-        * subxids and since they end at the same time it's sufficient to deal
-        * with them here.
-        */
-       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("logical decoding found consistent point at %X/%X",
-                                                 (uint32) (lsn >> 32), (uint32) lsn),
-                                  errdetail("Transaction ID %u finished; no more running transactions.",
-                                                        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
  */
@@ -955,10 +635,8 @@ SnapBuildCommitTxn(SnapBuild *builder, XLogRecPtr lsn, TransactionId xid,
        int                     nxact;
 
        bool            forced_timetravel = false;
-       bool            sub_needs_timetravel = false;
-       bool            top_needs_timetravel = false;
 
-       TransactionId xmax = xid;
+       TransactionId xmax;
 
        /*
         * If we couldn't observe every change of a transaction because it was
@@ -984,93 +662,36 @@ SnapBuildCommitTxn(SnapBuild *builder, XLogRecPtr lsn, TransactionId xid,
                elog(DEBUG1, "forced to assume catalog changes for xid %u because it was running too early", xid);
        }
 
+       xmax = builder->xmax;
+
+       if (NormalTransactionIdFollows(xid, xmax))
+               xmax = xid;
+       if (!forced_timetravel)
+       {
+               if (ReorderBufferXidHasCatalogChanges(builder->reorder, xid))
+                       forced_timetravel = true;
+       }
        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 (NormalTransactionIdFollows(subxid, xmax))
+                       xmax = subxid;
 
-               /*
-                * If we're forcing timetravel we also need visibility information
-                * about subtransaction, so keep track of subtransaction's state.
-                */
-               if (forced_timetravel)
+               if (!forced_timetravel)
                {
-                       SnapBuildAddCommittedTxn(builder, subxid);
-                       if (NormalTransactionIdFollows(subxid, xmax))
-                               xmax = subxid;
+                       if (ReorderBufferXidHasCatalogChanges(builder->reorder, subxid))
+                               forced_timetravel = true;
                }
-
-               /*
-                * Add subtransaction to base snapshot if it DDL, 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 toplevel txn is not tracked in running txn's anymore, switch
-        * state to consistent if possible.
-        */
-       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);
-       }
+       builder->xmax = xmax;
+       /* We use the commit record's LSN as the snapshot */
+       builder->snapshotcsn = (CommitSeqNo) lsn;
 
        /* if there's any reason to build a historic snapshot, do so now */
-       if (forced_timetravel || top_needs_timetravel || sub_needs_timetravel)
+       if (forced_timetravel)
        {
-               /*
-                * Adjust xmax of the snapshot builder, we only do that for committed,
-                * catalog modifying, transactions, everything else isn't interesting
-                * for us since we'll never look at the respective rows.
-                */
-               if (!TransactionIdIsValid(builder->xmax) ||
-                       TransactionIdFollowsOrEquals(xmax, builder->xmax))
-               {
-                       builder->xmax = xmax;
-                       TransactionIdAdvance(builder->xmax);
-               }
-
-               /*
-                * If we haven't built a complete snapshot yet there's no need to hand
-                * it out, it wouldn't (and couldn't) be used anyway.
-                */
-               if (builder->state < SNAPBUILD_FULL_SNAPSHOT)
-                       return;
-
                /*
                 * Decrease the snapshot builder's refcount of the old snapshot, note
                 * that it still will be used if it has been handed out to the
@@ -1095,11 +716,12 @@ SnapBuildCommitTxn(SnapBuild *builder, XLogRecPtr lsn, TransactionId xid,
                /* add a new Snapshot to all currently running transactions */
                SnapBuildDistributeNewCatalogSnapshot(builder, lsn);
        }
-       else
-       {
-               /* record that we cannot export a general snapshot anymore */
-               builder->committed.includes_all_transactions = false;
-       }
+}
+
+void
+SnapBuildAbortTxn(SnapBuild *builder, XLogRecPtr lsn, TransactionId xid,
+                                 int nsubxacts, TransactionId *subxacts)
+{
 }
 
 
@@ -1118,40 +740,17 @@ SnapBuildProcessRunningXacts(SnapBuild *builder, XLogRecPtr lsn, xl_running_xact
 {
        ReorderBufferTXN *txn;
 
-       /*
-        * If we're not consistent yet, inspect the record to see whether it
-        * allows to get closer to being consistent. If we are consistent, dump
-        * our snapshot so others or we, after a restart, can use it.
-        */
-       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 based on the running xacts
-        * information. We don't increase ->xmax using it, 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 since we
-        * only ever look at those.
-        *
-        * NB: Because of that xmax can be lower than xmin, because we only
-        * increase xmax when a catalog modifying transaction commits. While odd
-        * looking, it's correct and actually more efficient this way since we hit
-        * fast paths in tqual.c.
+        * information.
         */
        builder->xmin = running->oldestRunningXid;
+       builder->xmax = running->nextXid;
+       builder->snapshotcsn = (CommitSeqNo) lsn;
 
-       /* 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);
+       elog(DEBUG3, "xmin: %u, xmax: %u",
+                builder->xmin, builder->xmax);
+       Assert(lsn != InvalidXLogRecPtr);
 
        /*
         * Inrease shared memory limits, so vacuum can work on tuples we prevented
@@ -1171,12 +770,8 @@ SnapBuildProcessRunningXacts(SnapBuild *builder, XLogRecPtr lsn, xl_running_xact
         * 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;
+               builder->state = SNAPBUILD_CONSISTENT;
 
        txn = ReorderBufferGetOldestTXN(builder->reorder);
 
@@ -1186,732 +781,4 @@ SnapBuildProcessRunningXacts(SnapBuild *builder, XLogRecPtr lsn, xl_running_xact
         */
        if (txn != NULL && txn->restart_decoding_lsn != InvalidXLogRecPtr)
                LogicalIncreaseRestartDecodingForSlot(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)
-               LogicalIncreaseRestartDecodingForSlot(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
-        *        initially 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(DEBUG1,
-                               (errmsg_internal("skipping snapshot at %X/%X while building logical decoding snapshot, xmin horizon too low",
-                                                                (uint32) (lsn >> 32), (uint32) lsn),
-               errdetail_internal("initial xmin horizon of %u vs the snapshot's %u",
-                                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->start_decoding_at == InvalidXLogRecPtr ||
-                       builder->start_decoding_at <= lsn)
-                       /* can decode everything after this */
-                       builder->start_decoding_at = lsn + 1;
-
-               /* As no transactions were running xmin/xmax can be trivially set. */
-               builder->xmin = running->nextXid;               /* < are finished */
-               builder->xmax = running->nextXid;               /* >= are running */
-
-               /* so we can safely use the faster comparisons */
-               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("logical decoding found consistent point at %X/%X",
-                                               (uint32) (lsn >> 32), (uint32) lsn),
-                                errdetail("There are no running transactions.")));
-
-               return false;
-       }
-       /* c) valid on disk state */
-       else if (SnapBuildRestore(builder, lsn))
-       {
-               /* there won't be any state to cleanup */
-    &nbs