diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 4b60382778..e05f6eb732 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -6946,6 +6946,28 @@ log_line_prefix = '%m [%p] %q%u@%d/%a '
+
+ log_recovery_conflict_waits (boolean)
+
+ log_recovery_conflict_waits configuration parameter
+
+
+
+
+ Controls whether a log message is produced when the startup process
+ is waiting longer than deadlock_timeout
+ for recovery conflicts. This is useful in determining if recovery
+ conflicts prevent the recovery from applying WAL.
+
+
+
+ The default is off. This parameter can only be set
+ in the postgresql.conf file or on the server
+ command line.
+
+
+
+
log_parameter_max_length (integer)
diff --git a/doc/src/sgml/high-availability.sgml b/doc/src/sgml/high-availability.sgml
index 19d7bd2b28..2d32e141c2 100644
--- a/doc/src/sgml/high-availability.sgml
+++ b/doc/src/sgml/high-availability.sgml
@@ -2068,6 +2068,12 @@ if (!triggered)
server. The pg_stat_database system view also contains
summary information.
+
+
+ Users can control whether a log message is produced when WAL replay is waiting
+ longer than deadlock_timeout for conflicts. This
+ is controlled by the parameter.
+
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index ad0d1a9abc..8499ea54f5 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -3810,6 +3810,8 @@ LockBufferForCleanup(Buffer buffer)
{
BufferDesc *bufHdr;
char *new_status = NULL;
+ TimestampTz waitStart = 0;
+ bool logged_recovery_conflict = false;
Assert(BufferIsPinned(buffer));
Assert(PinCountWaitBuf == NULL);
@@ -3883,6 +3885,34 @@ LockBufferForCleanup(Buffer buffer)
new_status[len] = '\0'; /* truncate off " waiting" */
}
+ /*
+ * Emit the log message if the startup process is waiting longer
+ * than deadlock_timeout for recovery conflict on buffer pin.
+ *
+ * Skip this if first time through because the startup process has
+ * not started waiting yet in this case. So, the wait start
+ * timestamp is set after this logic.
+ */
+ if (waitStart != 0 && !logged_recovery_conflict)
+ {
+ TimestampTz now = GetCurrentTimestamp();
+
+ if (TimestampDifferenceExceeds(waitStart, now,
+ DeadlockTimeout))
+ {
+ LogRecoveryConflict(PROCSIG_RECOVERY_CONFLICT_BUFFERPIN,
+ waitStart, now, NULL);
+ logged_recovery_conflict = true;
+ }
+ }
+
+ /*
+ * Set the wait start timestamp if logging is enabled and first
+ * time through.
+ */
+ if (log_recovery_conflict_waits && waitStart == 0)
+ waitStart = GetCurrentTimestamp();
+
/* Publish the bufid that Startup process waits on */
SetStartupBufferPinWaitBufId(buffer - 1);
/* Set alarm and then wait to be signaled by UnpinBuffer() */
diff --git a/src/backend/storage/ipc/standby.c b/src/backend/storage/ipc/standby.c
index 52b2809dac..cf4e630aa2 100644
--- a/src/backend/storage/ipc/standby.c
+++ b/src/backend/storage/ipc/standby.c
@@ -39,6 +39,7 @@
int vacuum_defer_cleanup_age;
int max_standby_archive_delay = 30 * 1000;
int max_standby_streaming_delay = 30 * 1000;
+bool log_recovery_conflict_waits = false;
static HTAB *RecoveryLockLists;
@@ -49,6 +50,7 @@ static void ResolveRecoveryConflictWithVirtualXIDs(VirtualTransactionId *waitlis
static void SendRecoveryConflictWithBufferPin(ProcSignalReason reason);
static XLogRecPtr LogCurrentRunningXacts(RunningTransactions CurrRunningXacts);
static void LogAccessExclusiveLocks(int nlocks, xl_standby_lock *locks);
+static const char *get_recovery_conflict_desc(ProcSignalReason reason);
/*
* Keep track of all the locks owned by a given transaction.
@@ -215,15 +217,83 @@ WaitExceedsMaxStandbyDelay(uint32 wait_event_info)
return false;
}
+/*
+ * Log the recovery conflict.
+ *
+ * wait_start is the timestamp when the caller started to wait.
+ * now is the timestamp when this function has been called.
+ * wait_list is the list of virtual transaction ids assigned to
+ * conflicting processes.
+ */
+void
+LogRecoveryConflict(ProcSignalReason reason, TimestampTz wait_start,
+ TimestampTz now, VirtualTransactionId *wait_list)
+{
+ long secs;
+ int usecs;
+ long msecs;
+ StringInfoData buf;
+ int nprocs = 0;
+
+ TimestampDifference(wait_start, now, &secs, &usecs);
+ msecs = secs * 1000 + usecs / 1000;
+ usecs = usecs % 1000;
+
+ if (wait_list)
+ {
+ VirtualTransactionId *vxids;
+
+ /* Construct a string of list of the conflicting processes */
+ vxids = wait_list;
+ while (VirtualTransactionIdIsValid(*vxids))
+ {
+ PGPROC *proc = BackendIdGetProc(vxids->backendId);
+
+ /* proc can be NULL if the target backend is not active */
+ if (proc)
+ {
+ if (nprocs == 0)
+ {
+ initStringInfo(&buf);
+ appendStringInfo(&buf, "%d", proc->pid);
+ }
+ else
+ appendStringInfo(&buf, ", %d", proc->pid);
+
+ nprocs++;
+ }
+
+ vxids++;
+ }
+ }
+
+ /*
+ * If wait_list is specified, report the list of PIDs of active
+ * conflicting backends in a detail message. Note that if all the backends
+ * in the list are not active, no detail message is logged.
+ */
+ ereport(LOG,
+ errmsg("recovery still waiting after %ld.%03d ms: %s",
+ msecs, usecs, _(get_recovery_conflict_desc(reason))),
+ nprocs > 0 ? errdetail_log_plural("Conflicting process: %s.",
+ "Conflicting processes: %s.",
+ nprocs, buf.data) : 0);
+
+ if (nprocs > 0)
+ pfree(buf.data);
+}
+
/*
* This is the main executioner for any query backend that conflicts with
* 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.
*
- * If report_waiting is true, "waiting" is reported in PS display if necessary.
- * If the caller has already reported that, report_waiting should be false.
- * Otherwise, "waiting" is reported twice unexpectedly.
+ * If report_waiting is true, "waiting" is reported in PS display and the
+ * wait for recovery conflict is reported in the log, if necessary. If
+ * the caller is responsible for reporting them, report_waiting should be
+ * false. Otherwise, both the caller and this function report the same
+ * thing unexpectedly.
*/
static void
ResolveRecoveryConflictWithVirtualXIDs(VirtualTransactionId *waitlist,
@@ -231,15 +301,16 @@ ResolveRecoveryConflictWithVirtualXIDs(VirtualTransactionId *waitlist,
bool report_waiting)
{
TimestampTz waitStart = 0;
- char *new_status;
+ char *new_status = NULL;
+ bool logged_recovery_conflict = false;
/* Fast exit, to avoid a kernel call if there's no work to be done. */
if (!VirtualTransactionIdIsValid(*waitlist))
return;
- if (report_waiting)
+ /* Set the wait start timestamp for reporting */
+ if (report_waiting && (log_recovery_conflict_waits || update_process_title))
waitStart = GetCurrentTimestamp();
- new_status = NULL; /* we haven't changed the ps display */
while (VirtualTransactionIdIsValid(*waitlist))
{
@@ -249,25 +320,6 @@ ResolveRecoveryConflictWithVirtualXIDs(VirtualTransactionId *waitlist,
/* wait until the virtual xid is gone */
while (!VirtualXactLock(*waitlist, false))
{
- /*
- * Report via ps if we have been waiting for more than 500 msec
- * (should that be configurable?)
- */
- if (update_process_title && new_status == NULL && report_waiting &&
- TimestampDifferenceExceeds(waitStart, GetCurrentTimestamp(),
- 500))
- {
- const char *old_status;
- int len;
-
- old_status = get_ps_display(&len);
- new_status = (char *) palloc(len + 8 + 1);
- memcpy(new_status, old_status, len);
- strcpy(new_status + len, " waiting");
- set_ps_display(new_status);
- new_status[len] = '\0'; /* truncate off " waiting" */
- }
-
/* Is it time to kill it? */
if (WaitExceedsMaxStandbyDelay(wait_event_info))
{
@@ -286,6 +338,50 @@ ResolveRecoveryConflictWithVirtualXIDs(VirtualTransactionId *waitlist,
if (pid != 0)
pg_usleep(5000L);
}
+
+ if (waitStart != 0 && (!logged_recovery_conflict || new_status == NULL))
+ {
+ TimestampTz now = 0;
+ bool maybe_log_conflict;
+ bool maybe_update_title;
+
+ maybe_log_conflict = (log_recovery_conflict_waits && !logged_recovery_conflict);
+ maybe_update_title = (update_process_title && new_status == NULL);
+
+ /* Get the current timestamp if not report yet */
+ if (maybe_log_conflict || maybe_update_title)
+ now = GetCurrentTimestamp();
+
+ /*
+ * Report via ps if we have been waiting for more than 500
+ * msec (should that be configurable?)
+ */
+ if (maybe_update_title &&
+ TimestampDifferenceExceeds(waitStart, now, 500))
+ {
+ const char *old_status;
+ int len;
+
+ old_status = get_ps_display(&len);
+ new_status = (char *) palloc(len + 8 + 1);
+ memcpy(new_status, old_status, len);
+ strcpy(new_status + len, " waiting");
+ set_ps_display(new_status);
+ new_status[len] = '\0'; /* truncate off " waiting" */
+ }
+
+ /*
+ * Emit the log message if the startup process is waiting
+ * longer than deadlock_timeout for recovery conflict on
+ * buffer pin.
+ */
+ if (maybe_log_conflict &&
+ TimestampDifferenceExceeds(waitStart, now, DeadlockTimeout))
+ {
+ LogRecoveryConflict(reason, waitStart, now, waitlist);
+ logged_recovery_conflict = true;
+ }
+ }
}
/* The virtual transaction is gone now, wait for the next one */
@@ -398,9 +494,13 @@ ResolveRecoveryConflictWithDatabase(Oid dbid)
*
* Deadlocks involving the Startup process and an ordinary backend process
* will be detected by the deadlock detector within the ordinary backend.
+ *
+ * Enable the timer so that we can wake up after deadlock_timeout
+ * if logging_timer is true. This is used to log when the startup process
+ * is waiting longer than deadlock_timeout for recovery conflict on lock.
*/
void
-ResolveRecoveryConflictWithLock(LOCKTAG locktag)
+ResolveRecoveryConflictWithLock(LOCKTAG locktag, bool logging_timer)
{
TimestampTz ltime;
@@ -408,7 +508,35 @@ ResolveRecoveryConflictWithLock(LOCKTAG locktag)
ltime = GetStandbyLimitTime();
- if (GetCurrentTimestamp() >= ltime)
+ if (ltime == 0 || GetCurrentTimestamp() < ltime)
+ {
+ EnableTimeoutParams timeouts[2];
+ int cnt = 0;
+
+ /*
+ * Wait (or wait again) until ltime
+ */
+ if (ltime != 0)
+ {
+ timeouts[cnt].id = STANDBY_LOCK_TIMEOUT;
+ timeouts[cnt].type = TMPARAM_AT;
+ timeouts[cnt].fin_time = ltime;
+ cnt++;
+ }
+
+ /* Also, set the timer if necessary */
+ if (logging_timer)
+ {
+ timeouts[cnt].id = STANDBY_LOCK_TIMEOUT;
+ timeouts[cnt].type = TMPARAM_AFTER;
+ timeouts[cnt].delay_ms = DeadlockTimeout;
+ cnt++;
+ }
+
+ if (cnt > 0)
+ enable_timeouts(timeouts, cnt);
+ }
+ else
{
/*
* We're already behind, so clear a path as quickly as possible.
@@ -419,26 +547,15 @@ ResolveRecoveryConflictWithLock(LOCKTAG locktag)
/*
* Prevent ResolveRecoveryConflictWithVirtualXIDs() from reporting
- * "waiting" in PS display by disabling its argument report_waiting
- * because the caller, WaitOnLock(), has already reported that.
+ * "waiting" in PS display and logging recovery conflict by disabling
+ * its argument report_waiting because the callers may have already
+ * reported that.
*/
ResolveRecoveryConflictWithVirtualXIDs(backends,
PROCSIG_RECOVERY_CONFLICT_LOCK,
PG_WAIT_LOCK | locktag.locktag_type,
false);
}
- else
- {
- /*
- * Wait (or wait again) until ltime
- */
- EnableTimeoutParams timeouts[1];
-
- timeouts[0].id = STANDBY_LOCK_TIMEOUT;
- timeouts[0].type = TMPARAM_AT;
- timeouts[0].fin_time = ltime;
- enable_timeouts(timeouts, 1);
- }
/* Wait to be signaled by the release of the Relation Lock */
ProcWaitForSignal(PG_WAIT_LOCK | locktag.locktag_type);
@@ -1116,3 +1233,36 @@ LogStandbyInvalidations(int nmsgs, SharedInvalidationMessage *msgs,
nmsgs * sizeof(SharedInvalidationMessage));
XLogInsert(RM_STANDBY_ID, XLOG_INVALIDATIONS);
}
+
+/* Return the description of recovery conflict */
+static const char *
+get_recovery_conflict_desc(ProcSignalReason reason)
+{
+ const char *reasonDesc = gettext_noop("unknown reason");
+
+ switch (reason)
+ {
+ case PROCSIG_RECOVERY_CONFLICT_BUFFERPIN:
+ reasonDesc = gettext_noop("recovery conflict on buffer pin");
+ break;
+ case PROCSIG_RECOVERY_CONFLICT_LOCK:
+ reasonDesc = gettext_noop("recovery conflict on lock");
+ break;
+ case PROCSIG_RECOVERY_CONFLICT_TABLESPACE:
+ reasonDesc = gettext_noop("recovery conflict on tablespace");
+ break;
+ case PROCSIG_RECOVERY_CONFLICT_SNAPSHOT:
+ reasonDesc = gettext_noop("recovery conflict on snapshot");
+ break;
+ case PROCSIG_RECOVERY_CONFLICT_STARTUP_DEADLOCK:
+ reasonDesc = gettext_noop("recovery conflict on buffer deadlock");
+ break;
+ case PROCSIG_RECOVERY_CONFLICT_DATABASE:
+ reasonDesc = gettext_noop("recovery conflict on database");
+ break;
+ default:
+ break;
+ }
+
+ return reasonDesc;
+}
diff --git a/src/backend/storage/lmgr/proc.c b/src/backend/storage/lmgr/proc.c
index 7dc3911590..527e38c974 100644
--- a/src/backend/storage/lmgr/proc.c
+++ b/src/backend/storage/lmgr/proc.c
@@ -1063,8 +1063,10 @@ ProcSleep(LOCALLOCK *locallock, LockMethod lockMethodTable)
LWLock *partitionLock = LockHashPartitionLock(hashcode);
PROC_QUEUE *waitQueue = &(lock->waitProcs);
LOCKMASK myHeldLocks = MyProc->heldLocks;
+ TimestampTz standbyWaitStart = 0;
bool early_deadlock = false;
bool allow_autovacuum_cancel = true;
+ bool logged_recovery_conflict = false;
ProcWaitStatus myWaitStatus;
PGPROC *proc;
PGPROC *leader = MyProc->lockGroupLeader;
@@ -1260,6 +1262,14 @@ ProcSleep(LOCALLOCK *locallock, LockMethod lockMethodTable)
else
enable_timeout_after(DEADLOCK_TIMEOUT, DeadlockTimeout);
}
+ else if (log_recovery_conflict_waits)
+ {
+ /*
+ * Set the wait start timestamp if logging is enabled and in hot
+ * standby.
+ */
+ standbyWaitStart = GetCurrentTimestamp();
+ }
/*
* If somebody wakes us between LWLockRelease and WaitLatch, the latch
@@ -1279,8 +1289,48 @@ ProcSleep(LOCALLOCK *locallock, LockMethod lockMethodTable)
{
if (InHotStandby)
{
- /* Set a timer and wait for that or for the Lock to be granted */
- ResolveRecoveryConflictWithLock(locallock->tag.lock);
+ bool maybe_log_conflict =
+ (standbyWaitStart != 0 && !logged_recovery_conflict);
+
+ /*
+ * Set a timer and wait for that or for the lock to be granted.
+ *
+ * If the recovery conflict has not been logged yet even though
+ * logging is enabled, set a timer so that we can log when we are
+ * waiting longer than deadlock_timeout for the conflict.
+ */
+ ResolveRecoveryConflictWithLock(locallock->tag.lock,
+ maybe_log_conflict);
+
+ /*
+ * Emit the log message if the startup process is waiting longer
+ * than deadlock_timeout for recovery conflict on lock.
+ */
+ if (maybe_log_conflict)
+ {
+ TimestampTz now = GetCurrentTimestamp();
+
+ if (TimestampDifferenceExceeds(standbyWaitStart, now,
+ DeadlockTimeout))
+ {
+ VirtualTransactionId *vxids;
+ int cnt;
+
+ vxids = GetLockConflicts(&locallock->tag.lock,
+ AccessExclusiveLock, &cnt);
+
+ /*
+ * Log the recovery conflict and the list of PIDs of
+ * backends holding the conflicting lock. Note that we do
+ * logging even if there are no such backends right now
+ * because the startup process here has already waited
+ * longer than deadlock_timeout.
+ */
+ LogRecoveryConflict(PROCSIG_RECOVERY_CONFLICT_LOCK,
+ standbyWaitStart, now, cnt > 0 ? vxids : NULL);
+ logged_recovery_conflict = true;
+ }
+ }
}
else
{
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index dabcbb0736..23ebd03fb4 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -1570,7 +1570,15 @@ static struct config_bool ConfigureNamesBool[] =
false,
NULL, NULL, NULL
},
-
+ {
+ {"log_recovery_conflict_waits", PGC_SIGHUP, LOGGING_WHAT,
+ gettext_noop("Logs standby recovery conflict waits."),
+ NULL
+ },
+ &log_recovery_conflict_waits,
+ false,
+ NULL, NULL, NULL
+ },
{
{"log_hostname", PGC_SIGHUP, LOGGING_WHAT,
gettext_noop("Logs the host name in the connection logs."),
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index b7fb2ec1fe..ebf1672bad 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -552,6 +552,8 @@
# %% = '%'
# e.g. '<%u%%%d> '
#log_lock_waits = off # log lock waits >= deadlock_timeout
+#log_recovery_conflict_waits = off # log standby recovery conflict waits
+ # >= deadlock_timeout
#log_parameter_max_length = -1 # when logging statements, limit logged
# bind-parameter values to N bytes;
# -1 means print in full, 0 disables
diff --git a/src/include/storage/standby.h b/src/include/storage/standby.h
index faaf1d3817..76ccd8cde4 100644
--- a/src/include/storage/standby.h
+++ b/src/include/storage/standby.h
@@ -23,6 +23,7 @@
extern int vacuum_defer_cleanup_age;
extern int max_standby_archive_delay;
extern int max_standby_streaming_delay;
+extern bool log_recovery_conflict_waits;
extern void InitRecoveryTransactionEnvironment(void);
extern void ShutdownRecoveryTransactionEnvironment(void);
@@ -32,12 +33,14 @@ extern void ResolveRecoveryConflictWithSnapshot(TransactionId latestRemovedXid,
extern void ResolveRecoveryConflictWithTablespace(Oid tsid);
extern void ResolveRecoveryConflictWithDatabase(Oid dbid);
-extern void ResolveRecoveryConflictWithLock(LOCKTAG locktag);
+extern void ResolveRecoveryConflictWithLock(LOCKTAG locktag, bool logging_timer);
extern void ResolveRecoveryConflictWithBufferPin(void);
extern void CheckRecoveryConflictDeadlock(void);
extern void StandbyDeadLockHandler(void);
extern void StandbyTimeoutHandler(void);
extern void StandbyLockTimeoutHandler(void);
+extern void LogRecoveryConflict(ProcSignalReason reason, TimestampTz wait_start,
+ TimestampTz cur_ts, VirtualTransactionId *wait_list);
/*
* Standby Rmgr (RM_STANDBY_ID)