]> granicus.if.org Git - postgresql/blobdiff - src/backend/access/transam/xact.c
pgindent run for 9.4
[postgresql] / src / backend / access / transam / xact.c
index dbf82010e0e143303df51734a1dcd5ef22e2cb57..3e744097c79ce30920ed8ae7b70b045c0ba5166f 100644 (file)
@@ -5,7 +5,7 @@
  *
  * See src/backend/access/transam/README for more information.
  *
- * Portions Copyright (c) 1996-2012, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
  *
  *
@@ -34,6 +34,7 @@
 #include "commands/trigger.h"
 #include "executor/spi.h"
 #include "libpq/be-fsstubs.h"
+#include "libpq/pqsignal.h"
 #include "miscadmin.h"
 #include "pgstat.h"
 #include "replication/walsender.h"
@@ -52,6 +53,7 @@
 #include "utils/memutils.h"
 #include "utils/relmapper.h"
 #include "utils/snapmgr.h"
+#include "utils/timeout.h"
 #include "utils/timestamp.h"
 #include "pg_trace.h"
 
@@ -146,6 +148,7 @@ 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? */
        struct TransactionStateData *parent;            /* back link to parent */
 } TransactionStateData;
 
@@ -175,6 +178,7 @@ static TransactionStateData TopTransactionStateData = {
        0,                                                      /* previous SecurityRestrictionContext */
        false,                                          /* entry-time xact r/o state */
        false,                                          /* startedInRecovery */
+       false,                                          /* didLogXid */
        NULL                                            /* link to parent state block */
 };
 
@@ -265,6 +269,8 @@ static void CallSubXactCallbacks(SubXactEvent event,
                                         SubTransactionId mySubid,
                                         SubTransactionId parentSubid);
 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 StartTransaction(void);
@@ -390,6 +396,19 @@ 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
  *
@@ -431,6 +450,7 @@ 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));
@@ -465,6 +485,20 @@ AssignTransactionId(TransactionState s)
                pfree(parents);
        }
 
+       /*
+        * When wal_level=logical, guarantee that a subtransaction's xid can only
+        * be seen in the WAL stream if its toplevel xid has been logged before.
+        * If necessary we log a xact_assignment record with fewer than
+        * PGPROC_MAX_CACHED_SUBXIDS. Note that it is fine if didLogXid isn't set
+        * for a transaction even though it appears in a WAL record, we just might
+        * 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.
+        */
+       if (isSubXact && XLogLogicalInfoActive() &&
+               !TopTransactionStateData.didLogXid)
+               log_unknown_top = true;
+
        /*
         * Generate a new Xid and record it in PG_PROC and pg_subtrans.
         *
@@ -519,6 +553,9 @@ AssignTransactionId(TransactionState s)
         * 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())
        {
@@ -529,7 +566,8 @@ AssignTransactionId(TransactionState s)
                 * ensure this test matches similar one in
                 * RecoverPreparedTransactions()
                 */
-               if (nUnreportedXids >= PGPROC_MAX_CACHED_SUBXIDS)
+               if (nUnreportedXids >= PGPROC_MAX_CACHED_SUBXIDS ||
+                       log_unknown_top)
                {
                        XLogRecData rdata[2];
                        xl_xact_assignment xlrec;
@@ -548,13 +586,15 @@ AssignTransactionId(TransactionState s)
                        rdata[0].next = &rdata[1];
 
                        rdata[1].data = (char *) unreportedXids;
-                       rdata[1].len = PGPROC_MAX_CACHED_SUBXIDS * sizeof(TransactionId);
+                       rdata[1].len = nUnreportedXids * sizeof(TransactionId);
                        rdata[1].buffer = InvalidBuffer;
                        rdata[1].next = NULL;
 
                        (void) XLogInsert(RM_XACT_ID, XLOG_XACT_ASSIGNMENT, rdata);
 
                        nUnreportedXids = 0;
+                       /* mark top, not current xact as having been logged */
+                       TopTransactionStateData.didLogXid = true;
                }
        }
 }
@@ -570,13 +610,34 @@ GetCurrentSubTransactionId(void)
        return s->subTransactionId;
 }
 
+/*
+ *     SubTransactionIsActive
+ *
+ * Test if the specified subxact ID is still active.  Note caller is
+ * responsible for checking whether this ID is relevant to the current xact.
+ */
+bool
+SubTransactionIsActive(SubTransactionId subxid)
+{
+       TransactionState s;
+
+       for (s = CurrentTransactionState; s != NULL; s = s->parent)
+       {
+               if (s->state == TRANS_ABORT)
+                       continue;
+               if (s->subTransactionId == subxid)
+                       return true;
+       }
+       return false;
+}
+
 
 /*
  *     GetCurrentCommandId
  *
  * "used" must be TRUE if the caller intends to use the command ID to mark
  * inserted/updated/deleted tuples.  FALSE means the ID is being fetched
- * for read-only purposes (ie, as a snapshot validity cutoff). See
+ * for read-only purposes (ie, as a snapshot validity cutoff).  See
  * CommandCounterIncrement() for discussion.
  */
 CommandId
@@ -663,7 +724,7 @@ TransactionIdIsCurrentTransactionId(TransactionId xid)
 
        /*
         * We always say that BootstrapTransactionId is "not my transaction ID"
-        * even when it is (ie, during bootstrap).      Along with the fact that
+        * even when it is (ie, during bootstrap).  Along with the fact that
         * transam.c always treats BootstrapTransactionId as already committed,
         * this causes the tqual.c routines to see all tuples as committed, which
         * is what we need during bootstrap.  (Bootstrap mode only inserts tuples,
@@ -745,12 +806,12 @@ CommandCounterIncrement(void)
        if (currentCommandIdUsed)
        {
                currentCommandId += 1;
-               if (currentCommandId == FirstCommandId) /* check for overflow */
+               if (currentCommandId == InvalidCommandId)
                {
                        currentCommandId -= 1;
                        ereport(ERROR,
                                        (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
-                                        errmsg("cannot have more than 2^32-1 commands in a transaction")));
+                                        errmsg("cannot have more than 2^32-2 commands in a transaction")));
                }
                currentCommandIdUsed = false;
 
@@ -805,7 +866,7 @@ AtStart_Memory(void)
        /*
         * If this is the first time through, create a private context for
         * AbortTransaction to work in.  By reserving some space now, we can
-        * insulate AbortTransaction from out-of-memory scenarios.      Like
+        * insulate AbortTransaction from out-of-memory scenarios.  Like
         * ErrorContext, we set it up with slow growth rate and a nonzero minimum
         * size, so that space will be reserved immediately.
         */
@@ -908,7 +969,7 @@ AtSubStart_ResourceOwner(void)
        Assert(s->parent != NULL);
 
        /*
-        * Create a resource owner for the subtransaction.      We make it a child of
+        * Create a resource owner for the subtransaction.  We make it a child of
         * the immediate parent's resource owner.
         */
        s->curTransactionOwner =
@@ -928,7 +989,7 @@ 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 the xact has no XID.  (We compute that here just because it's easier.)
  */
 static TransactionId
 RecordTransactionCommit(void)
@@ -973,7 +1034,7 @@ RecordTransactionCommit(void)
 
                /*
                 * If we didn't create XLOG entries, we're done here; otherwise we
-                * should flush those entries the same as a commit record.      (An
+                * should flush those entries the same as a commit record.  (An
                 * example of a possible record that wouldn't cause an XID to be
                 * assigned is a sequence advance record due to nextval() --- we want
                 * to flush that to disk before reporting commit.)
@@ -990,7 +1051,7 @@ RecordTransactionCommit(void)
                BufmgrCommit();
 
                /*
-                * Mark ourselves as within our "commit critical section".      This
+                * Mark ourselves as within our "commit critical section".  This
                 * forces any concurrent checkpoint to wait until we've updated
                 * pg_clog.  Without this, it is possible for the checkpoint to set
                 * REDO after the XLOG record but fail to flush the pg_clog update to
@@ -998,13 +1059,13 @@ RecordTransactionCommit(void)
                 * crashes a little later.
                 *
                 * Note: we could, but don't bother to, set this flag in
-                * RecordTransactionAbort.      That's because loss of a transaction abort
+                * RecordTransactionAbort.  That's because loss of a transaction abort
                 * is noncritical; the presumption would be that it aborted, anyway.
                 *
                 * It's safe to change the delayChkpt flag of our own backend without
                 * holding the ProcArrayLock, since we're the only one modifying it.
-                * This makes checkpoint's determination of which xacts are delayChkpt a
-                * bit fuzzy, but it doesn't matter.
+                * This makes checkpoint's determination of which xacts are delayChkpt
+                * bit fuzzy, but it doesn't matter.
                 */
                START_CRIT_SECTION();
                MyPgXact->delayChkpt = true;
@@ -1013,8 +1074,16 @@ RecordTransactionCommit(void)
 
                /*
                 * Do we need the long commit record? If not, use the compact format.
+                *
+                * For now always use the non-compact version if wal_level=logical, so
+                * we can hide commits from other databases. TODO: In the future we
+                * should merge compact and non-compact commits and use a flags
+                * variable to determine if it contains subxacts, relations or
+                * invalidation messages, that's more extensible and degrades more
+                * gracefully. Till then, it's just 20 bytes of overhead.
                 */
-               if (nrels > 0 || nmsgs > 0 || RelcacheInitFileInval || forceSyncCommit)
+               if (nrels > 0 || nmsgs > 0 || RelcacheInitFileInval || forceSyncCommit ||
+                       XLogLogicalInfoActive())
                {
                        XLogRecData rdata[4];
                        int                     lastrdata = 0;
@@ -1099,15 +1168,15 @@ RecordTransactionCommit(void)
        /*
         * Check if we want to commit asynchronously.  We can allow the XLOG flush
         * to happen asynchronously if synchronous_commit=off, or if the current
-        * transaction has not performed any WAL-logged operation.      The latter
+        * transaction has not performed any WAL-logged operation.  The latter
         * case can arise if the current transaction wrote only to temporary
-        * and/or unlogged tables.      In case of a crash, the loss of such a
+        * and/or unlogged tables.  In case of a crash, the loss of such a
         * transaction will be irrelevant since temp tables will be lost anyway,
         * and unlogged tables will be truncated.  (Given the foregoing, you might
         * think that it would be unnecessary to emit the XLOG record at all in
         * this case, but we don't currently try to do that.  It would certainly
         * cause problems at least in Hot Standby mode, where the
-        * KnownAssignedXids machinery requires tracking every XID assignment.  It
+        * KnownAssignedXids machinery requires tracking every XID assignment.  It
         * might be OK to skip it only when wal_level < hot_standby, but for now
         * we don't.)
         *
@@ -1354,7 +1423,7 @@ 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.)
+ * if the xact has no XID.  (We compute that here just because it's easier.)
  */
 static TransactionId
 RecordTransactionAbort(bool isSubXact)
@@ -1371,7 +1440,7 @@ RecordTransactionAbort(bool isSubXact)
 
        /*
         * If we haven't been assigned an XID, nobody will care whether we aborted
-        * or not.      Hence, we're done in that case.  It does not matter if we have
+        * or not.  Hence, we're done in that case.  It does not matter if we have
         * rels to delete (note that this routine is not responsible for actually
         * deleting 'em).  We cannot have any child XIDs, either.
         */
@@ -1387,7 +1456,7 @@ RecordTransactionAbort(bool isSubXact)
         * We have a valid XID, so we should write an ABORT record for it.
         *
         * We do not flush XLOG to disk here, since the default assumption after a
-        * crash would be that we aborted, anyway.      For the same reason, we don't
+        * crash would be that we aborted, anyway.  For the same reason, we don't
         * need to worry about interlocking against checkpoint start.
         */
 
@@ -1555,7 +1624,7 @@ AtSubAbort_childXids(void)
 
        /*
         * We keep the child-XID arrays in TopTransactionContext (see
-        * AtSubCommit_childXids).      This means we'd better free the array
+        * AtSubCommit_childXids).  This means we'd better free the array
         * explicitly at abort to avoid leakage.
         */
        if (s->childXids != NULL)
@@ -1712,6 +1781,7 @@ StartTransaction(void)
         * initialize reported xid accounting
         */
        nUnreportedXids = 0;
+       s->didLogXid = false;
 
        /*
         * must initialize resource-management stuff first
@@ -1732,7 +1802,7 @@ StartTransaction(void)
        VirtualXactLockTableInsert(vxid);
 
        /*
-        * Advertise it in the proc array.      We assume assignment of
+        * Advertise it in the proc array.  We assume assignment of
         * LocalTransactionID is atomic, and the backendId should be set already.
         */
        Assert(MyProc->backendId == vxid.backendId);
@@ -1825,9 +1895,11 @@ CommitTransaction(void)
                        break;
        }
 
+       CallXactCallbacks(XACT_EVENT_PRE_COMMIT);
+
        /*
         * The remaining actions cannot call any user-defined code, so it's safe
-        * to start shutting down within-transaction services.  But note that most
+        * to start shutting down within-transaction services.  But note that most
         * of this stuff could still throw an error, which would switch us into
         * the transaction-abort path.
         */
@@ -2028,9 +2100,11 @@ PrepareTransaction(void)
                        break;
        }
 
+       CallXactCallbacks(XACT_EVENT_PRE_PREPARE);
+
        /*
         * The remaining actions cannot call any user-defined code, so it's safe
-        * to start shutting down within-transaction services.  But note that most
+        * to start shutting down within-transaction services.  But note that most
         * of this stuff could still throw an error, which would switch us into
         * the transaction-abort path.
         */
@@ -2150,7 +2224,7 @@ PrepareTransaction(void)
        XactLastRecEnd = 0;
 
        /*
-        * Let others know about no transaction in progress by me.      This has to be
+        * Let others know about no transaction in progress by me.  This has to be
         * done *after* the prepared transaction has been marked valid, else
         * someone may think it is unlocked and recyclable.
         */
@@ -2159,7 +2233,7 @@ PrepareTransaction(void)
        /*
         * This is all post-transaction cleanup.  Note that if an error is raised
         * here, it's too late to abort the transaction.  This should be just
-        * noncritical resource releasing.      See notes in CommitTransaction.
+        * noncritical resource releasing.  See notes in CommitTransaction.
         */
 
        CallXactCallbacks(XACT_EVENT_PREPARE);
@@ -2206,8 +2280,9 @@ PrepareTransaction(void)
        AtEOXact_Files();
        AtEOXact_ComboCid();
        AtEOXact_HashTables(true);
-       /* don't call AtEOXact_PgStat here */
+       /* don't call AtEOXact_PgStat here; we fixed pgstat state above */
        AtEOXact_Snapshot(true);
+       pgstat_report_xact_timestamp(0);
 
        CurrentResourceOwner = NULL;
        ResourceOwnerDelete(TopTransactionResourceOwner);
@@ -2269,6 +2344,22 @@ AbortTransaction(void)
         */
        LockErrorCleanup();
 
+       /*
+        * If any timeout events are still active, make sure the timeout interrupt
+        * is scheduled.  This covers possible loss of a timeout interrupt due to
+        * longjmp'ing out of the SIGINT handler (see notes in handle_sig_alarm).
+        * We delay this till after LockErrorCleanup so that we don't uselessly
+        * reschedule lock or deadlock check timeouts.
+        */
+       reschedule_timeouts();
+
+       /*
+        * Re-enable signals, in case we got here by longjmp'ing out of a signal
+        * handler.  We do this fairly early in the sequence so that the timeout
+        * infrastructure will be functional if needed while aborting.
+        */
+       PG_SETMASK(&UnBlockSig);
+
        /*
         * check the current transaction state
         */
@@ -2320,7 +2411,7 @@ AbortTransaction(void)
        ProcArrayEndTransaction(MyProc, latestXid);
 
        /*
-        * Post-abort cleanup.  See notes in CommitTransaction() concerning
+        * Post-abort cleanup.  See notes in CommitTransaction() concerning
         * ordering.  We can skip all of it if the transaction failed before
         * creating a resource owner.
         */
@@ -2555,7 +2646,7 @@ CommitTransactionCommand(void)
 
                        /*
                         * Here we were in a perfectly good transaction block but the user
-                        * told us to ROLLBACK anyway.  We have to abort the transaction
+                        * told us to ROLLBACK anyway.  We have to abort the transaction
                         * and then clean up.
                         */
                case TBLOCK_ABORT_PENDING:
@@ -2575,7 +2666,7 @@ CommitTransactionCommand(void)
 
                        /*
                         * We were just issued a SAVEPOINT inside a transaction block.
-                        * Start a subtransaction.      (DefineSavepoint already did
+                        * Start a subtransaction.  (DefineSavepoint already did
                         * PushTransaction, so as to have someplace to put the SUBBEGIN
                         * state.)
                         */
@@ -2779,7 +2870,7 @@ AbortCurrentTransaction(void)
                        break;
 
                        /*
-                        * Here, we failed while trying to COMMIT.      Clean up the
+                        * Here, we failed while trying to COMMIT.  Clean up the
                         * transaction and return to idle state (we do not want to stay in
                         * the transaction).
                         */
@@ -2841,7 +2932,7 @@ AbortCurrentTransaction(void)
 
                        /*
                         * If we failed while trying to create a subtransaction, clean up
-                        * the broken subtransaction and abort the parent.      The same
+                        * the broken subtransaction and abort the parent.  The same
                         * applies if we get a failure while ending a subtransaction.
                         */
                case TBLOCK_SUBBEGIN:
@@ -2923,6 +3014,26 @@ PreventTransactionChain(bool isTopLevel, const char *stmtType)
        /* all okay */
 }
 
+/*
+ *     These two functions allow for warnings or errors if a command is
+ *     executed outside of a transaction block.
+ *
+ *     While top-level transaction control commands (BEGIN/COMMIT/ABORT) and
+ *     SET that have no effect issue warnings, all other no-effect commands
+ *     generate errors.
+ */
+void
+WarnNoTransactionChain(bool isTopLevel, const char *stmtType)
+{
+       CheckTransactionChain(isTopLevel, false, stmtType);
+}
+
+void
+RequireTransactionChain(bool isTopLevel, const char *stmtType)
+{
+       CheckTransactionChain(isTopLevel, true, stmtType);
+}
+
 /*
  *     RequireTransactionChain
  *
@@ -2932,16 +3043,16 @@ PreventTransactionChain(bool isTopLevel, const char *stmtType)
  *     is presumably an error).  DECLARE CURSOR is an example.
  *
  *     If we appear to be running inside a user-defined function, we do not
- *     issue an error, since the function could issue more commands that make
+ *     issue anything, since the function could issue more commands that make
  *     use of the current statement's results.  Likewise subtransactions.
  *     Thus this is an inverse for PreventTransactionChain.
  *
  *     isTopLevel: passed down from ProcessUtility to determine whether we are
  *     inside a function.
- *     stmtType: statement type name, for error messages.
+ *     stmtType: statement type name, for warning or error messages.
  */
-void
-RequireTransactionChain(bool isTopLevel, const char *stmtType)
+static void
+CheckTransactionChain(bool isTopLevel, bool throwError, const char *stmtType)
 {
        /*
         * xact block already started?
@@ -2961,11 +3072,12 @@ RequireTransactionChain(bool isTopLevel, const char *stmtType)
        if (!isTopLevel)
                return;
 
-       ereport(ERROR,
+       ereport(throwError ? ERROR : WARNING,
                        (errcode(ERRCODE_NO_ACTIVE_SQL_TRANSACTION),
        /* translator: %s represents an SQL statement name */
                         errmsg("%s can only be used in transaction blocks",
                                        stmtType)));
+       return;
 }
 
 /*
@@ -3373,7 +3485,7 @@ UserAbortTransactionBlock(void)
                        break;
 
                        /*
-                        * We are inside a subtransaction.      Mark everything up to top
+                        * We are inside a subtransaction.  Mark everything up to top
                         * level as exitable.
                         */
                case TBLOCK_SUBINPROGRESS:
@@ -3400,12 +3512,12 @@ UserAbortTransactionBlock(void)
 
                        /*
                         * The user issued ABORT when not inside a transaction. Issue a
-                        * NOTICE and go to abort state.  The upcoming call to
+                        * WARNING and go to abort state.  The upcoming call to
                         * CommitTransactionCommand() will then put us back into the
                         * default state.
                         */
                case TBLOCK_STARTED:
-                       ereport(NOTICE,
+                       ereport(WARNING,
                                        (errcode(ERRCODE_NO_ACTIVE_SQL_TRANSACTION),
                                         errmsg("there is no transaction in progress")));
                        s->blockState = TBLOCK_ABORT_PENDING;
@@ -3507,7 +3619,7 @@ ReleaseSavepoint(List *options)
                        break;
 
                        /*
-                        * We are in a non-aborted subtransaction.      This is the only valid
+                        * We are in a non-aborted subtransaction.  This is the only valid
                         * case.
                         */
                case TBLOCK_SUBINPROGRESS:
@@ -3564,7 +3676,7 @@ ReleaseSavepoint(List *options)
 
        /*
         * Mark "commit pending" all subtransactions up to the target
-        * subtransaction.      The actual commits will happen when control gets to
+        * subtransaction.  The actual commits will happen when control gets to
         * CommitTransactionCommand.
         */
        xact = CurrentTransactionState;
@@ -3663,7 +3775,7 @@ RollbackToSavepoint(List *options)
 
        /*
         * Mark "abort pending" all subtransactions up to the target
-        * subtransaction.      The actual aborts will happen when control gets to
+        * subtransaction.  The actual aborts will happen when control gets to
         * CommitTransactionCommand.
         */
        xact = CurrentTransactionState;
@@ -4058,15 +4170,19 @@ CommitSubTransaction(void)
                elog(WARNING, "CommitSubTransaction while in %s state",
                         TransStateAsString(s->state));
 
-       /* Pre-commit processing goes here -- nothing to do at the moment */
+       /* Pre-commit processing goes here */
 
+       CallSubXactCallbacks(SUBXACT_EVENT_PRE_COMMIT_SUB, s->subTransactionId,
+                                                s->parent->subTransactionId);
+
+       /* Do the actual "commit", such as it is */
        s->state = TRANS_COMMIT;
 
        /* Must CCI to ensure commands of subtransaction are seen as done */
        CommandCounterIncrement();
 
        /*
-        * Prior to 8.4 we marked subcommit in clog at this point.      We now only
+        * Prior to 8.4 we marked subcommit in clog at this point.  We now only
         * perform that step, if required, as part of the atomic update of the
         * whole transaction tree at top level commit or abort.
         */
@@ -4170,8 +4286,28 @@ AbortSubTransaction(void)
        AbortBufferIO();
        UnlockBuffers();
 
+       /*
+        * Also clean up any open wait for lock, since the lock manager will choke
+        * if we try to wait for another lock before doing this.
+        */
        LockErrorCleanup();
 
+       /*
+        * If any timeout events are still active, make sure the timeout interrupt
+        * is scheduled.  This covers possible loss of a timeout interrupt due to
+        * longjmp'ing out of the SIGINT handler (see notes in handle_sig_alarm).
+        * We delay this till after LockErrorCleanup so that we don't uselessly
+        * reschedule lock or deadlock check timeouts.
+        */
+       reschedule_timeouts();
+
+       /*
+        * Re-enable signals, in case we got here by longjmp'ing out of a signal
+        * handler.  We do this fairly early in the sequence so that the timeout
+        * infrastructure will be functional if needed while aborting.
+        */
+       PG_SETMASK(&UnBlockSig);
+
        /*
         * check the current transaction state
         */
@@ -4505,7 +4641,7 @@ TransStateAsString(TransState state)
 /*
  * xactGetCommittedChildren
  *
- * Gets the list of committed children of the current transaction.     The return
+ * Gets the list of committed children of the current transaction.  The return
  * value is the number of child transactions.  *ptr is set to point to an
  * array of TransactionIds.  The array is allocated in TopTransactionContext;
  * the caller should *not* pfree() it (this is a change from pre-8.4 code!).
@@ -4654,12 +4790,11 @@ xact_redo_commit_internal(TransactionId xid, XLogRecPtr lsn,
         * from the template database, and then commit the transaction. If we
         * crash after all the files have been copied but before the commit, you
         * have files in the data directory without an entry in pg_database. To
-        * minimize the window
-        * for that, we use ForceSyncCommit() to rush the commit record to disk as
-        * quick as possible. We have the same window during recovery, and forcing
-        * an XLogFlush() (which updates minRecoveryPoint during recovery) helps
-        * to reduce that problem window, for any user that requested
-        * ForceSyncCommit().
+        * minimize the window for that, we use ForceSyncCommit() to rush the
+        * commit record to disk as quick as possible. We have the same window
+        * during recovery, and forcing an XLogFlush() (which updates
+        * minRecoveryPoint during recovery) helps to reduce that problem window,
+        * for any user that requested ForceSyncCommit().
         */
        if (XactCompletionForceSyncCommit(xinfo))
                XLogFlush(lsn);