From a8ce974cddef3957b0615d743a1d090d12e81d50 Mon Sep 17 00:00:00 2001
From: Simon Riggs <simon@2ndQuadrant.com>
Date: Sat, 16 Jan 2010 10:05:59 +0000
Subject: [PATCH] Teach standby conflict resolution to use SIGUSR1 Conflict
 reason is passed through directly to the backend, so we can take decisions
 about the effect of the conflict based upon the local state. No specific
 changes, as yet, though this prepares for later work.
 CancelVirtualTransaction() sends signals while holding ProcArrayLock.
 Introduce errdetail_abort() to give message detail explaining that the abort
 was caused by conflict processing. Remove CONFLICT_MODE states in favour of
 using PROCSIG_RECOVERY_CONFLICT states directly, for clarity.

---
 src/backend/access/transam/xact.c    |   5 +-
 src/backend/storage/ipc/procarray.c  |  52 ++++---
 src/backend/storage/ipc/procsignal.c |  16 ++-
 src/backend/storage/ipc/standby.c    |  67 ++-------
 src/backend/storage/lmgr/proc.c      |   4 +-
 src/backend/tcop/postgres.c          | 201 +++++++++++++++++++--------
 src/include/storage/proc.h           |  10 +-
 src/include/storage/procarray.h      |   6 +-
 src/include/storage/procsignal.h     |   8 +-
 src/include/storage/standby.h        |   7 +-
 src/include/tcop/tcopprot.h          |   4 +-
 11 files changed, 227 insertions(+), 153 deletions(-)

diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 5ecd3c5725b..1a89d78ba95 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -10,7 +10,7 @@
  *
  *
  * IDENTIFICATION
- *	  $PostgreSQL: pgsql/src/backend/access/transam/xact.c,v 1.280 2010/01/09 16:49:27 sriggs Exp $
+ *	  $PostgreSQL: pgsql/src/backend/access/transam/xact.c,v 1.281 2010/01/16 10:05:50 sriggs Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -313,8 +313,7 @@ IsTransactionState(void)
 /*
  *	IsAbortedTransactionBlockState
  *
- *	This returns true if we are currently running a query
- *	within an aborted transaction block.
+ *	This returns true if we are within an aborted transaction block.
  */
 bool
 IsAbortedTransactionBlockState(void)
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index 01a7c2123f1..6075de55c53 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -37,7 +37,7 @@
  *
  *
  * IDENTIFICATION
- *	  $PostgreSQL: pgsql/src/backend/storage/ipc/procarray.c,v 1.55 2010/01/10 15:44:28 sriggs Exp $
+ *	  $PostgreSQL: pgsql/src/backend/storage/ipc/procarray.c,v 1.56 2010/01/16 10:05:50 sriggs Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -324,6 +324,7 @@ ProcArrayEndTransaction(PGPROC *proc, TransactionId latestXid)
 		/* must be cleared with xid/xmin: */
 		proc->vacuumFlags &= ~PROC_VACUUM_STATE_MASK;
 		proc->inCommit = false; /* be sure this is cleared in abort */
+		proc->recoveryConflictPending = false;
 
 		/* Clear the subtransaction-XID cache too while holding the lock */
 		proc->subxids.nxids = 0;
@@ -350,6 +351,7 @@ ProcArrayEndTransaction(PGPROC *proc, TransactionId latestXid)
 		/* must be cleared with xid/xmin: */
 		proc->vacuumFlags &= ~PROC_VACUUM_STATE_MASK;
 		proc->inCommit = false; /* be sure this is cleared in abort */
+		proc->recoveryConflictPending = false;
 
 		Assert(proc->subxids.nxids == 0);
 		Assert(proc->subxids.overflowed == false);
@@ -377,7 +379,7 @@ ProcArrayClearTransaction(PGPROC *proc)
 	proc->xid = InvalidTransactionId;
 	proc->lxid = InvalidLocalTransactionId;
 	proc->xmin = InvalidTransactionId;
-	proc->recoveryConflictMode = 0;
+	proc->recoveryConflictPending = false;
 
 	/* redundant, but just in case */
 	proc->vacuumFlags &= ~PROC_VACUUM_STATE_MASK;
@@ -1665,7 +1667,7 @@ GetConflictingVirtualXIDs(TransactionId limitXmin, Oid dbOid,
 		if (proc->pid == 0)
 			continue;
 
-		if (skipExistingConflicts && proc->recoveryConflictMode > 0)
+		if (skipExistingConflicts && proc->recoveryConflictPending)
 			continue;
 
 		if (!OidIsValid(dbOid) ||
@@ -1704,7 +1706,7 @@ GetConflictingVirtualXIDs(TransactionId limitXmin, Oid dbOid,
  * Returns pid of the process signaled, or 0 if not found.
  */
 pid_t
-CancelVirtualTransaction(VirtualTransactionId vxid, int cancel_mode)
+CancelVirtualTransaction(VirtualTransactionId vxid, ProcSignalReason sigmode)
 {
 	ProcArrayStruct *arrayP = procArray;
 	int			index;
@@ -1722,28 +1724,22 @@ CancelVirtualTransaction(VirtualTransactionId vxid, int cancel_mode)
 		if (procvxid.backendId == vxid.backendId &&
 			procvxid.localTransactionId == vxid.localTransactionId)
 		{
-			/*
-			 * Issue orders for the proc to read next time it receives SIGINT
-			 */
-			if (proc->recoveryConflictMode < cancel_mode)
-				proc->recoveryConflictMode = cancel_mode;
-
+			proc->recoveryConflictPending = true;
 			pid = proc->pid;
+			if (pid != 0)
+			{
+				/*
+				 * Kill the pid if it's still here. If not, that's what we wanted
+				 * so ignore any errors.
+				 */
+				(void) SendProcSignal(pid, sigmode, vxid.backendId);
+			}
 			break;
 		}
 	}
 
 	LWLockRelease(ProcArrayLock);
 
-	if (pid != 0)
-	{
-		/*
-		 * Kill the pid if it's still here. If not, that's what we wanted
-		 * so ignore any errors.
-		 */
-		kill(pid, SIGINT);
-	}
-
 	return pid;
 }
 
@@ -1834,6 +1830,7 @@ CancelDBBackends(Oid databaseid)
 {
 	ProcArrayStruct *arrayP = procArray;
 	int			index;
+	pid_t		pid = 0;
 
 	/* tell all backends to die */
 	LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
@@ -1844,8 +1841,21 @@ CancelDBBackends(Oid databaseid)
 
 		if (proc->databaseId == databaseid)
 		{
-			proc->recoveryConflictMode = CONFLICT_MODE_FATAL;
-			kill(proc->pid, SIGINT);
+			VirtualTransactionId procvxid;
+
+			GET_VXID_FROM_PGPROC(procvxid, *proc);
+
+			proc->recoveryConflictPending = true;
+			pid = proc->pid;
+			if (pid != 0)
+			{
+				/*
+				 * Kill the pid if it's still here. If not, that's what we wanted
+				 * so ignore any errors.
+				 */
+				(void) SendProcSignal(pid, PROCSIG_RECOVERY_CONFLICT_DATABASE,
+										procvxid.backendId);
+			}
 		}
 	}
 
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index 753306ca295..453d94aea5b 100644
--- a/src/backend/storage/ipc/procsignal.c
+++ b/src/backend/storage/ipc/procsignal.c
@@ -8,7 +8,7 @@
  * Portions Copyright (c) 1994, Regents of the University of California
  *
  * IDENTIFICATION
- *	  $PostgreSQL: pgsql/src/backend/storage/ipc/procsignal.c,v 1.2 2010/01/02 16:57:51 momjian Exp $
+ *	  $PostgreSQL: pgsql/src/backend/storage/ipc/procsignal.c,v 1.3 2010/01/16 10:05:50 sriggs Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -24,6 +24,8 @@
 #include "storage/procsignal.h"
 #include "storage/shmem.h"
 #include "storage/sinval.h"
+#include "storage/standby.h"
+#include "tcop/tcopprot.h"
 
 
 /*
@@ -258,5 +260,17 @@ procsignal_sigusr1_handler(SIGNAL_ARGS)
 	if (CheckProcSignal(PROCSIG_NOTIFY_INTERRUPT))
 		HandleNotifyInterrupt();
 
+	if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_DATABASE))
+		RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_DATABASE);
+
+	if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_TABLESPACE))
+		RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_TABLESPACE);
+
+	if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_LOCK))
+		RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_LOCK);
+
+	if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_SNAPSHOT))
+		RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_SNAPSHOT);
+
 	errno = save_errno;
 }
diff --git a/src/backend/storage/ipc/standby.c b/src/backend/storage/ipc/standby.c
index bc9302c4bca..5eca3c89dba 100644
--- a/src/backend/storage/ipc/standby.c
+++ b/src/backend/storage/ipc/standby.c
@@ -3,19 +3,15 @@
  * standby.c
  *	  Misc functions used in Hot Standby mode.
  *
- *	InitRecoveryTransactionEnvironment()
- *  ShutdownRecoveryTransactionEnvironment()
- *
- *  ResolveRecoveryConflictWithVirtualXIDs()
- *
  *  All functions for handling RM_STANDBY_ID, which relate to
  *  AccessExclusiveLocks and starting snapshots for Hot Standby mode.
+ *  Plus conflict recovery processing.
  *
  * Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
  *
  * IDENTIFICATION
- *	  $PostgreSQL: pgsql/src/backend/storage/ipc/standby.c,v 1.4 2010/01/14 11:08:02 sriggs Exp $
+ *	  $PostgreSQL: pgsql/src/backend/storage/ipc/standby.c,v 1.5 2010/01/16 10:05:50 sriggs Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -38,7 +34,7 @@ int		vacuum_defer_cleanup_age;
 static List *RecoveryLockList;
 
 static void ResolveRecoveryConflictWithVirtualXIDs(VirtualTransactionId *waitlist,
-									   char *reason, int cancel_mode);
+									   ProcSignalReason reason);
 static void ResolveRecoveryConflictWithLock(Oid dbOid, Oid relOid);
 static void LogCurrentRunningXacts(RunningTransactions CurrRunningXacts);
 static void LogAccessExclusiveLocks(int nlocks, xl_standby_lock *locks);
@@ -162,17 +158,13 @@ WaitExceedsMaxStandbyDelay(void)
  * recovery processing. Judgement has already been passed on it within
  * a specific rmgr. Here we just issue the orders to the procs. The procs
  * then throw the required error as instructed.
- *
- * We may ask for a specific cancel_mode, typically ERROR or FATAL.
  */
 static void
 ResolveRecoveryConflictWithVirtualXIDs(VirtualTransactionId *waitlist,
-									   char *reason, int cancel_mode)
+									   ProcSignalReason reason)
 {
 	char		waitactivitymsg[100];
 
-	Assert(cancel_mode > 0);
-
 	while (VirtualTransactionIdIsValid(*waitlist))
 	{
 		long wait_s;
@@ -206,12 +198,6 @@ ResolveRecoveryConflictWithVirtualXIDs(VirtualTransactionId *waitlist,
 					len = 100;
 				memcpy(waitactivitymsg, oldactivitymsg, len);
 
-				ereport(trace_recovery(DEBUG5),
-						(errmsg("virtual transaction %u/%u is blocking %s",
-								waitlist->backendId,
-								waitlist->localTransactionId,
-								reason)));
-
 				pgstat_report_waiting(true);
 
 				logged = true;
@@ -226,40 +212,14 @@ ResolveRecoveryConflictWithVirtualXIDs(VirtualTransactionId *waitlist,
 				 * Now find out who to throw out of the balloon.
 				 */
 				Assert(VirtualTransactionIdIsValid(*waitlist));
-				pid = CancelVirtualTransaction(*waitlist, cancel_mode);
+				pid = CancelVirtualTransaction(*waitlist, reason);
 
+				/*
+				 * Wait awhile for it to die so that we avoid flooding an
+				 * unresponsive backend when system is heavily loaded.
+				 */
 				if (pid != 0)
-				{
-					/*
-					 * Startup process debug messages
-					 */
-					switch (cancel_mode)
-					{
-						case CONFLICT_MODE_FATAL:
-							elog(trace_recovery(DEBUG1),
-								 "recovery disconnects session with pid %ld because of conflict with %s",
-								 (long) pid,
-								 reason);
-							break;
-						case CONFLICT_MODE_ERROR:
-							elog(trace_recovery(DEBUG1),
-								 "recovery cancels virtual transaction %u/%u pid %ld because of conflict with %s",
-								 waitlist->backendId,
-								 waitlist->localTransactionId,
-								 (long) pid,
-								 reason);
-							break;
-						default:
-							/* No conflict pending, so fall through */
-							break;
-					}
-
-					/*
-					 * Wait awhile for it to die so that we avoid flooding an
-					 * unresponsive backend when system is heavily loaded.
-					 */
 					pg_usleep(5000);
-				}
 			}
 		}
 
@@ -285,8 +245,7 @@ ResolveRecoveryConflictWithSnapshot(TransactionId latestRemovedXid)
 										 true);
 
 	ResolveRecoveryConflictWithVirtualXIDs(backends,
-										   "snapshot conflict",
-										   CONFLICT_MODE_ERROR);
+										   PROCSIG_RECOVERY_CONFLICT_SNAPSHOT);
 }
 
 void
@@ -317,8 +276,7 @@ ResolveRecoveryConflictWithTablespace(Oid tsid)
 												InvalidOid,
 												false);
 	ResolveRecoveryConflictWithVirtualXIDs(temp_file_users,
-										   "drop tablespace",
-										   CONFLICT_MODE_ERROR);
+										   PROCSIG_RECOVERY_CONFLICT_TABLESPACE);
 }
 
 void
@@ -379,8 +337,7 @@ ResolveRecoveryConflictWithLock(Oid dbOid, Oid relOid)
 		}
 
 		ResolveRecoveryConflictWithVirtualXIDs(backends,
-											   "exclusive lock",
-											   CONFLICT_MODE_ERROR);
+											   PROCSIG_RECOVERY_CONFLICT_LOCK);
 
 		if (LockAcquireExtended(&locktag, AccessExclusiveLock, true, true, false)
 											!= LOCKACQUIRE_NOT_AVAIL)
diff --git a/src/backend/storage/lmgr/proc.c b/src/backend/storage/lmgr/proc.c
index 91065b368cd..ac891827ec1 100644
--- a/src/backend/storage/lmgr/proc.c
+++ b/src/backend/storage/lmgr/proc.c
@@ -8,7 +8,7 @@
  *
  *
  * IDENTIFICATION
- *	  $PostgreSQL: pgsql/src/backend/storage/lmgr/proc.c,v 1.212 2010/01/15 09:19:03 heikki Exp $
+ *	  $PostgreSQL: pgsql/src/backend/storage/lmgr/proc.c,v 1.213 2010/01/16 10:05:50 sriggs Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -324,7 +324,7 @@ InitProcess(void)
 	MyProc->waitProcLock = NULL;
 	for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
 		SHMQueueInit(&(MyProc->myProcLocks[i]));
-	MyProc->recoveryConflictMode = 0;
+	MyProc->recoveryConflictPending = false;
 
 	/*
 	 * We might be reusing a semaphore that belonged to a failed process. So
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index ad8678e2200..8d55e3575a0 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -8,7 +8,7 @@
  *
  *
  * IDENTIFICATION
- *	  $PostgreSQL: pgsql/src/backend/tcop/postgres.c,v 1.582 2010/01/15 09:19:04 heikki Exp $
+ *	  $PostgreSQL: pgsql/src/backend/tcop/postgres.c,v 1.583 2010/01/16 10:05:50 sriggs Exp $
  *
  * NOTES
  *	  this is the "main" module of the postgres backend and
@@ -63,7 +63,6 @@
 #include "storage/proc.h"
 #include "storage/procsignal.h"
 #include "storage/sinval.h"
-#include "storage/standby.h"
 #include "tcop/fastpath.h"
 #include "tcop/pquery.h"
 #include "tcop/tcopprot.h"
@@ -173,6 +172,8 @@ static int	UseNewLine = 1;		/* Use newlines query delimiters (the default) */
 static int	UseNewLine = 0;		/* Use EOF as query delimiters */
 #endif   /* TCOP_DONTUSENEWLINE */
 
+/* whether we were cancelled during recovery by conflict processing or not */
+static bool RecoveryConflictPending = false;
 
 /* ----------------------------------------------------------------
  *		decls for routines only used in this file
@@ -186,6 +187,7 @@ static List *pg_rewrite_query(Query *query);
 static bool check_log_statement(List *stmt_list);
 static int	errdetail_execute(List *raw_parsetree_list);
 static int	errdetail_params(ParamListInfo params);
+static int  errdetail_abort(void);
 static void start_xact_command(void);
 static void finish_xact_command(void);
 static bool IsTransactionExitStmt(Node *parsetree);
@@ -944,7 +946,8 @@ exec_simple_query(const char *query_string)
 			ereport(ERROR,
 					(errcode(ERRCODE_IN_FAILED_SQL_TRANSACTION),
 					 errmsg("current transaction is aborted, "
-						"commands ignored until end of transaction block")));
+							"commands ignored until end of transaction block"),
+					 errdetail_abort()));
 
 		/* Make sure we are in a transaction command */
 		start_xact_command();
@@ -1253,7 +1256,8 @@ exec_parse_message(const char *query_string,	/* string to execute */
 			ereport(ERROR,
 					(errcode(ERRCODE_IN_FAILED_SQL_TRANSACTION),
 					 errmsg("current transaction is aborted, "
-						"commands ignored until end of transaction block")));
+							"commands ignored until end of transaction block"),
+					 errdetail_abort()));
 
 		/*
 		 * Set up a snapshot if parse analysis/planning will need one.
@@ -1533,7 +1537,8 @@ exec_bind_message(StringInfo input_message)
 		ereport(ERROR,
 				(errcode(ERRCODE_IN_FAILED_SQL_TRANSACTION),
 				 errmsg("current transaction is aborted, "
-						"commands ignored until end of transaction block")));
+						"commands ignored until end of transaction block"),
+				 errdetail_abort()));
 
 	/*
 	 * Create the portal.  Allow silent replacement of an existing portal only
@@ -1974,7 +1979,8 @@ exec_execute_message(const char *portal_name, long max_rows)
 		ereport(ERROR,
 				(errcode(ERRCODE_IN_FAILED_SQL_TRANSACTION),
 				 errmsg("current transaction is aborted, "
-						"commands ignored until end of transaction block")));
+						"commands ignored until end of transaction block"),
+				 errdetail_abort()));
 
 	/* Check for cancel signal before we start execution */
 	CHECK_FOR_INTERRUPTS();
@@ -2234,6 +2240,20 @@ errdetail_params(ParamListInfo params)
 	return 0;
 }
 
+/*
+ * errdetail_abort
+ *
+ * Add an errdetail() line showing abort reason, if any.
+ */
+static int
+errdetail_abort(void)
+{
+	if (MyProc->recoveryConflictPending)
+		errdetail("abort reason: recovery conflict");
+
+	return 0;
+}
+
 /*
  * exec_describe_statement_message
  *
@@ -2291,7 +2311,8 @@ exec_describe_statement_message(const char *stmt_name)
 		ereport(ERROR,
 				(errcode(ERRCODE_IN_FAILED_SQL_TRANSACTION),
 				 errmsg("current transaction is aborted, "
-						"commands ignored until end of transaction block")));
+						"commands ignored until end of transaction block"),
+				 errdetail_abort()));
 
 	if (whereToSendOutput != DestRemote)
 		return;					/* can't actually do anything... */
@@ -2371,7 +2392,8 @@ exec_describe_portal_message(const char *portal_name)
 		ereport(ERROR,
 				(errcode(ERRCODE_IN_FAILED_SQL_TRANSACTION),
 				 errmsg("current transaction is aborted, "
-						"commands ignored until end of transaction block")));
+						"commands ignored until end of transaction block"),
+				 errdetail_abort()));
 
 	if (whereToSendOutput != DestRemote)
 		return;					/* can't actually do anything... */
@@ -2678,6 +2700,97 @@ SigHupHandler(SIGNAL_ARGS)
 	got_SIGHUP = true;
 }
 
+/*
+ * RecoveryConflictInterrupt: out-of-line portion of recovery conflict
+ * handling ollowing receipt of SIGUSR1. Designed to be similar to die()
+ * and StatementCancelHandler(). Called only by a normal user backend
+ * that begins a transaction during recovery.
+ */
+void
+RecoveryConflictInterrupt(ProcSignalReason reason)
+{
+	int                     save_errno = errno;
+
+	/*
+	* Don't joggle the elbow of proc_exit
+	*/
+	if (!proc_exit_inprogress)
+	{
+		switch (reason)
+		{
+			case PROCSIG_RECOVERY_CONFLICT_LOCK:
+			case PROCSIG_RECOVERY_CONFLICT_TABLESPACE:
+			case PROCSIG_RECOVERY_CONFLICT_SNAPSHOT:
+					/*
+					 * If we aren't in a transaction any longer then ignore.
+					 */
+					if (!IsTransactionOrTransactionBlock())
+						return;
+
+					/*
+					 * If we can abort just the current subtransaction then we
+					 * are OK to throw an ERROR to resolve the conflict. Otherwise
+					 * drop through to the FATAL case.
+					 *
+					 * XXX other times that we can throw just an ERROR *may* be
+					 *   PROCSIG_RECOVERY_CONFLICT_LOCK
+					 *		if no locks are held in parent transactions
+					 *
+					 *   PROCSIG_RECOVERY_CONFLICT_SNAPSHOT
+					 *		if no snapshots are held by parent transactions
+					 *		and the transaction is not serializable
+					 *
+					 *   PROCSIG_RECOVERY_CONFLICT_TABLESPACE
+					 *		if no temp files or cursors open in parent transactions
+					 */
+					if (!IsSubTransaction())
+					{
+						/*
+						 * If we already aborted then we no longer need to cancel.
+						 * We do this here since we do not wish to ignore aborted
+						 * subtransactions, which must cause FATAL, currently.
+						 */
+						if (IsAbortedTransactionBlockState())
+							return;
+
+						RecoveryConflictPending = true;
+						QueryCancelPending = true;
+						InterruptPending = true;
+						break;
+					}
+
+					/* Intentional drop through to session cancel */
+
+			case PROCSIG_RECOVERY_CONFLICT_DATABASE:
+					RecoveryConflictPending = true;
+					ProcDiePending = true;
+					InterruptPending = true;
+					break;
+
+			default:
+					elog(FATAL, "Unknown conflict mode");
+		}
+
+		/*
+		 * If it's safe to interrupt, and we're waiting for input or a lock,
+		 * service the interrupt immediately
+		 */
+		if (ImmediateInterruptOK && InterruptHoldoffCount == 0 &&
+			CritSectionCount == 0)
+		{
+			/* bump holdoff count to make ProcessInterrupts() a no-op */
+			/* until we are done getting ready for it */
+			InterruptHoldoffCount++;
+			LockWaitCancel();	/* prevent CheckDeadLock from running */
+			DisableNotifyInterrupt();
+			DisableCatchupInterrupt();
+			InterruptHoldoffCount--;
+			ProcessInterrupts();
+		}
+	}
+
+	errno = save_errno;
+}
 
 /*
  * ProcessInterrupts: out-of-line portion of CHECK_FOR_INTERRUPTS() macro
@@ -2707,6 +2820,10 @@ ProcessInterrupts(void)
 			ereport(FATAL,
 					(errcode(ERRCODE_ADMIN_SHUTDOWN),
 					 errmsg("terminating autovacuum process due to administrator command")));
+		else if (RecoveryConflictPending)
+			ereport(FATAL,
+					(errcode(ERRCODE_ADMIN_SHUTDOWN),
+					 errmsg("terminating connection due to conflict with recovery")));
 		else
 			ereport(FATAL,
 					(errcode(ERRCODE_ADMIN_SHUTDOWN),
@@ -2745,57 +2862,26 @@ ProcessInterrupts(void)
 					(errcode(ERRCODE_QUERY_CANCELED),
 					 errmsg("canceling autovacuum task")));
 		}
+		if (RecoveryConflictPending)
 		{
-			int cancelMode = MyProc->recoveryConflictMode;
+			ImmediateInterruptOK = false;	/* not idle anymore */
+			DisableNotifyInterrupt();
+			DisableCatchupInterrupt();
 
-			/*
-			 * XXXHS: We don't yet have a clean way to cancel an
-			 * idle-in-transaction session, so make it FATAL instead.
-			 * This isn't as bad as it looks because we don't issue a
-			 * CONFLICT_MODE_ERROR for a session with proc->xmin == 0
-			 * on cleanup conflicts. There's a possibility that we
-			 * marked somebody as a conflict and then they go idle.
-			 */
-			if (DoingCommandRead && IsTransactionBlock() &&
-				cancelMode == CONFLICT_MODE_ERROR)
+			if (DoingCommandRead)
 			{
-				cancelMode = CONFLICT_MODE_FATAL;
+				ProcDiePending = false;
+				QueryCancelPending = false;
+				ereport(FATAL,
+						(errcode(ERRCODE_ADMIN_SHUTDOWN),
+						 errmsg("terminating connection due to conflict with recovery")));
 			}
-
-			switch (cancelMode)
+			else
 			{
-				case CONFLICT_MODE_FATAL:
-					ImmediateInterruptOK = false;	/* not idle anymore */
-					DisableNotifyInterrupt();
-					DisableCatchupInterrupt();
-					Assert(RecoveryInProgress());
-					ereport(FATAL,
-							(errcode(ERRCODE_QUERY_CANCELED),
-							 errmsg("canceling session due to conflict with recovery")));
-
-				case CONFLICT_MODE_ERROR:
-					/*
-					 * We are aborting because we need to release
-					 * locks. So we need to abort out of all
-					 * subtransactions to make sure we release
-					 * all locks at whatever their level.
-					 *
-					 * XXX Should we try to examine the
-					 * transaction tree and cancel just enough
-					 * subxacts to remove locks? Doubt it.
-					 */
-					ImmediateInterruptOK = false;	/* not idle anymore */
-					DisableNotifyInterrupt();
-					DisableCatchupInterrupt();
-					Assert(RecoveryInProgress());
-					AbortOutOfAnyTransaction();
-					ereport(ERROR,
-							(errcode(ERRCODE_QUERY_CANCELED),
-							 errmsg("canceling statement due to conflict with recovery")));
-
-				default:
-					/* No conflict pending, so fall through */
-					break;
+				QueryCancelPending = false;
+				ereport(ERROR,
+						(errcode(ERRCODE_QUERY_CANCELED),
+						 errmsg("canceling statement due to conflict with recovery")));
 			}
 		}
 
@@ -3637,7 +3723,12 @@ PostgresMain(int argc, char *argv[], const char *username)
 		 */
 		if (send_ready_for_query)
 		{
-			if (IsTransactionOrTransactionBlock())
+			if (IsAbortedTransactionBlockState())
+			{
+				set_ps_display("idle in transaction (aborted)", false);
+				pgstat_report_activity("<IDLE> in transaction (aborted)");
+			}
+			else if (IsTransactionOrTransactionBlock())
 			{
 				set_ps_display("idle in transaction", false);
 				pgstat_report_activity("<IDLE> in transaction");
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index cee02c359d7..50500903245 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -7,7 +7,7 @@
  * Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
  *
- * $PostgreSQL: pgsql/src/include/storage/proc.h,v 1.117 2010/01/15 09:19:09 heikki Exp $
+ * $PostgreSQL: pgsql/src/include/storage/proc.h,v 1.118 2010/01/16 10:05:50 sriggs Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -96,11 +96,11 @@ struct PGPROC
 	uint8		vacuumFlags;	/* vacuum-related flags, see above */
 
 	/*
-	 * While in hot standby mode, setting recoveryConflictMode instructs
-	 * the backend to commit suicide. Possible values are the same as those
-	 * passed to ResolveRecoveryConflictWithVirtualXIDs().
+	 * While in hot standby mode, shows that a conflict signal has been sent
+	 * for the current transaction. Set/cleared while holding ProcArrayLock,
+	 * though not required. Accessed without lock, if needed.
 	 */
-	int			recoveryConflictMode;
+	bool		recoveryConflictPending;
 
 	/* Info about LWLock the process is currently waiting for, if any. */
 	bool		lwWaiting;		/* true if waiting for an LW lock */
diff --git a/src/include/storage/procarray.h b/src/include/storage/procarray.h
index 4572f489af8..42953b0843c 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -7,7 +7,7 @@
  * Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
  *
- * $PostgreSQL: pgsql/src/include/storage/procarray.h,v 1.29 2010/01/10 15:44:28 sriggs Exp $
+ * $PostgreSQL: pgsql/src/include/storage/procarray.h,v 1.30 2010/01/16 10:05:56 sriggs Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -15,6 +15,7 @@
 #define PROCARRAY_H
 
 #include "storage/lock.h"
+#include "storage/procsignal.h"
 #include "storage/standby.h"
 #include "utils/snapshot.h"
 
@@ -58,8 +59,7 @@ extern VirtualTransactionId *GetCurrentVirtualXIDs(TransactionId limitXmin,
 					  int *nvxids);
 extern VirtualTransactionId *GetConflictingVirtualXIDs(TransactionId limitXmin,
 					Oid dbOid, bool skipExistingConflicts);
-extern pid_t CancelVirtualTransaction(VirtualTransactionId vxid,
-						 int cancel_mode);
+extern pid_t CancelVirtualTransaction(VirtualTransactionId vxid, ProcSignalReason sigmode);
 
 extern int	CountActiveBackends(void);
 extern int	CountDBBackends(Oid databaseid);
diff --git a/src/include/storage/procsignal.h b/src/include/storage/procsignal.h
index 9b707520ef8..cbe0e24c672 100644
--- a/src/include/storage/procsignal.h
+++ b/src/include/storage/procsignal.h
@@ -7,7 +7,7 @@
  * Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
  *
- * $PostgreSQL: pgsql/src/include/storage/procsignal.h,v 1.2 2010/01/02 16:58:08 momjian Exp $
+ * $PostgreSQL: pgsql/src/include/storage/procsignal.h,v 1.3 2010/01/16 10:05:57 sriggs Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -32,6 +32,12 @@ typedef enum
 	PROCSIG_CATCHUP_INTERRUPT,	/* sinval catchup interrupt */
 	PROCSIG_NOTIFY_INTERRUPT,	/* listen/notify interrupt */
 
+	/* Recovery conflict reasons */
+	PROCSIG_RECOVERY_CONFLICT_DATABASE,
+	PROCSIG_RECOVERY_CONFLICT_TABLESPACE,
+	PROCSIG_RECOVERY_CONFLICT_LOCK,
+	PROCSIG_RECOVERY_CONFLICT_SNAPSHOT,
+
 	NUM_PROCSIGNALS				/* Must be last! */
 } ProcSignalReason;
 
diff --git a/src/include/storage/standby.h b/src/include/storage/standby.h
index 3f2e2c2d855..8c982cffb63 100644
--- a/src/include/storage/standby.h
+++ b/src/include/storage/standby.h
@@ -7,7 +7,7 @@
  * Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
  *
- * $PostgreSQL: pgsql/src/include/storage/standby.h,v 1.3 2010/01/14 11:08:02 sriggs Exp $
+ * $PostgreSQL: pgsql/src/include/storage/standby.h,v 1.4 2010/01/16 10:05:57 sriggs Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -19,11 +19,6 @@
 
 extern int	vacuum_defer_cleanup_age;
 
-/* cancel modes for ResolveRecoveryConflictWithVirtualXIDs */
-#define CONFLICT_MODE_NOT_SET		0
-#define CONFLICT_MODE_ERROR			1	/* Conflict can be resolved by canceling query */
-#define CONFLICT_MODE_FATAL			2	/* Conflict can only be resolved by disconnecting session */
-
 extern void ResolveRecoveryConflictWithSnapshot(TransactionId latestRemovedXid);
 extern void ResolveRecoveryConflictWithTablespace(Oid tsid);
 extern void ResolveRecoveryConflictWithDatabase(Oid dbid);
diff --git a/src/include/tcop/tcopprot.h b/src/include/tcop/tcopprot.h
index 891979dc8a2..b46160160ea 100644
--- a/src/include/tcop/tcopprot.h
+++ b/src/include/tcop/tcopprot.h
@@ -7,7 +7,7 @@
  * Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
  *
- * $PostgreSQL: pgsql/src/include/tcop/tcopprot.h,v 1.102 2010/01/02 16:58:09 momjian Exp $
+ * $PostgreSQL: pgsql/src/include/tcop/tcopprot.h,v 1.103 2010/01/16 10:05:59 sriggs Exp $
  *
  * OLD COMMENTS
  *	  This file was created so that other c files could get the two
@@ -21,6 +21,7 @@
 
 #include "executor/execdesc.h"
 #include "nodes/parsenodes.h"
+#include "storage/procsignal.h"
 #include "utils/guc.h"
 
 
@@ -64,6 +65,7 @@ extern void die(SIGNAL_ARGS);
 extern void quickdie(SIGNAL_ARGS);
 extern void StatementCancelHandler(SIGNAL_ARGS);
 extern void FloatExceptionHandler(SIGNAL_ARGS);
+extern void RecoveryConflictInterrupt(ProcSignalReason reason); /* called from SIGUSR1 handler */
 extern void prepare_for_client_read(void);
 extern void client_read_ended(void);
 extern const char *process_postgres_switches(int argc, char *argv[],
-- 
GitLab