Proposal: "Causal reads" mode for load balancing reads without stale data

Started by Thomas Munroabout 10 years ago76 messages
#1Thomas Munro
thomas.munro@enterprisedb.com
1 attachment(s)

Hi hackers,

Many sites use hot standby servers to spread read-heavy workloads over more
hardware, or at least would like to. This works well today if your
application can tolerate some time lag on standbys. The problem is that
there is no guarantee of when a particular commit will become visible for
clients connected to standbys. The existing synchronous commit feature is
no help here because it guarantees only that the WAL has been flushed on
another server before commit returns. It says nothing about whether it has
been applied or whether it has been applied on the standby that you happen
to be talking to.

A while ago I posted a small patch[1]/messages/by-id/CAEepm=1fqkivL4V-OTPHwSgw4aF9HcoGiMrCW-yBtjipX9gsag@mail.gmail.com to allow synchronous_commit to wait
for remote apply on the current synchronous standby, but (as Simon Riggs
rightly pointed out in that thread) that part isn't the main problem. It
seems to me that the main problem for a practical 'writer waits' system is
how to support a dynamic set of servers, gracefully tolerating failures and
timing out laggards, while also providing a strong guarantee during any
such transitions. Now I would like to propose something to do that, and
share a proof-of-concept patch.

=== PROPOSAL ===

The working name for the proposed feature is "causal reads", because it
provides a form of "causal consistency"[2]From http://queue.acm.org/detail.cfm?id=1466448 (and "read-your-writes"
consistency) no matter which server the client is connected to. There is a
similar feature by the same name in another product (albeit implemented
differently -- 'reader waits'; more about that later). I'm not wedded to
the name.

The feature allows arbitrary read-only transactions to be run on any hot
standby, with a specific guarantee about the visibility of preceding
transactions. The guarantee is that if you set a new GUC "causal_reads =
on" in any pair of consecutive transactions (tx1, tx2) where tx2 begins
after tx1 successfully returns, then tx2 will either see tx1 or fail with a
new error "standby is not available for causal reads", no matter which
server it runs on. A discovery mechanism is also provided, giving an
instantaneous snapshot of the set of standbys that are currently available
for causal reads (ie won't raise the error), in the form of a new column in
pg_stat_replication.

For example, a web server might run tx1 to insert a new row representing a
message in a discussion forum on the primary server, and then send the user
to another web page that runs tx2 to load all messages in the forum on an
arbitrary hot standby server. If causal_reads = on in both tx1 and tx2
(for example, because it's on globally), then tx2 is guaranteed to see the
new post, or get a (hopefully rare) error telling the client to retry on
another server.

Very briefly, the approach is:
1. The primary tracks apply lag on each standby (including between
commits).
2. The primary deems standbys 'available' for causal reads if they are
applying WAL and replying to keepalives fast enough, and periodically sends
the standby an authorization to consider itself available for causal reads
until a time in the near future.
3. Commit on the primary with "causal_reads = on" waits for all
'available' standbys either to apply the commit record, or to cease to be
'available' and begin raising the error if they are still alive (because
their authorizations have expired).
4. Standbys can start causal reads transactions only while they have an
authorization with an expiry time in the future; otherwise they raise an
error when an initial snapshot is taken.

In a follow-up email I can write about the design trade-offs considered
(mainly 'writer waits' vs 'reader waits'), comparison with some other
products, method of estimating replay lag, wait and timeout logic and how
it maintains the guarantee in various failure scenarios, logic for standbys
joining and leaving, implications of system clock skew between servers, or
any other questions you may have, depending on feedback/interest (but see
comments in the attached patch for some of those subjects). For now I
didn't want to clog up the intertubes with too large a wall of text.

=== PROOF-OF-CONCEPT ===

Please see the POC patch attached. It adds two new GUCs. After setting up
one or more hot standbys as per usual, simply add "causal_reads_timeout =
4s" to the primary's postgresql.conf and restart. Now, you can set
"causal_reads = on" in some/all sessions to get guaranteed causal
consistency. Expected behaviour: the causal reads guarantee is maintained
at all times, even when you overwhelm, kill, crash, disconnect, restart,
pause, add and remove standbys, and the primary drops them from the set it
waits for in a timely fashion. You can monitor the system with the
replay_lag and causal_reads_status in pg_stat_replication and some state
transition LOG messages on the primary. (The patch also supports
"synchronous_commit = apply", but it's not clear how useful that is in
practice, as already discussed.)

Lastly, a few notes about how this feature related to some other work:

The current version of this patch has causal_reads as a feature separate
from synchronous_commit, from a user's point of view. The thinking behind
this is that load balancing and data loss avoidance are separate concerns:
synchronous_commit deals with the latter, and causal_reads with the
former. That said, existing SyncRep machinery is obviously used
(specifically SyncRep queues, with a small modification, as a way to wait
for apply messages to arrive from standbys). (An earlier prototype had
causal reads as a new level for synchronous_commit and associated states as
new walsender states above 'streaming'. When contemplating how to combine
this proposal with the multiple-synchronous-standby patch, some colleagues
and I came around to the view that the concerns are separate. The reason
for wanting to configure complicated quorum definitions is to control data
loss risks and has nothing to do with load balancing requirements, so we
thought the features should probably be separate.)

The multiple-synchronous-servers patch[3]/messages/by-id/CAOG9ApHYCPmTypAAwfD3_V7sVOkbnECFivmRc1AxhB40ZBSwNQ@mail.gmail.com could be applied or not
independently of this feature as a result of that separation, as it doesn't
use synchronous_standby_names or indeed any kind of statically defined
quorum.

The standby WAL writer patch[4]/messages/by-id/CA+U5nMJifauXvVbx=v3UbYbHO3Jw2rdT4haL6CCooEDM5=4ASQ@mail.gmail.com would significantly improve walreceiver
performance and smoothness which would work very well with this proposal.

Please let me know what you think!

Thanks,

[1]: /messages/by-id/CAEepm=1fqkivL4V-OTPHwSgw4aF9HcoGiMrCW-yBtjipX9gsag@mail.gmail.com
/messages/by-id/CAEepm=1fqkivL4V-OTPHwSgw4aF9HcoGiMrCW-yBtjipX9gsag@mail.gmail.com

[2]: From http://queue.acm.org/detail.cfm?id=1466448

"Causal consistency. If process A has communicated to process B that it has
updated a data item, a subsequent access by process B will return the
updated value, and a write is guaranteed to supersede the earlier write.
Access by process C that has no causal relationship to process A is subject
to the normal eventual consistency rules.

Read-your-writes consistency. This is an important model where process A,
after it has updated a data item, always accesses the updated value and
will never see an older value. This is a special case of the causal
consistency model."

[3]: /messages/by-id/CAOG9ApHYCPmTypAAwfD3_V7sVOkbnECFivmRc1AxhB40ZBSwNQ@mail.gmail.com
/messages/by-id/CAOG9ApHYCPmTypAAwfD3_V7sVOkbnECFivmRc1AxhB40ZBSwNQ@mail.gmail.com

[4]: /messages/by-id/CA+U5nMJifauXvVbx=v3UbYbHO3Jw2rdT4haL6CCooEDM5=4ASQ@mail.gmail.com
/messages/by-id/CA+U5nMJifauXvVbx=v3UbYbHO3Jw2rdT4haL6CCooEDM5=4ASQ@mail.gmail.com

--
Thomas Munro
http://www.enterprisedb.com

Attachments:

causal-reads-poc.patchapplication/octet-stream; name=causal-reads-poc.patchDownload
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 47312f6..535b5a9 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,10 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
+	{
+		CausalReadsWaitForLSN(XactLastRecEnd);
 		SyncRepWaitForLSN(XactLastRecEnd);
+	}
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5117,6 +5120,13 @@ XactLogCommitRecord(TimestampTz commit_time,
 		xl_xinfo.xinfo |= XACT_COMPLETION_FORCE_SYNC_COMMIT;
 
 	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this commit is applied.
+	 */
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY || causal_reads)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
+	/*
 	 * Relcache invalidations requires information about the current database
 	 * and so does logical decoding.
 	 */
@@ -5452,6 +5462,19 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 	if (XactCompletionForceSyncCommit(parsed->xinfo))
 		XLogFlush(lsn);
 
+	/*
+	 * Record the primary's timestamp for the commit record, so it can be used
+	 * for tracking replay lag.
+	 */
+	SetXLogReplayTimestamp(parsed->xact_time);
+
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit or causal reads), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 /*
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 08d1682..7779c34 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -80,6 +80,8 @@ extern uint32 bootstrap_data_checksum_version;
 #define PROMOTE_SIGNAL_FILE		"promote"
 #define FALLBACK_PROMOTE_SIGNAL_FILE "fallback_promote"
 
+/* Size of the circular buffer of timestamped LSNs. */
+#define MAX_TIMESTAMPED_LSNS 256
 
 /* User-settable parameters */
 int			max_wal_size = 64;	/* 1 GB */
@@ -346,6 +348,12 @@ static XLogRecPtr RedoRecPtr;
 static bool doPageWrites;
 
 /*
+ * doRequestWalReceiverReply is used by recovery code to ask the main recovery
+ * loop to trigger a walreceiver reply.
+ */
+static bool doRequestWalReceiverReply;
+
+/*
  * RedoStartLSN points to the checkpoint's REDO location which is specified
  * in a backup label file, backup history file or control file. In standby
  * mode, XLOG streaming usually starts from the position where an invalid
@@ -357,6 +365,13 @@ static bool doPageWrites;
  */
 static XLogRecPtr RedoStartLSN = InvalidXLogRecPtr;
 
+/*
+ * LastReplayedTimestamp can be set by redo handlers when they apply a record
+ * that carries a timestamp, by calling SetXLogReplayedTimestamp.  The xlog
+ * apply loop can then update the value in shared memory.
+ */
+static TimestampTz LastReplayedTimestamp = 0;
+
 /*----------
  * Shared-memory data structures for XLOG control
  *
@@ -632,6 +647,21 @@ typedef struct XLogCtlData
 	/* current effective recovery target timeline */
 	TimeLineID	RecoveryTargetTLI;
 
+	/* timestamp from the most recently applied record that carried a timestamp. */
+	TimestampTz lastReplayedTimestamp;
+
+	/*
+	 * We maintain a circular buffer of LSNs and associated timestamps.
+	 * Walreceiver writes into it using information from timestamps, and the
+	 * startup recovery process reads from it and notifies walreceiver when
+	 * LSNs are replayed so that the timestamps can be fed back to the
+	 * upstream server, to track lag.
+	 */
+	Index		timestampedLsnRead;
+	Index		timestampedLsnWrite;
+	XLogRecPtr	timestampedLsn[MAX_TIMESTAMPED_LSNS];
+	TimestampTz	timestampedLsnTime[MAX_TIMESTAMPED_LSNS];
+
 	/*
 	 * timestamp of when we started replaying the current chunk of WAL data,
 	 * only relevant for replication or archive recovery
@@ -6844,14 +6874,57 @@ StartupXLOG(void)
 				error_context_stack = errcallback.previous;
 
 				/*
-				 * Update lastReplayedEndRecPtr after this record has been
-				 * successfully replayed.
+				 * Update lastReplayedEndRecPtr and lastReplayedTimestamp
+				 * after this record has been successfully replayed.
 				 */
 				SpinLockAcquire(&XLogCtl->info_lck);
 				XLogCtl->lastReplayedEndRecPtr = EndRecPtr;
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
+				if (LastReplayedTimestamp != 0)
+				{
+					/* If replaying a record produced a timestamp, use that. */
+					XLogCtl->lastReplayedTimestamp = LastReplayedTimestamp;
+					LastReplayedTimestamp = 0;
+				}
+				else
+				{
+					/*
+					 * If we have applied LSNs associated with timestamps
+					 * received by walreceiver, then use the recorded
+					 * timestamp.  We consume from the read end of the
+					 * circular buffer.
+					 */
+					while (XLogCtl->timestampedLsnRead !=
+						   XLogCtl->timestampedLsnWrite &&
+						   XLogCtl->timestampedLsn[XLogCtl->timestampedLsnRead]
+						   <= EndRecPtr)
+					{
+						if (XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead] >
+							XLogCtl->lastReplayedTimestamp)
+						{
+							XLogCtl->lastReplayedTimestamp =
+								XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead];
+							doRequestWalReceiverReply = true;
+						}
+						++XLogCtl->timestampedLsnRead;
+					}
+				}
 				SpinLockRelease(&XLogCtl->info_lck);
 
+				/*
+				 * If rm_redo reported that it applied a commit record that
+				 * the master is waiting for by calling
+				 * XLogRequestWalReceiverReply, or we encountered a WAL
+				 * location that was associated with a timestamp above, then
+				 * we wake up the receiver so that it notices the updated
+				 * lastReplayedEndRecPtr and sends a reply to the master.
+				 */
+				if (doRequestWalReceiverReply)
+				{
+					doRequestWalReceiverReply = false;
+					WalRcvWakeup();
+				}
+
 				/* Remember this record as the last-applied one */
 				LastRec = ReadRecPtr;
 
@@ -11580,3 +11653,103 @@ SetWalWriterSleeping(bool sleeping)
 	XLogCtl->WalWriterSleeping = sleeping;
 	SpinLockRelease(&XLogCtl->info_lck);
 }
+
+/*
+ * Called by redo code to indicate that the xlog replay loop should wake up
+ * the walreceiver process so that a reply can be sent to the primary.
+ */
+void
+XLogRequestWalReceiverReply(void)
+{
+	doRequestWalReceiverReply = true;
+}
+
+/*
+ * Merge timestamps from keepalive messages with the timestamps from WAL
+ * records, so that we can track lag while idle or while replaying large
+ * amounts of WAL without commit records.  In the former case there is no lag,
+ * and in the latter case we will remember a timestamp that goes with an
+ * arbitrary LSN, and wait for that LSN to be replayed before using the
+ * timestamp.
+ *
+ * This is called by walreceiver on standby servers when keepalive messages
+ * arrive.
+ */
+void
+SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn)
+{
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn == XLogCtl->lastReplayedEndRecPtr)
+	{
+		/*
+		 * That is the last replayed LSN: we are fully replayed, so we can
+		 * update the replay timestamp immediately.
+		 */
+		XLogCtl->lastReplayedTimestamp = timestamp;
+	}
+	else
+	{
+		/*
+		 * There is WAL still to be applied.  We will associate the timestamp
+		 * with this WAL position and wait for it to be replayed.  We add it
+		 * at the 'write' end of the circular buffer of LSN/timestamp
+		 * mappings, which the replay loop will eventually read.
+		 */
+		Index w = XLogCtl->timestampedLsnWrite;
+		Index r = XLogCtl->timestampedLsnRead;
+
+		XLogCtl->timestampedLsn[w] = lsn;
+		XLogCtl->timestampedLsnTime[w] = timestamp;
+
+		/* Advance the write point. */
+		w = (w + 1) % MAX_TIMESTAMPED_LSNS;
+		XLogCtl->timestampedLsnWrite = w;
+		if (w == r)
+		{
+			/*
+			 * The buffer is full.  Advance the read point (throwing away
+			 * oldest values; we will begin to oversestimate replay lag, until
+			 * lag decreases to a size our buffer can manage, or the next
+			 * commit record is replayed).
+			 */
+			r = (r + 1) % MAX_TIMESTAMPED_LSNS;
+			XLogCtl->timestampedLsnRead = r;
+		}
+	}
+	SpinLockRelease(&XLogCtl->info_lck);
+}
+
+/*
+ * Set the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary.  This can be called by redo handlers that have
+ * an appropriate timestamp (currently only commit records).  Updating the
+ * shared memory value is deferred until after the redo handler returns.
+ */
+void
+SetXLogReplayTimestamp(TimestampTz timestamp)
+{
+	LastReplayedTimestamp = timestamp;
+}
+
+/*
+ * Get the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the master, and also the most recently applied LSN.  (Note
+ * that the timestamp and the LSN don't necessarily relate to the same
+ * record.)
+ *
+ * This is similar to GetLatestXTime, except that it is not only advanced by
+ * commit records (see SetXLogReplayTimestampAtLsn).
+ */
+TimestampTz
+GetXLogReplayTimestamp(XLogRecPtr *lsn)
+{
+	TimestampTz result;
+
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn)
+		*lsn = XLogCtl->lastReplayedEndRecPtr;
+	result = XLogCtl->lastReplayedTimestamp;
+	SpinLockRelease(&XLogCtl->info_lck);
+
+	return result;
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index ccc030f..f9b0e53 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -647,8 +647,10 @@ CREATE VIEW pg_stat_replication AS
             W.write_location,
             W.flush_location,
             W.replay_location,
+            W.replay_lag,
             W.sync_priority,
-            W.sync_state
+            W.sync_state,
+            W.causal_reads_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
             pg_stat_get_wal_senders() AS W
     WHERE S.usesysid = U.oid AND
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 325239d..2abf299 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -57,6 +57,10 @@
 #include "utils/builtins.h"
 #include "utils/ps_status.h"
 
+/* GUC variables */
+int causal_reads_timeout;
+bool causal_reads;
+
 /* User-settable parameters for sync rep */
 char	   *SyncRepStandbyNames;
 
@@ -69,7 +73,7 @@ static int	SyncRepWaitMode = SYNC_REP_NO_WAIT;
 
 static void SyncRepQueueInsert(int mode);
 static void SyncRepCancelWait(void);
-static int	SyncRepWakeQueue(bool all, int mode);
+static int	SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn);
 
 static int	SyncRepGetStandbyPriority(void);
 
@@ -83,6 +87,239 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * ===========================================================
  */
 
+static bool
+SyncRepCheckEarlyExit(void)
+{
+	/*
+	 * If a wait for synchronous replication is pending, we can neither
+	 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
+	 * lead the client to believe that that the transaction aborted, which
+	 * is not true: it's already committed locally. The former is no good
+	 * either: the client has requested synchronous replication, and is
+	 * entitled to assume that an acknowledged commit is also replicated,
+	 * which might not be true. So in this case we issue a WARNING (which
+	 * some clients may be able to interpret) and shut off further output.
+	 * We do NOT reset ProcDiePending, so that the process will die after
+	 * the commit is cleaned up.
+	 */
+	if (ProcDiePending)
+	{
+		ereport(WARNING,
+				(errcode(ERRCODE_ADMIN_SHUTDOWN),
+				 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * It's unclear what to do if a query cancel interrupt arrives.  We
+	 * can't actually abort at this point, but ignoring the interrupt
+	 * altogether is not helpful, so we just terminate the wait with a
+	 * suitable warning.
+	 */
+	if (QueryCancelPending)
+	{
+		QueryCancelPending = false;
+		ereport(WARNING,
+				(errmsg("canceling wait for synchronous replication due to user request"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * If the postmaster dies, we'll probably never get an
+	 * acknowledgement, because all the wal sender processes will exit. So
+	 * just bail out.
+	 */
+	if (!PostmasterIsAlive())
+	{
+		ProcDiePending = true;
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	return false;
+}
+
+/*
+ * Check if we can stop waiting for causal consistency.  We can stop waiting
+ * when the following conditions are met:
+ *
+ * 1.  All walsenders currently in 'joining' or 'available' state have
+ * applied the target LSN.
+ *
+ * 2.  Any stall periods caused by standbys dropping out of 'available' state
+ * have passed, so that we can be sure that any causalReadsUntil authorization
+ * has expired.
+ *
+ * The output parameter 'waitingFor' is set to the number of nodes we are
+ * currently waiting for.  The output parameters 'stallTimeMillis' is set to
+ * the number of milliseconds we need to wait for to observe any current
+ * commit stall.
+ *
+ * Returns true if commit can return control, because every standby has either
+ * applied the LSN or started rejecting causal_reads transactions.
+ */
+static bool
+CausalReadsCommitCanReturn(XLogRecPtr XactCommitLSN,
+						   int *waitingFor,
+						   long *stallTimeMillis)
+{
+	int i;
+	TimestampTz now;
+
+	/* Count how many joining/available nodes we are waiting for. */
+	*waitingFor = 0;
+	for (i = 0; i < max_wal_senders; ++i)
+	{
+		WalSnd *walsnd = &WalSndCtl->walsnds[i];
+
+		/*
+		 * Assuming atomic read of pid_t, we can check walsnd->pid without
+		 * acquiring the spinlock to avoid memory synchronization costs for
+		 * unused walsender slots.  We see a value that existed sometime at
+		 * least as recently as the last memory barrier.
+		 */
+		if (walsnd->pid != 0)
+		{
+			/*
+			 * We need to hold the spinlock to read LSNs, because we can't be
+			 * sure they can be read atomically.
+			 */
+			SpinLockAcquire(&walsnd->mutex);
+			if (walsnd->pid != 0 && walsnd->causal_reads_state >= WALSNDCRSTATE_JOINING)
+			{
+				if (walsnd->apply < XactCommitLSN)
+					++*waitingFor;
+			}
+			SpinLockRelease(&walsnd->mutex);
+		}
+	}
+
+	/* Check if there is a stall in progress that we need to observe. */
+	now = GetCurrentTimestamp();
+	LWLockAcquire(SyncRepLock, LW_SHARED);
+	if (WalSndCtl->stall_causal_reads_until > now)
+	{
+		long seconds;
+		int usecs;
+
+		/* Compute how long we have to wait, rounded up to nearest ms. */
+		TimestampDifference(now, WalSndCtl->stall_causal_reads_until,
+							&seconds, &usecs);
+		*stallTimeMillis = seconds * 1000 + (usecs + 999) / 1000;
+	}
+	else
+		*stallTimeMillis = 0;
+	LWLockRelease(SyncRepLock);
+
+	/* We are done if we are not waiting for any nodes or stalls. */
+	return *waitingFor == 0 && *stallTimeMillis == 0;
+}
+
+/*
+ * Wait for causal consistency in causal_reads mode, if requested by user.
+ */
+void
+CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN)
+{
+	long stallTimeMillis;
+	int waitingFor;
+
+	/* Leave if we aren't in causal_reads mode. */
+	if (!causal_reads)
+		return;
+
+	for (;;)
+	{
+		/* Reset latch before checking state. */
+		ResetLatch(MyLatch);
+
+		/*
+		 * Join the queue to be woken up if any causal reads joining/available
+		 * standby applies XactCommitLSN, if we aren't already in it.  We
+		 * don't actually know if we need to wait for any peers yet, but we
+		 * have to register just in case before checking the walsenders' state
+		 * to avoid a race condition that could occur if we did it after
+		 * calling CausalReadsCommitCanReturn.  (SyncRepWaitForLSN doesn't
+		 * have to do this because it can check the highest-seen LSN in
+		 * walsndctl->lsn[mode] which is protected by SyncRepLock, the same
+		 * lock as the queues.  We can't do that here, because there is no
+		 * single highest-seen LSN that is useful.  We must check
+		 * walsnd->apply for all relevant walsenders.  Therefore we must
+		 * register for notifications first, so that we can be notified via
+		 * our latch of any standby applying the LSN we're interested in after
+		 * we check but before we start waiting, or we could wait forever for
+		 * something that has already happened.)
+		 */
+		LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+		if (MyProc->syncRepState != SYNC_REP_WAITING)
+		{
+			MyProc->waitLSN = XactCommitLSN;
+			MyProc->syncRepState = SYNC_REP_WAITING;
+			SyncRepQueueInsert(SYNC_REP_WAIT_CAUSAL_READS_APPLY);
+			Assert(SyncRepQueueIsOrderedByLSN(SYNC_REP_WAIT_CAUSAL_READS_APPLY));
+		}
+		LWLockRelease(SyncRepLock);
+
+		/* Check if we're done. */
+		if (CausalReadsCommitCanReturn(XactCommitLSN, &waitingFor, &stallTimeMillis))
+		{
+			SyncRepCancelWait();
+			break;
+		}
+
+		Assert(waitingFor > 0 || stallTimeMillis > 0);
+
+		/* If we aren't actually waiting for any standbys, leave the queue. */
+		if (waitingFor == 0)
+			SyncRepCancelWait();
+
+		/* Update the ps title. */
+		if (update_process_title)
+		{
+			char buffer[80];
+
+			snprintf(buffer, sizeof(buffer),
+					 "waiting for %d peer(s) to apply %X/%X%s",
+					 waitingFor,
+					 (uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN,
+					 stallTimeMillis > 0 ? " (stalling)" : "");
+			set_ps_display(buffer, false);
+		}
+
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit()) /* Calls SyncRepCancelWait() if true. */
+			break;
+
+		/*
+		 * If are still waiting for peers, then we wait for any joining or
+		 * available peer to reach the LSN (or possibly stop being in one of
+		 * those states or go away).
+		 *
+		 * If not, there must be a non-zero stall time, so we wait for that to
+		 * elapse.
+		 */
+		if (waitingFor > 0)
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH, -1);
+		else
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH | WL_TIMEOUT,
+					  stallTimeMillis);
+	}
+
+	/* There is no way out of the loop that could leave us in the queue. */
+	Assert(SHMQueueIsDetached(&(MyProc->syncRepLinks)));
+	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
+	MyProc->waitLSN = 0;
+
+	if (update_process_title)
+		set_ps_display("", false); /* TODO: restore what was there */
+}
+
 /*
  * Wait for synchronous replication, if requested by user.
  *
@@ -180,57 +417,9 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 		if (syncRepState == SYNC_REP_WAIT_COMPLETE)
 			break;
 
-		/*
-		 * If a wait for synchronous replication is pending, we can neither
-		 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
-		 * lead the client to believe that that the transaction aborted, which
-		 * is not true: it's already committed locally. The former is no good
-		 * either: the client has requested synchronous replication, and is
-		 * entitled to assume that an acknowledged commit is also replicated,
-		 * which might not be true. So in this case we issue a WARNING (which
-		 * some clients may be able to interpret) and shut off further output.
-		 * We do NOT reset ProcDiePending, so that the process will die after
-		 * the commit is cleaned up.
-		 */
-		if (ProcDiePending)
-		{
-			ereport(WARNING,
-					(errcode(ERRCODE_ADMIN_SHUTDOWN),
-					 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * It's unclear what to do if a query cancel interrupt arrives.  We
-		 * can't actually abort at this point, but ignoring the interrupt
-		 * altogether is not helpful, so we just terminate the wait with a
-		 * suitable warning.
-		 */
-		if (QueryCancelPending)
-		{
-			QueryCancelPending = false;
-			ereport(WARNING,
-					(errmsg("canceling wait for synchronous replication due to user request"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			SyncRepCancelWait();
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit())
 			break;
-		}
-
-		/*
-		 * If the postmaster dies, we'll probably never get an
-		 * acknowledgement, because all the wal sender processes will exit. So
-		 * just bail out.
-		 */
-		if (!PostmasterIsAlive())
-		{
-			ProcDiePending = true;
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
-			break;
-		}
 
 		/*
 		 * Wait on latch.  Any condition that should wake us up will set the
@@ -410,22 +599,27 @@ SyncRepGetSynchronousStandby(void)
  * perhaps also which information we store as well.
  */
 void
-SyncRepReleaseWaiters(void)
+SyncRepReleaseWaiters(bool walsender_cr_available_or_joining)
 {
 	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
+	int			numapply = 0;
+	int			numcausalreadsapply = 0;
+	bool		is_highest_priority_sync_standby;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
-	 * potential standbys then we have nothing to do. If we are still starting
-	 * up, still running base backup or the current flush position is still
-	 * invalid, then leave quickly also.
+	 * potential standbys and not in a state that causal_reads waits for, then
+	 * we have nothing to do. If we are still starting up, still running base
+	 * backup or the current flush position is still invalid, then leave
+	 * quickly also.
 	 */
-	if (MyWalSnd->sync_standby_priority == 0 ||
-		MyWalSnd->state < WALSNDSTATE_STREAMING ||
-		XLogRecPtrIsInvalid(MyWalSnd->flush))
+	if (!walsender_cr_available_or_joining &&
+		(MyWalSnd->sync_standby_priority == 0 ||
+		 MyWalSnd->state < WALSNDSTATE_STREAMING ||
+		 XLogRecPtrIsInvalid(MyWalSnd->flush)))
 		return;
 
 	/*
@@ -435,45 +629,77 @@ SyncRepReleaseWaiters(void)
 	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
 	syncWalSnd = SyncRepGetSynchronousStandby();
 
-	/* We should have found ourselves at least */
-	Assert(syncWalSnd != NULL);
+	/*
+	 * If we aren't managing the highest priority standby then make a note of
+	 * that so we can announce a takeover in the log if we ever get that job.
+	 */
+	is_highest_priority_sync_standby = syncWalSnd == MyWalSnd;
+	if (!is_highest_priority_sync_standby)
+		announce_next_takeover = true;
 
 	/*
-	 * If we aren't managing the highest priority standby then just leave.
+	 * If we aren't managing the highest priority standby or a standby in
+	 * causal reads 'joining' or 'available' state, then just leave.
 	 */
-	if (syncWalSnd != MyWalSnd)
+	if (!is_highest_priority_sync_standby && !walsender_cr_available_or_joining)
 	{
 		LWLockRelease(SyncRepLock);
-		announce_next_takeover = true;
 		return;
 	}
 
 	/*
 	 * Set the lsn first so that when we wake backends they will release up to
-	 * this location.
+	 * this location.  For the single-standby synchronous commit levels, we
+	 * only do this if we are the current synchronous standby and we are
+	 * advancing the LSN further than it has been advanced before, so that
+	 * SyncRepWaitForLSN can skip waiting in some cases.
 	 */
-	if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
-		numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
+	if (is_highest_priority_sync_standby)
 	{
-		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
-		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
+		if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
+			numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE,
+										MyWalSnd->write);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->write)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
+			numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH,
+										MyWalSnd->flush);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+			numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY,
+										MyWalSnd->apply);
+		}
 	}
+	/*
+	 * For causal_reads, all walsenders currently in available or joining
+	 * state must reach the LSN on their own, and standbys will reach LSNs in
+	 * any order.  It doesn't make sense to keep the highest seen LSN in a
+	 * single walsndctl->lsn element.  (CausalReadsWaitForLSN has handling for
+	 * LSNs that have already been reached).
+	 */
+	if (walsender_cr_available_or_joining)
+		numcausalreadsapply =
+			SyncRepWakeQueue(false, SYNC_REP_WAIT_CAUSAL_READS_APPLY,
+							 MyWalSnd->apply);
 
 	LWLockRelease(SyncRepLock);
 
-	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X",
+	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%X, %d procs to causal_reads apply",
 		 numwrite, (uint32) (MyWalSnd->write >> 32), (uint32) MyWalSnd->write,
-	   numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush);
+		 numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush,
+		 numapply, (uint32) (MyWalSnd->apply >> 32), (uint32) MyWalSnd->apply,
+		 numcausalreadsapply);
 
 	/*
 	 * If we are managing the highest priority standby, though we weren't
 	 * prior to this, then announce we are now the sync standby.
 	 */
-	if (announce_next_takeover)
+	if (is_highest_priority_sync_standby && announce_next_takeover)
 	{
 		announce_next_takeover = false;
 		ereport(LOG,
@@ -548,9 +774,8 @@ SyncRepGetStandbyPriority(void)
  * Must hold SyncRepLock.
  */
 static int
-SyncRepWakeQueue(bool all, int mode)
+SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn)
 {
-	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	PGPROC	   *proc = NULL;
 	PGPROC	   *thisproc = NULL;
 	int			numprocs = 0;
@@ -567,7 +792,7 @@ SyncRepWakeQueue(bool all, int mode)
 		/*
 		 * Assume the queue is ordered by LSN
 		 */
-		if (!all && walsndctl->lsn[mode] < proc->waitLSN)
+		if (!all && lsn < proc->waitLSN)
 			return numprocs;
 
 		/*
@@ -627,7 +852,7 @@ SyncRepUpdateSyncStandbysDefined(void)
 			int			i;
 
 			for (i = 0; i < NUM_SYNC_REP_WAIT_MODE; i++)
-				SyncRepWakeQueue(true, i);
+				SyncRepWakeQueue(true, i, InvalidXLogRecPtr);
 		}
 
 		/*
@@ -679,6 +904,28 @@ SyncRepQueueIsOrderedByLSN(int mode)
 #endif
 
 /*
+ * Make sure that CausalReadsWaitForLSN can't return until after
+ * any 'causalReadsUntil' time that walsender could possibly have sent to any
+ * standby in a keepalive message.  This maintains the causal consistency
+ * guarantee: in causal_reads mode, we will not return control until any
+ * standby we have lost contact with has started generating 'standby not
+ * available for causal reads' errors.
+ *
+ * Also, wake up all backends waiting in CausalReadsWaitForLSN, because the
+ * set of available/joining peers has changed, and there is a new stall time
+ * they need to observe.
+ */
+void
+CausalReadsBeginStall(void)
+{
+	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+	WalSndCtl->stall_causal_reads_until =
+		TimestampTzPlusMilliseconds(GetCurrentTimestamp(), causal_reads_timeout);
+	SyncRepWakeQueue(true, SYNC_REP_WAIT_CAUSAL_READS_APPLY, InvalidXLogRecPtr);
+	LWLockRelease(SyncRepLock);
+}
+
+/*
  * ===========================================================
  * Synchronous Replication functions executed by any process
  * ===========================================================
@@ -728,6 +975,9 @@ assign_synchronous_commit(int newval, void *extra)
 		case SYNCHRONOUS_COMMIT_REMOTE_FLUSH:
 			SyncRepWaitMode = SYNC_REP_WAIT_FLUSH;
 			break;
+		case SYNCHRONOUS_COMMIT_REMOTE_APPLY:
+			SyncRepWaitMode = SYNC_REP_WAIT_APPLY;
+			break;
 		default:
 			SyncRepWaitMode = SYNC_REP_NO_WAIT;
 			break;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 183a3a5..94dd334 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -52,6 +52,7 @@
 #include "libpq/pqformat.h"
 #include "libpq/pqsignal.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "replication/walsender.h"
 #include "storage/ipc.h"
@@ -96,6 +97,7 @@ static uint32 recvOff = 0;
  */
 static volatile sig_atomic_t got_SIGHUP = false;
 static volatile sig_atomic_t got_SIGTERM = false;
+static volatile sig_atomic_t got_SIGUSR1 = false;
 
 /*
  * LogstreamResult indicates the byte positions that we have already
@@ -140,7 +142,8 @@ static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
 static void XLogWalRcvSendReply(bool force, bool requestReply);
 static void XLogWalRcvSendHSFeedback(bool immed);
-static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
+static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+								  TimestampTz *causalReadsUntil);
 
 /* Signal handlers */
 static void WalRcvSigHupHandler(SIGNAL_ARGS);
@@ -195,6 +198,7 @@ WalReceiverMain(void)
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz last_recv_timestamp;
 	bool		ping_sent;
+	bool		forceReply;
 
 	/*
 	 * WalRcv should be set up already (if we are a backend, we inherit this
@@ -246,6 +250,7 @@ WalReceiverMain(void)
 
 	/* Initialise to a sanish value */
 	walrcv->lastMsgSendTime = walrcv->lastMsgReceiptTime = walrcv->latestWalEndTime = GetCurrentTimestamp();
+	walrcv->causalReadsUntil = 0;
 
 	SpinLockRelease(&walrcv->mutex);
 
@@ -410,7 +415,7 @@ WalReceiverMain(void)
 					 * Process the received data, and any subsequent data we
 					 * can read without blocking.
 					 */
-					for (;;)
+					while (!got_SIGUSR1)
 					{
 						if (len > 0)
 						{
@@ -437,8 +442,16 @@ WalReceiverMain(void)
 						len = walrcv_receive(0, &buf);
 					}
 
+					if (got_SIGUSR1)
+					{
+						/* The recovery process asked us to force a reply. */
+						got_SIGUSR1 = false;
+						forceReply = true;
+					}
+
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(false, false);
+					XLogWalRcvSendReply(forceReply, false);
+					forceReply = false;
 
 					/*
 					 * If we've written some records, flush them to disk and
@@ -493,7 +506,15 @@ WalReceiverMain(void)
 						}
 					}
 
-					XLogWalRcvSendReply(requestReply, requestReply);
+					/* Check if the startup process has signaled us. */
+					if (got_SIGUSR1)
+					{
+						got_SIGUSR1 = false;
+						forceReply = true;
+					}
+
+					XLogWalRcvSendReply(requestReply || forceReply, requestReply);
+					forceReply = false;
 					XLogWalRcvSendHSFeedback(false);
 				}
 			}
@@ -730,6 +751,7 @@ WalRcvSigUsr1Handler(SIGNAL_ARGS)
 {
 	int			save_errno = errno;
 
+	got_SIGUSR1 = true;
 	latch_sigusr1_handler();
 
 	errno = save_errno;
@@ -795,6 +817,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 	XLogRecPtr	walEnd;
 	TimestampTz sendTime;
 	bool		replyRequested;
+	TimestampTz causalReadsUntil;
 
 	resetStringInfo(&incoming_message);
 
@@ -815,7 +838,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				walEnd = pq_getmsgint64(&incoming_message);
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, NULL);
 
 				buf += hdrlen;
 				len -= hdrlen;
@@ -825,7 +848,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 		case 'k':				/* Keepalive */
 			{
 				/* copy message to StringInfo */
-				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
+				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char) + sizeof(int64);
 				if (len != hdrlen)
 					ereport(ERROR,
 							(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -837,8 +860,12 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
 				replyRequested = pq_getmsgbyte(&incoming_message);
+				causalReadsUntil = IntegerTimestampToTimestampTz(
+					pq_getmsgint64(&incoming_message));
+				ProcessWalSndrMessage(walEnd, sendTime, &causalReadsUntil);
 
-				ProcessWalSndrMessage(walEnd, sendTime);
+				/* Remember primary's timestamp at this WAL location. */
+				SetXLogReplayTimestampAtLsn(sendTime, walEnd);
 
 				/* If the primary requested a reply, send one immediately */
 				if (replyRequested)
@@ -1032,6 +1059,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	XLogRecPtr	applyPtr;
 	static TimestampTz sendTime = 0;
 	TimestampTz now;
+	TimestampTz applyTimestamp = 0;
 
 	/*
 	 * If the user doesn't want status to be reported to the master, be sure
@@ -1063,7 +1091,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	/* Construct a new message */
 	writePtr = LogstreamResult.Write;
 	flushPtr = LogstreamResult.Flush;
-	applyPtr = GetXLogReplayRecPtr(NULL);
+	applyTimestamp = GetXLogReplayTimestamp(&applyPtr);
 
 	resetStringInfo(&reply_message);
 	pq_sendbyte(&reply_message, 'r');
@@ -1071,6 +1099,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	pq_sendint64(&reply_message, flushPtr);
 	pq_sendint64(&reply_message, applyPtr);
 	pq_sendint64(&reply_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&reply_message, TimestampTzToIntegerTimestamp(applyTimestamp));
 	pq_sendbyte(&reply_message, requestReply ? 1 : 0);
 
 	/* Send it */
@@ -1169,15 +1198,52 @@ XLogWalRcvSendHSFeedback(bool immed)
  * Update shared memory status upon receiving a message from primary.
  *
  * 'walEnd' and 'sendTime' are the end-of-WAL and timestamp of the latest
- * message, reported by primary.
+ * message, reported by primary.  'causalReadsUntil' is a pointer to
+ * the time the primary promises that this standby can safely claim to be
+ * causally consistent, to 0 if it cannot, or a NULL pointer for no change.
  */
 static void
-ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
+ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+					  TimestampTz *causalReadsUntil)
 {
 	WalRcvData *walrcv = WalRcv;
 
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
 
+	/* Sanity check for the causalReadsUntil time. */
+	if (causalReadsUntil != NULL && *causalReadsUntil != 0)
+	{
+		/* Deduce max_clock_skew from the causalReadsUntil and sendTime. */
+#ifdef HAVE_INT64_TIMESTAMP
+		int64 diffMillis = (*causalReadsUntil - sendTime) / 1000;
+#else
+		int64 diffMillis = (*causalReadsUntil - sendTime) * 1000;
+#endif
+		int64 max_clock_skew = diffMillis / (CAUSAL_READS_CLOCK_SKEW_RATIO - 1);
+
+		if (sendTime > TimestampTzPlusMilliseconds(lastMsgReceiptTime, max_clock_skew))
+		{
+			/*
+			 * The primary's clock is more than max_clock_skew + network
+			 * latency ahead of the standby's clock.  (If the primary's clock
+			 * is more than max_clock_skew ahead of the standby's clock, but
+			 * by less than the network latency, then there isn't much we can
+			 * do to detect that; but it still seems useful to have this basic
+			 * sanity check for wildly misconfigured servers.)
+			 */
+			elog(LOG, "the primary server's clock time is too far ahead");
+			causalReadsUntil = NULL;
+		}
+		/*
+		 * We could also try to detect cases where sendTime is more than
+		 * max_clock_skew in the past according to the standby's clock, but
+		 * that is indistinguishable from network latency/buffering, so we
+		 * could produce misleading error messages; if we do nothing, the
+		 * consequence is 'standby is not available for causal reads' errors
+		 * which should cause the user to investigate.
+		 */
+	}
+
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
 	if (walrcv->latestWalEnd < walEnd)
@@ -1185,6 +1251,8 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->latestWalEnd = walEnd;
 	walrcv->lastMsgSendTime = sendTime;
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
+	if (causalReadsUntil != NULL)
+		walrcv->causalReadsUntil = *causalReadsUntil;
 	SpinLockRelease(&walrcv->mutex);
 
 	if (log_min_messages <= DEBUG2)
@@ -1215,3 +1283,23 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 		pfree(receipttime);
 	}
 }
+
+/*
+ * Wake up the walreceiver if it happens to be blocked in walrcv_receive,
+ * and tell it that a commit record has been applied.
+ *
+ * This is called by the startup process whenever interesting xlog records
+ * are applied, so that walreceiver can check if it needs to send an apply
+ * notification back to the master which may be waiting in a COMMIT with
+ * synchronous_commit = apply or causal_reads = on.
+ *
+ * TODO: This may change -- Simon Riggs suggested latches for this.  Maybe
+ * pipes would work too (and avoid interrupting systems calls and allow for
+ * multiplexed IO with the replication socket).
+ */
+void
+WalRcvWakeup(void)
+{
+	if (WalRcv->pid != 0)
+		kill(WalRcv->pid, SIGUSR1);
+}
diff --git a/src/backend/replication/walreceiverfuncs.c b/src/backend/replication/walreceiverfuncs.c
index 4452f25..db9c397 100644
--- a/src/backend/replication/walreceiverfuncs.c
+++ b/src/backend/replication/walreceiverfuncs.c
@@ -25,9 +25,11 @@
 
 #include "access/xlog_internal.h"
 #include "postmaster/startup.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "storage/pmsignal.h"
 #include "storage/shmem.h"
+#include "utils/guc.h"
 #include "utils/timestamp.h"
 
 WalRcvData *WalRcv = NULL;
@@ -374,3 +376,23 @@ GetReplicationTransferLatency(void)
 
 	return ms;
 }
+
+/*
+ * Used by snapmgr to check if this standby has been authorized by the primary
+ * server to consider itself available for causal reads.  That is, to have
+ * applied all commits for which the COMMIT command has returned control on
+ * the primary server.
+ */
+bool
+WalRcvCausalReadsAvailable(void)
+{
+	WalRcvData *walrcv = WalRcv;
+	TimestampTz now = GetCurrentTimestamp();
+	bool result;
+
+	SpinLockAcquire(&walrcv->mutex);
+	result = walrcv->causalReadsUntil != 0 && now <= walrcv->causalReadsUntil;
+	SpinLockRelease(&walrcv->mutex);
+
+	return result;
+}
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 4a4643e..01b9c20 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -153,9 +153,14 @@ static StringInfoData tmpbuf;
  */
 static TimestampTz last_reply_timestamp = 0;
 
+static TimestampTz last_keepalive_timestamp = 0;
+
 /* Have we sent a heartbeat message asking for reply, since last reply? */
 static bool waiting_for_ping_response = false;
 
+/* How long do need to stay in JOINING state? */
+static TimestampTz causal_reads_joining_until = 0;
+
 /*
  * While streaming WAL in Copy mode, streamingDoneSending is set to true
  * after we have sent CopyDone. We should not send any more CopyData messages
@@ -242,6 +247,57 @@ InitWalSender(void)
 }
 
 /*
+ * If we are exiting unexpectedly, we may need to communicate with concurrent
+ * causal_reads commits to maintain the causal consistency guarantee.
+ */
+static void
+PrepareUncleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * We've lost contact with the standby, but it may still be alive.  We
+		 * can't let any causal_reads transactions return until we've stalled
+		 * for long enough for a zombie standby to start raising errors due
+		 * to lack of keepalives with early enough timestamps.
+		 */
+		elog(LOG, "standby \"%s\" is lost (no longer available for causal reads)", application_name);
+		CausalReadsBeginStall();
+
+		/*
+		 * We set the state to a lower level _after_ beginning the stall,
+		 * otherwise there would be a tiny window where commits could return
+		 * without observing the stall.
+		 */
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
+ * We are shutting down because we received a goodbye message from the
+ * walreceiver.
+ */
+static void
+PrepareCleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * The standby is shutting down, so it won't be running any more
+		 * transations.  It is therefore safe to stop waiting for it, and no
+		 * stall is necessary.
+		 */
+		elog(LOG, "standby \"%s\" is leaving (no longer available for causal reads)", application_name);
+
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
  * Clean up after an error.
  *
  * WAL sender processes don't use transactions like regular backends do.
@@ -264,7 +320,10 @@ WalSndErrorCleanup(void)
 
 	replication_active = false;
 	if (walsender_ready_to_stop)
+	{
+		PrepareUncleanExit();
 		proc_exit(0);
+	}
 
 	/* Revert back to startup state */
 	WalSndSetState(WALSNDSTATE_STARTUP);
@@ -276,6 +335,8 @@ WalSndErrorCleanup(void)
 static void
 WalSndShutdown(void)
 {
+	PrepareUncleanExit();
+
 	/*
 	 * Reset whereToSendOutput to prevent ereport from attempting to send any
 	 * more messages to the standby.
@@ -1386,6 +1447,7 @@ ProcessRepliesIfAny(void)
 		if (r < 0)
 		{
 			/* unexpected error or EOF */
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1402,6 +1464,7 @@ ProcessRepliesIfAny(void)
 		resetStringInfo(&reply_message);
 		if (pq_getmessage(&reply_message, 0))
 		{
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1451,6 +1514,7 @@ ProcessRepliesIfAny(void)
 				 * 'X' means that the standby is closing down the socket.
 				 */
 			case 'X':
+				PrepareCleanExit();
 				proc_exit(0);
 
 			default:
@@ -1543,15 +1607,29 @@ ProcessStandbyReplyMessage(void)
 	XLogRecPtr	writePtr,
 				flushPtr,
 				applyPtr;
+	int			applyLagMs;
 	bool		replyRequested;
+	TimestampTz now = GetCurrentTimestamp();
+	TimestampTz applyTimestamp;
 
 	/* the caller already consumed the msgtype byte */
 	writePtr = pq_getmsgint64(&reply_message);
 	flushPtr = pq_getmsgint64(&reply_message);
 	applyPtr = pq_getmsgint64(&reply_message);
 	(void) pq_getmsgint64(&reply_message);		/* sendTime; not used ATM */
+	applyTimestamp = IntegerTimestampToTimestampTz(pq_getmsgint64(&reply_message));
 	replyRequested = pq_getmsgbyte(&reply_message);
 
+	/* Compute the apply lag in milliseconds. */
+	if (applyTimestamp == 0)
+		applyLagMs = -1;
+	else
+#ifdef HAVE_INT64_TIMESTAMP
+		applyLagMs = (now - applyTimestamp) / 1000;
+#else
+		applyLagMs = (now - applyTimestamp) * 1000.0;
+#endif
+
 	elog(DEBUG2, "write %X/%X flush %X/%X apply %X/%X%s",
 		 (uint32) (writePtr >> 32), (uint32) writePtr,
 		 (uint32) (flushPtr >> 32), (uint32) flushPtr,
@@ -1568,16 +1646,81 @@ ProcessStandbyReplyMessage(void)
 	 */
 	{
 		WalSnd *walsnd = MyWalSnd;
+		WalSndCausalReadsState causal_reads_state = walsnd->causal_reads_state;
+		bool causal_reads_state_changed = false;
+
+		/*
+		 * Handle causal reads state transitions, if a causal_reads_timeout is
+		 * configured.
+		 */
+		if (causal_reads_timeout != 0)
+		{
+			if (applyLagMs >= 0 && applyLagMs < causal_reads_timeout)
+			{
+				if (causal_reads_state == WALSNDCRSTATE_UNAVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_JOINING;
+					causal_reads_joining_until =
+						TimestampTzPlusMilliseconds(now, causal_reads_timeout);
+					causal_reads_state_changed = true;
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING &&
+						 now >= causal_reads_joining_until)
+				{
+					causal_reads_state = WALSNDCRSTATE_AVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+			else
+			{
+				if (causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+					/*
+					 * We are dropping a causal reads available standby, so we
+					 * mustn't let any commit command that is waiting in
+					 * CausalReadsWaitForLSN return until we are sure that the
+					 * standby definitely knows that it's not available and
+					 * starts raising errors for causal_reads transactions.
+					 */
+					CausalReadsBeginStall();
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING)
+				{
+					/*
+					 * Dropping a joining standby doesn't require a stall,
+					 * because the standby doesn't think it's available, so
+					 * it's already raising the error for causal_reads
+					 * transactions.
+					 */
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+		}
 
 		SpinLockAcquire(&walsnd->mutex);
 		walsnd->write = writePtr;
 		walsnd->flush = flushPtr;
 		walsnd->apply = applyPtr;
+		walsnd->applyLagMs = applyLagMs;
+		walsnd->causal_reads_state = causal_reads_state;
 		SpinLockRelease(&walsnd->mutex);
+
+		if (causal_reads_state_changed)
+		{
+			WalSndKeepalive(true);
+			elog(LOG, "standby \"%s\" is %s", application_name,
+				 causal_reads_state == WALSNDCRSTATE_UNAVAILABLE ? "unavailable for causal reads" :
+				 causal_reads_state == WALSNDCRSTATE_JOINING ? "joining as a causal reads standby..." :
+				 causal_reads_state == WALSNDCRSTATE_AVAILABLE ? "available for causal reads" :
+				 "UNKNOWN");
+		}
 	}
 
 	if (!am_cascading_walsender)
-		SyncRepReleaseWaiters();
+		SyncRepReleaseWaiters(MyWalSnd->causal_reads_state >= WALSNDCRSTATE_JOINING);
 
 	/*
 	 * Advance our local xmin horizon when the client confirmed a flush.
@@ -1724,27 +1867,34 @@ WalSndComputeSleeptime(TimestampTz now)
 {
 	long		sleeptime = 10000;		/* 10 s */
 
-	if (wal_sender_timeout > 0 && last_reply_timestamp > 0)
+	if ((wal_sender_timeout > 0 || causal_reads_timeout > 0) && last_reply_timestamp > 0)
 	{
 		TimestampTz wakeup_time;
 		long		sec_to_timeout;
 		int			microsec_to_timeout;
 
-		/*
-		 * At the latest stop sleeping once wal_sender_timeout has been
-		 * reached.
-		 */
-		wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-												  wal_sender_timeout);
-
-		/*
-		 * If no ping has been sent yet, wakeup when it's time to do so.
-		 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
-		 * the timeout passed without a response.
-		 */
-		if (!waiting_for_ping_response)
+		if (causal_reads_timeout != 0)
+			wakeup_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+													  causal_reads_timeout /
+													  CAUSAL_READS_KEEPALIVE_RATIO);
+		else
+		{
+			/*
+			 * At the latest stop sleeping once wal_sender_timeout has been
+			 * reached.
+			 */
 			wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-													  wal_sender_timeout / 2);
+													  wal_sender_timeout);
+
+			/*
+			 * If no ping has been sent yet, wakeup when it's time to do so.
+			 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
+			 * the timeout passed without a response.
+			 */
+			if (!waiting_for_ping_response)
+				wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+														  wal_sender_timeout / 2);
+		}
 
 		/* Compute relative time until wakeup. */
 		TimestampDifference(now, wakeup_time,
@@ -1765,15 +1915,28 @@ static void
 WalSndCheckTimeOut(TimestampTz now)
 {
 	TimestampTz timeout;
+	int allowed_time;
 
 	/* don't bail out if we're doing something that doesn't require timeouts */
 	if (last_reply_timestamp <= 0)
 		return;
 
+	/*
+	 * If a causal_reads_timeout is configured, it is used instead of
+	 * wal_sender_timeout.  Ideally we'd use causal_reads_timeout / 2 +
+	 * allowance for network latency, but since walreceiver can become quite
+	 * bogged down fsyncing WAL we allow more tolerance.  (This could be
+	 * tightened up once standbys hand writing off to the WAL writer).
+	 */
+	if (causal_reads_timeout != 0)
+		allowed_time = causal_reads_timeout;
+	else
+		allowed_time = wal_sender_timeout;
+
 	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
-										  wal_sender_timeout);
+										  allowed_time);
 
-	if (wal_sender_timeout > 0 && now >= timeout)
+	if (allowed_time > 0 && now >= timeout)
 	{
 		/*
 		 * Since typically expiration of replication timeout means
@@ -1963,6 +2126,7 @@ InitWalSenderSlot(void)
 			walsnd->pid = MyProcPid;
 			walsnd->sentPtr = InvalidXLogRecPtr;
 			walsnd->state = WALSNDSTATE_STARTUP;
+			walsnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
 			walsnd->latch = &MyProc->procLatch;
 			SpinLockRelease(&walsnd->mutex);
 			/* don't need the lock anymore */
@@ -2732,6 +2896,24 @@ WalSndGetStateString(WalSndState state)
 	return "UNKNOWN";
 }
 
+/*
+ * Return a string constant representing the causal reads state. This is used
+ * in system views, and should *not* be translated.
+ */
+static const char *
+WalSndGetCausalReadsStateString(WalSndCausalReadsState causal_reads_state)
+{
+	switch (causal_reads_state)
+	{
+		case WALSNDCRSTATE_UNAVAILABLE:
+			return "unavailable";
+		case WALSNDCRSTATE_JOINING:
+			return "joining";
+		case WALSNDCRSTATE_AVAILABLE:
+			return "available";
+	}
+	return "UNKNOWN";
+}
 
 /*
  * Returns activity of walsenders, including pids and xlog locations sent to
@@ -2740,7 +2922,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	8
+#define PG_STAT_GET_WAL_SENDERS_COLS	10
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2788,8 +2970,10 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		XLogRecPtr	write;
 		XLogRecPtr	flush;
 		XLogRecPtr	apply;
+		int			applyLagMs;
 		int			priority;
 		WalSndState state;
+		WalSndCausalReadsState causalReadsState;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
 		bool		nulls[PG_STAT_GET_WAL_SENDERS_COLS];
 
@@ -2799,9 +2983,11 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		SpinLockAcquire(&walsnd->mutex);
 		sentPtr = walsnd->sentPtr;
 		state = walsnd->state;
+		causalReadsState = walsnd->causal_reads_state;
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
+		applyLagMs = walsnd->applyLagMs;
 		priority = walsnd->sync_standby_priority;
 		SpinLockRelease(&walsnd->mutex);
 
@@ -2833,6 +3019,23 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				nulls[5] = true;
 			values[5] = LSNGetDatum(apply);
 
+			if (applyLagMs < 0)
+				nulls[6] = true;
+			else
+			{
+				Interval *applyLagInterval = palloc(sizeof(Interval));
+
+				applyLagInterval->month = 0;
+				applyLagInterval->day = 0;
+#ifdef HAVE_INT64_TIMESTAMP
+				applyLagInterval->time = applyLagMs * 1000;
+#else
+				applyLagInterval->time = applyLagMs / 1000.0;
+#endif
+				nulls[6] = false;
+				values[6] = IntervalPGetDatum(applyLagInterval);
+			}
+
 			/*
 			 * Treat a standby such as a pg_basebackup background process
 			 * which always returns an invalid flush location, as an
@@ -2840,18 +3043,21 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 			 */
 			priority = XLogRecPtrIsInvalid(walsnd->flush) ? 0 : priority;
 
-			values[6] = Int32GetDatum(priority);
+			values[7] = Int32GetDatum(priority);
 
 			/*
 			 * More easily understood version of standby state. This is purely
 			 * informational, not different from priority.
 			 */
 			if (priority == 0)
-				values[7] = CStringGetTextDatum("async");
+				values[8] = CStringGetTextDatum("async");
 			else if (walsnd == sync_standby)
-				values[7] = CStringGetTextDatum("sync");
+				values[8] = CStringGetTextDatum("sync");
 			else
-				values[7] = CStringGetTextDatum("potential");
+				values[8] = CStringGetTextDatum("potential");
+
+			values[9] =
+				CStringGetTextDatum(WalSndGetCausalReadsStateString(causalReadsState));
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
@@ -2871,14 +3077,50 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 static void
 WalSndKeepalive(bool requestReply)
 {
+	TimestampTz now;
+	TimestampTz causal_reads_until;
+
 	elog(DEBUG2, "sending replication keepalive");
 
+	/*
+	 * If the walsender currently deems the standby to be available for causal
+	 * reads, then we authorize the standby to consider itself avialable until
+	 * a certain time in the future.  If we lose contact with the standby or
+	 * drop it from the set of standbys we wait for in causal_reads mode
+	 * because of excessive lag, then we'll stall until after that time to
+	 * maintain our causal consistency guarantee.
+	 */
+	now = GetCurrentTimestamp();
+	if (MyWalSnd->causal_reads_state < WALSNDCRSTATE_AVAILABLE)
+		causal_reads_until = 0; /* Not available */
+	else
+	{
+		/*
+		 * Since this timestamp is being sent to the standby where it will be
+		 * compared against a time generated by the standby's system clock, we
+		 * must consider clock skew.  First, we decide on a maximum tolerable
+		 * difference between system clocks.  If the primary's clock is ahead
+		 * of the standby's by more than this, then all bets are off (the
+		 * standby could falsely believe it is available).  If the primary's
+		 * clock is behind the standby's by more than this, then the standby
+		 * will err the other way and generate spurious errors in
+		 * causal_reads mode.  Rather than having a separate GUC for this,
+		 * we derive it from causal_reads_timeout.
+		 */
+		int max_clock_skew = causal_reads_timeout / CAUSAL_READS_CLOCK_SKEW_RATIO;
+
+		causal_reads_until =
+			TimestampTzPlusMilliseconds(now,
+										causal_reads_timeout - max_clock_skew);
+	}
+
 	/* construct the message... */
 	resetStringInfo(&output_message);
 	pq_sendbyte(&output_message, 'k');
 	pq_sendint64(&output_message, sentPtr);
-	pq_sendint64(&output_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(now));
 	pq_sendbyte(&output_message, requestReply ? 1 : 0);
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(causal_reads_until));
 
 	/* ... and send it wrapped in CopyData */
 	pq_putmessage_noblock('d', output_message.data, output_message.len);
@@ -2896,23 +3138,32 @@ WalSndKeepaliveIfNecessary(TimestampTz now)
 	 * Don't send keepalive messages if timeouts are globally disabled or
 	 * we're doing something not partaking in timeouts.
 	 */
-	if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
+	if ((wal_sender_timeout <= 0 && causal_reads_timeout == 0) || last_reply_timestamp <= 0)
 		return;
 
-	if (waiting_for_ping_response)
+	if (waiting_for_ping_response && causal_reads_timeout == 0)
 		return;
 
 	/*
 	 * If half of wal_sender_timeout has lapsed without receiving any reply
 	 * from the standby, send a keep-alive message to the standby requesting
 	 * an immediate reply.
+	 *
+	 * If causal_reads_timeout has been configured, use it to control
+	 * keepalive intervals rather than wal_sender_timeout.
 	 */
-	ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-											wal_sender_timeout / 2);
+	if (causal_reads_timeout != 0)
+		ping_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+												causal_reads_timeout /
+												CAUSAL_READS_KEEPALIVE_RATIO);
+	else
+		ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+												wal_sender_timeout / 2);
 	if (now >= ping_time)
 	{
 		WalSndKeepalive(true);
 		waiting_for_ping_response = true;
+		last_keepalive_timestamp = now;
 
 		/* Try to flush pending output to the client */
 		if (pq_flush_if_writable() != 0)
diff --git a/src/backend/utils/adt/timestamp.c b/src/backend/utils/adt/timestamp.c
index 8fbb310..12c8b88 100644
--- a/src/backend/utils/adt/timestamp.c
+++ b/src/backend/utils/adt/timestamp.c
@@ -1611,6 +1611,20 @@ IntegerTimestampToTimestampTz(int64 timestamp)
 #endif
 
 /*
+ * TimestampTzToIntegerTimestamp -- convert a native timestamp to int64 format
+ *
+ * When compiled with --enable-integer-datetimes, this is implemented as a
+ * no-op macro.
+ */
+#ifndef HAVE_INT64_TIMESTAMP
+int64
+TimestampTzToIntegerTimestamp(TimestampTz timestamp)
+{
+	return timestamp * 1000000;
+}
+#endif
+
+/*
  * TimestampDifference -- convert the difference between two timestamps
  *		into integer seconds and microseconds
  *
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index fda0fb9..23cfa85 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -351,6 +351,7 @@ static const struct config_enum_entry constraint_exclusion_options[] = {
 static const struct config_enum_entry synchronous_commit_options[] = {
 	{"local", SYNCHRONOUS_COMMIT_LOCAL_FLUSH, false},
 	{"remote_write", SYNCHRONOUS_COMMIT_REMOTE_WRITE, false},
+	{"apply", SYNCHRONOUS_COMMIT_REMOTE_APPLY, false},
 	{"on", SYNCHRONOUS_COMMIT_ON, false},
 	{"off", SYNCHRONOUS_COMMIT_OFF, false},
 	{"true", SYNCHRONOUS_COMMIT_ON, true},
@@ -1618,6 +1619,16 @@ static struct config_bool ConfigureNamesBool[] =
 		NULL, NULL, NULL
 	},
 
+	{
+		{"causal_reads", PGC_USERSET, REPLICATION_STANDBY,
+		 gettext_noop("Enables causal reads."),
+		 NULL
+		},
+		&causal_reads,
+		false,
+		NULL, NULL, NULL
+	},
+
 	/* End-of-list marker */
 	{
 		{NULL, 0, 0, NULL, NULL}, NULL, false, NULL, NULL, NULL
@@ -1776,6 +1787,17 @@ static struct config_int ConfigureNamesInt[] =
 	},
 
 	{
+		{"causal_reads_timeout", PGC_SIGHUP, REPLICATION_STANDBY,
+			gettext_noop("Sets the maximum apply lag before causal reads standbys are no longer available."),
+			NULL,
+			GUC_UNIT_MS
+		},
+		&causal_reads_timeout,
+		0, 0, INT_MAX,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"max_connections", PGC_POSTMASTER, CONN_AUTH_SETTINGS,
 			gettext_noop("Sets the maximum number of concurrent connections."),
 			NULL
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index 074935c..a466732 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -46,8 +46,11 @@
 
 #include "access/transam.h"
 #include "access/xact.h"
+#include "access/xlog.h"
 #include "lib/pairingheap.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
+#include "replication/walreceiver.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
 #include "storage/procarray.h"
@@ -209,6 +212,16 @@ GetTransactionSnapshot(void)
 				 "cannot take query snapshot during a parallel operation");
 
 		/*
+		 * In causal_reads mode on a standby, check if we have definitely
+		 * applied WAL for any COMMIT that returned successfully on the
+		 * primary.
+		 *
+		 * TODO: Machine readable error code?
+		 */
+		if (causal_reads && RecoveryInProgress() && !WalRcvCausalReadsAvailable())
+			elog(ERROR, "standby is not available for causal reads");
+
+		/*
 		 * In transaction-snapshot mode, the first snapshot must live until
 		 * end of xact regardless of what the caller does with it, so we must
 		 * make a copy of it rather than returning CurrentSnapshotData
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index cb1c2db..0f08ff5 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -60,7 +60,11 @@ typedef enum
 	SYNCHRONOUS_COMMIT_LOCAL_FLUSH,		/* wait for local flush only */
 	SYNCHRONOUS_COMMIT_REMOTE_WRITE,	/* wait for local flush and remote
 										 * write */
-	SYNCHRONOUS_COMMIT_REMOTE_FLUSH		/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_FLUSH,	/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_APPLY,	/* wait for local flush and remote
+										 * apply */
+	SYNCHRONOUS_COMMIT_CONSISTENT_APPLY /* wait for local flusha and remote
+										   apply with causal consistency */
 }	SyncCommitLevel;
 
 /* Define the default setting for synchonous_commit */
@@ -144,10 +148,13 @@ typedef void (*SubXactCallback) (SubXactEvent event, SubTransactionId mySubid,
  * EOXact... routines which run at the end of the original transaction
  * completion.
  */
+#define XACT_COMPLETION_SYNC_APPLY_FEEDBACK		(1U << 29)
 #define XACT_COMPLETION_UPDATE_RELCACHE_FILE	(1U << 30)
 #define XACT_COMPLETION_FORCE_SYNC_COMMIT		(1U << 31)
 
 /* Access macros for above flags */
+#define XactCompletionSyncApplyFeedback(xinfo) \
+	(!!(xinfo & XACT_COMPLETION_SYNC_APPLY_FEEDBACK))
 #define XactCompletionRelcacheInitFileInval(xinfo) \
 	(!!(xinfo & XACT_COMPLETION_UPDATE_RELCACHE_FILE))
 #define XactCompletionForceSyncCommit(xinfo) \
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 790ca66..8aeda11 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -235,6 +235,9 @@ extern void GetXLogReceiptTime(TimestampTz *rtime, bool *fromStream);
 extern XLogRecPtr GetXLogReplayRecPtr(TimeLineID *replayTLI);
 extern XLogRecPtr GetXLogInsertRecPtr(void);
 extern XLogRecPtr GetXLogWriteRecPtr(void);
+extern void SetXLogReplayTimestamp(TimestampTz timestamp);
+extern void SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn);
+extern TimestampTz GetXLogReplayTimestamp(XLogRecPtr *lsn);
 extern bool RecoveryIsPaused(void);
 extern void SetRecoveryPause(bool recoveryPause);
 extern TimestampTz GetLatestXTime(void);
@@ -267,6 +270,8 @@ extern bool CheckPromoteSignal(void);
 extern void WakeupRecovery(void);
 extern void SetWalWriterSleeping(bool sleeping);
 
+extern void XLogRequestWalReceiverReply(void);
+
 extern void assign_max_wal_size(int newval, void *extra);
 extern void assign_checkpoint_completion_target(double newval, void *extra);
 
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index d8640db..acb6796 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2783,7 +2783,7 @@ DATA(insert OID = 1936 (  pg_stat_get_backend_idset		PGNSP PGUID 12 1 100 0 0 f
 DESCR("statistics: currently active backend IDs");
 DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f f f t s r 1 0 2249 "23" "{23,26,23,26,25,25,25,16,1184,1184,1184,1184,869,25,23,28,28,16,25,25,23,16,25}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{pid,datid,pid,usesysid,application_name,state,query,waiting,xact_start,query_start,backend_start,state_change,client_addr,client_hostname,client_port,backend_xid,backend_xmin,ssl,sslversion,sslcipher,sslbits,sslcompression,sslclientdn}" _null_ _null_ pg_stat_get_activity _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active backends");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,23,25}" "{o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25,25}" "{o,o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state,causal_reads_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 2026 (  pg_backend_pid				PGNSP PGUID 12 1 0 0 0 f f f f t f s r 0 0 23 "" _null_ _null_ _null_ _null_ _null_ pg_backend_pid _null_ _null_ _null_ ));
 DESCR("statistics: current backend PID");
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index 71e2857..5746383 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -23,14 +23,33 @@
 #define SYNC_REP_NO_WAIT		-1
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
+#define SYNC_REP_WAIT_APPLY		2
+#define SYNC_REP_WAIT_CAUSAL_READS_APPLY 3
 
-#define NUM_SYNC_REP_WAIT_MODE	2
+#define NUM_SYNC_REP_WAIT_MODE	4
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
 #define SYNC_REP_WAITING			1
 #define SYNC_REP_WAIT_COMPLETE		2
 
+/*
+ * ratio of causal_read_timeout to max_clock_skew (4 means than the maximum
+ * tolerated clock difference between primary and standbys using causal_reads
+ * is 1/4 of causal_reads_timeout)
+ */
+#define CAUSAL_READS_CLOCK_SKEW_RATIO 4
+
+/*
+ * ratio of causal_reads_timeout to keepalive time (2 means that the effective
+ * keepalive time is 1/2 of the causal_reads_timeout GUC when it is non-zero)
+ */
+#define CAUSAL_READS_KEEPALIVE_RATIO 2
+
+/* GUC variables */
+extern int causal_reads_timeout;
+extern bool causal_reads;
+
 /* user-settable parameters for synchronous replication */
 extern char *SyncRepStandbyNames;
 
@@ -42,11 +61,17 @@ extern void SyncRepCleanupAtProcExit(void);
 
 /* called by wal sender */
 extern void SyncRepInitConfig(void);
-extern void SyncRepReleaseWaiters(void);
+extern void SyncRepReleaseWaiters(bool walsender_cr_available_or_joining);
 
 /* called by checkpointer */
 extern void SyncRepUpdateSyncStandbysDefined(void);
 
+/* called by user backend (xact.c) */
+extern void CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN);
+
+/* called by wal sender */
+extern void CausalReadsBeginStall(void);
+
 /* forward declaration to avoid pulling in walsender_private.h */
 struct WalSnd;
 extern struct WalSnd *SyncRepGetSynchronousStandby(void);
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 61255a9..507af9f 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -79,6 +79,13 @@ typedef struct
 	TimeLineID	receivedTLI;
 
 	/*
+	 * causallyReadsUntil is the time until which the primary has authorized
+	 * this standby to consider itself avialable for causal_reads mode, or 0
+	 * for not authorized.
+	 */
+	TimestampTz causalReadsUntil;
+
+	/*
 	 * latestChunkStart is the starting byte position of the current "batch"
 	 * of received WAL.  It's actually the same as the previous value of
 	 * receivedUpto before the last flush to disk.  Startup process can use
@@ -160,5 +167,8 @@ extern void RequestXLogStreaming(TimeLineID tli, XLogRecPtr recptr,
 extern XLogRecPtr GetWalRcvWriteRecPtr(XLogRecPtr *latestChunkStart, TimeLineID *receiveTLI);
 extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
+extern void WalRcvWakeup(void);
+
+extern bool WalRcvCausalReadsAvailable(void);
 
 #endif   /* _WALRECEIVER_H */
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 6dae480..deeb277 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -27,6 +27,13 @@ typedef enum WalSndState
 	WALSNDSTATE_STREAMING
 } WalSndState;
 
+typedef enum WalSndCausalReadsState
+{
+	WALSNDCRSTATE_UNAVAILABLE = 0,
+	WALSNDCRSTATE_JOINING,
+	WALSNDCRSTATE_AVAILABLE
+} WalSndCausalReadsState;
+
 /*
  * Each walsender has a WalSnd struct in shared memory.
  */
@@ -34,6 +41,7 @@ typedef struct WalSnd
 {
 	pid_t		pid;			/* this walsender's process id, or 0 */
 	WalSndState state;			/* this walsender's state */
+	WalSndCausalReadsState causal_reads_state; /* the walsender's causal reads state */
 	XLogRecPtr	sentPtr;		/* WAL has been sent up to this point */
 	bool		needreload;		/* does currently-open file need to be
 								 * reloaded? */
@@ -46,6 +54,7 @@ typedef struct WalSnd
 	XLogRecPtr	write;
 	XLogRecPtr	flush;
 	XLogRecPtr	apply;
+	int			applyLagMs;
 
 	/* Protects shared variables shown above. */
 	slock_t		mutex;
@@ -88,6 +97,14 @@ typedef struct
 	 */
 	bool		sync_standbys_defined;
 
+	/*
+	 * Until when must commits in causal_reads stall?  This is set to a time
+	 * in the future whenever a standby is dropped from the set of consistent
+	 * standbys, to give standbys time to know that they are not able to
+	 * provide causal consistency guarantees.
+	 */
+	TimestampTz	stall_causal_reads_until;
+
 	WalSnd		walsnds[FLEXIBLE_ARRAY_MEMBER];
 } WalSndCtlData;
 
diff --git a/src/include/utils/timestamp.h b/src/include/utils/timestamp.h
index 530fef1..0f4b166 100644
--- a/src/include/utils/timestamp.h
+++ b/src/include/utils/timestamp.h
@@ -227,9 +227,11 @@ extern bool TimestampDifferenceExceeds(TimestampTz start_time,
 #ifndef HAVE_INT64_TIMESTAMP
 extern int64 GetCurrentIntegerTimestamp(void);
 extern TimestampTz IntegerTimestampToTimestampTz(int64 timestamp);
+extern int64 TimestampTzToIntegerTimestamp(TimestampTz timestamp);
 #else
 #define GetCurrentIntegerTimestamp()	GetCurrentTimestamp()
 #define IntegerTimestampToTimestampTz(timestamp) (timestamp)
+#define TimestampTzToIntegerTimestamp(timestamp) (timestamp)
 #endif
 
 extern TimestampTz time_t_to_timestamptz(pg_time_t tm);
#2Simon Riggs
simon@2ndQuadrant.com
In reply to: Thomas Munro (#1)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On 11 November 2015 at 05:37, Thomas Munro <thomas.munro@enterprisedb.com>
wrote:

Many sites use hot standby servers to spread read-heavy workloads over more

hardware, or at least would like to. This works well today if your
application can tolerate some time lag on standbys. The problem is that
there is no guarantee of when a particular commit will become visible for
clients connected to standbys. The existing synchronous commit feature is
no help here because it guarantees only that the WAL has been flushed on
another server before commit returns. It says nothing about whether it has
been applied or whether it has been applied on the standby that you happen
to be talking to.

Thanks for working on this issue.

3. Commit on the primary with "causal_reads = on" waits for all
'available' standbys either to apply the commit record, or to cease to be
'available' and begin raising the error if they are still alive (because
their authorizations have expired).

This causes every writer to wait.

What we want is to isolate the wait only to people performing a write-read
sequence, so I think it should be readers that wait. Let's have that debate
up front before we start reviewing the patch.

--
Simon Riggs http://www.2ndQuadrant.com/
<http://www.2ndquadrant.com/&gt;
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#3Heikki Linnakangas
hlinnaka@iki.fi
In reply to: Simon Riggs (#2)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On 11/11/2015 10:23 AM, Simon Riggs wrote:

On 11 November 2015 at 05:37, Thomas Munro <thomas.munro@enterprisedb.com>
wrote:

Many sites use hot standby servers to spread read-heavy workloads over more

hardware, or at least would like to. This works well today if your
application can tolerate some time lag on standbys. The problem is that
there is no guarantee of when a particular commit will become visible for
clients connected to standbys. The existing synchronous commit feature is
no help here because it guarantees only that the WAL has been flushed on
another server before commit returns. It says nothing about whether it has
been applied or whether it has been applied on the standby that you happen
to be talking to.

Thanks for working on this issue.

+1.

3. Commit on the primary with "causal_reads = on" waits for all
'available' standbys either to apply the commit record, or to cease to be
'available' and begin raising the error if they are still alive (because
their authorizations have expired).

This causes every writer to wait.

What we want is to isolate the wait only to people performing a write-read
sequence, so I think it should be readers that wait. Let's have that debate
up front before we start reviewing the patch.

Agreed. And in the write-read sequence, you don't need to wait at the
write either, it's enough that you wait just before you start doing the
read. An application might do a lot of other things between the two, so
that in most cases, there would in fact be no waiting as the record is
already applied when you perform the read.

I'm thinking the client should get some kind of a token back from the
commit, and it could use the token on the standby, to wait for that
commit to be applied. The token could be just the XID, or the LSN of the
commit record. Or the application could generate the token and pass it
to the server in the commit, similar to how 2PC works. So the
interaction would be something like:

In master:
BEGIN;
INSERT INTO FOO ...;
COMMIT;
Server returns: COMMITted with token 1234

Later, in standby:
BEGIN WAIT FOR COMMIT 1234 TO BE VISIBLE;
SELECT * FROM foo;
...

- Heikki

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#4Atri Sharma
atri.jiit@gmail.com
In reply to: Heikki Linnakangas (#3)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

I'm thinking the client should get some kind of a token back from the

commit, and it could use the token on the standby, to wait for that commit
to be applied. The token could be just the XID, or the LSN of the commit
record. Or the application could generate the token and pass it to the
server in the commit, similar to how 2PC works. So the interaction would be
something like:

In master:
BEGIN;
INSERT INTO FOO ...;
COMMIT;
Server returns: COMMITted with token 1234

Later, in standby:
BEGIN WAIT FOR COMMIT 1234 TO BE VISIBLE;
SELECT * FROM foo;

+1.

The LSN should be good enough IMO.

#5Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Heikki Linnakangas (#3)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Wed, Nov 11, 2015 at 9:42 PM, Heikki Linnakangas <hlinnaka@iki.fi> wrote:

On 11/11/2015 10:23 AM, Simon Riggs wrote:

On 11 November 2015 at 05:37, Thomas Munro <thomas.munro@enterprisedb.com

wrote:

Many sites use hot standby servers to spread read-heavy workloads over
more

hardware, or at least would like to. This works well today if your
application can tolerate some time lag on standbys. The problem is that
there is no guarantee of when a particular commit will become visible for
clients connected to standbys. The existing synchronous commit feature
is
no help here because it guarantees only that the WAL has been flushed on
another server before commit returns. It says nothing about whether it
has
been applied or whether it has been applied on the standby that you
happen
to be talking to.

Thanks for working on this issue.

+1.

3. Commit on the primary with "causal_reads = on" waits for all

'available' standbys either to apply the commit record, or to cease to be
'available' and begin raising the error if they are still alive (because
their authorizations have expired).

This causes every writer to wait.

What we want is to isolate the wait only to people performing a write-read
sequence, so I think it should be readers that wait. Let's have that
debate
up front before we start reviewing the patch.

Agreed. And in the write-read sequence, you don't need to wait at the
write either, it's enough that you wait just before you start doing the
read. An application might do a lot of other things between the two, so
that in most cases, there would in fact be no waiting as the record is
already applied when you perform the read.

I'm thinking the client should get some kind of a token back from the
commit, and it could use the token on the standby, to wait for that commit
to be applied. The token could be just the XID, or the LSN of the commit
record. Or the application could generate the token and pass it to the
server in the commit, similar to how 2PC works. So the interaction would be
something like:

In master:
BEGIN;
INSERT INTO FOO ...;
COMMIT;
Server returns: COMMITted with token 1234

Later, in standby:
BEGIN WAIT FOR COMMIT 1234 TO BE VISIBLE;
SELECT * FROM foo;
...

I thought about this question, and considered three different approaches:

1. Reader waits with exposed LSNs, as Heikki suggests. This is what
BerkeleyDB does in "read-your-writes" mode. It means that application
developers have the responsibility for correctly identifying transactions
with causal dependencies and dealing with LSNs (or whatever equivalent
tokens), potentially even passing them to other processes where the
transactions are causally dependent but run by multiple communicating
clients (for example, communicating microservices). This makes it
difficult to retrofit load balancing to pre-existing applications and (like
anything involving concurrency) difficult to reason about as applications
grow in size and complexity. It is efficient if done correctly, but it is
a tax on application complexity.

2. Reader waits for a conservatively chosen LSN. This is roughly what
MySQL derivatives do in their "causal_reads = on" and "wsrep_sync_wait = 1"
modes. Read transactions would start off by finding the current end of WAL
on the primary, since that must be later than any commit that already
completed, and then waiting for that to apply locally. That means every
read transaction waits for a complete replication lag period, potentially
unnecessarily. This is tax on readers with unnecessary waiting.

3. Writer waits, as proposed. In this model, there is no tax on readers
(they have zero overhead, aside from the added complexity of dealing with
the possibility of transactions being rejected when a standby falls behind
and is dropped from 'available' status; but database clients must already
deal with certain types of rare rejected queries/failures such as
deadlocks, serialization failures, server restarts etc). This is a tax on
writers.

My thinking was that the reason for wanting to load balance over a set of
hot standbys is because you have a very read-heavy workload, so it makes
sense to tax the writers and leave the many dominant readers unburdened, so
(3) should be better than (2) for the majority of users who want such a
configuration. (Note also that it's not a requirement to tax every write;
with this proposal you can set causal_reads to off for those transactions
where you know there is no possibility of a causally dependent read).

As for (1), my thinking was that most application developers would probably
prefer not to have to deal with that type of interface. For users who do
want to do that, it would be comparatively simple to make that possible,
and would not conflict with this proposal. This proposal could be used by
people retrofitting load balancing to an existing applications with
relative ease, or simply not wanting to have to deal with LSNs and
complexity. (I have considered proposing
pg_wait_for_xlog_replay_location(lsn, timeout) separately, which could be
called on a standby with the lsn obtained from pg_current_xlog_location()
on the primary any time after a COMMIT completes, but I was thinking of
that as a different feature addressing a different user base: people
prepared to do more work to squeeze out some extra performance.)

--
Thomas Munro
http://www.enterprisedb.com

#6Ants Aasma
ants.aasma@eesti.ee
In reply to: Thomas Munro (#5)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Wed, Nov 11, 2015 at 11:22 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Wed, Nov 11, 2015 at 9:42 PM, Heikki Linnakangas <hlinnaka@iki.fi> wrote:

On 11/11/2015 10:23 AM, Simon Riggs wrote:

Thanks for working on this issue.

+1.

+1. I have seen a lot of interest for something along these lines.

I'm thinking the client should get some kind of a token back from the
commit, and it could use the token on the standby, to wait for that commit
to be applied. The token could be just the XID, or the LSN of the commit
record. Or the application could generate the token and pass it to the
server in the commit, similar to how 2PC works. So the interaction would be
something like:

In master:
BEGIN;
INSERT INTO FOO ...;
COMMIT;
Server returns: COMMITted with token 1234

Later, in standby:
BEGIN WAIT FOR COMMIT 1234 TO BE VISIBLE;
SELECT * FROM foo;
...

To avoid read anomalies (backwards timetravel) it should also be
possible to receive a token from read-only transactions based on the
latest snapshot used.

My thinking was that the reason for wanting to load balance over a set of
hot standbys is because you have a very read-heavy workload, so it makes
sense to tax the writers and leave the many dominant readers unburdened, so
(3) should be better than (2) for the majority of users who want such a
configuration. (Note also that it's not a requirement to tax every write;
with this proposal you can set causal_reads to off for those transactions
where you know there is no possibility of a causally dependent read).

As for (1), my thinking was that most application developers would probably
prefer not to have to deal with that type of interface. For users who do
want to do that, it would be comparatively simple to make that possible, and
would not conflict with this proposal. This proposal could be used by
people retrofitting load balancing to an existing applications with relative
ease, or simply not wanting to have to deal with LSNs and complexity. (I
have considered proposing pg_wait_for_xlog_replay_location(lsn, timeout)
separately, which could be called on a standby with the lsn obtained from
pg_current_xlog_location() on the primary any time after a COMMIT completes,
but I was thinking of that as a different feature addressing a different
user base: people prepared to do more work to squeeze out some extra
performance.)

Although I still think that 1) is the correct long term solution I
must say that I agree with the reasoning presented. I think we should
review the API in the light that in the future we might have a mix of
clients, some clients that are able to keep track of causality tokens
and either want to wait when a read request arrives, or pick a host to
use based on the token, and then there are "dumb" clients that want to
use write side waits.

Also, it should be possible to configure which standbys are considered
for waiting on. Otherwise a reporting slave will occasionally catch up
enough to be considered "available" and then cause a latency peak when
a long query blocks apply again.

Regards,
Ants Aasma
--
Cybertec Schönig & Schönig GmbH
Gröhrmühlgasse 26
A-2700 Wiener Neustadt
Web: http://www.postgresql-support.de

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#7Robert Haas
robertmhaas@gmail.com
In reply to: Simon Riggs (#2)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Wed, Nov 11, 2015 at 3:23 AM, Simon Riggs <simon@2ndquadrant.com> wrote:

This causes every writer to wait.

What we want is to isolate the wait only to people performing a write-read
sequence, so I think it should be readers that wait. Let's have that debate
up front before we start reviewing the patch.

One advantage of having writers wait is that the master and its read
slaves can't ever get too far apart. Suppose the master is generating
WAL much faster than the read slaves (or one of them) can replay it.
You might say it sucks to slow down the master to the speed the slaves
can keep up with, and that's true. On the other hand, if the master
is allowed to run ahead, then a process that sends a read query to a
standby which has gotten far behind might have to wait minutes or
hours for it to catch up. I think a lot of people are enabling
synchronous replication today just for the purpose of avoiding this
problem - keeping the two machines "together in time" makes the
overall system behavior a lot more predictable.

Also, if we made readers wait, wouldn't that require a network
roundtrip to the master every time a query on a reader wanted a new
snapshot? That seems like it would be unbearably expensive.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#8Peter Eisentraut
peter_e@gmx.net
In reply to: Thomas Munro (#5)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On 11/11/15 4:22 AM, Thomas Munro wrote:

My thinking was that the reason for wanting to load balance over a set
of hot standbys is because you have a very read-heavy workload, so it
makes sense to tax the writers and leave the many dominant readers
unburdened, so (3) should be better than (2) for the majority of users
who want such a configuration.

One problem I can see is that even if you have a read-heavy workload,
the writes can still be a bottleneck, since they are necessarily bound
to one node. And so if the feature proposal is, we can make your reads
more consistent but the writes will become slower, then that's not a
good deal.

More generally, no matter whether you pick the writers or the readers to
wait, if you assume that read-only slaves are an application performance
feature, then it's questionable how much better such applications will
perform overall when network-bound waits are introduced in the system.

I think in practice applications that are busy enough to worry about
this don't really work like that anyway. For example, the writes should
go to a message queue and are written out whenever, with a copy kept in
a cache for display in the meantime. Maybe there could be additional
features to make managing this easier.

I think there are a lot of different variations of this in practice, not
only depending on the workload and other measurables, but also
business-dependent decisions on application behavior and degradability.

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#9Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Ants Aasma (#6)
1 attachment(s)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Thu, Nov 12, 2015 at 12:10 AM, Ants Aasma <ants.aasma@eesti.ee> wrote:

On Wed, Nov 11, 2015 at 11:22 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Wed, Nov 11, 2015 at 9:42 PM, Heikki Linnakangas <hlinnaka@iki.fi>

wrote:

On 11/11/2015 10:23 AM, Simon Riggs wrote:

Thanks for working on this issue.

+1.

+1. I have seen a lot of interest for something along these lines.

I'm thinking the client should get some kind of a token back from the
commit, and it could use the token on the standby, to wait for that

commit

to be applied. The token could be just the XID, or the LSN of the commit
record. Or the application could generate the token and pass it to the
server in the commit, similar to how 2PC works. So the interaction

would be

something like:

In master:
BEGIN;
INSERT INTO FOO ...;
COMMIT;
Server returns: COMMITted with token 1234

Later, in standby:
BEGIN WAIT FOR COMMIT 1234 TO BE VISIBLE;
SELECT * FROM foo;
...

To avoid read anomalies (backwards timetravel) it should also be
possible to receive a token from read-only transactions based on the
latest snapshot used.

My thinking was that the reason for wanting to load balance over a set of
hot standbys is because you have a very read-heavy workload, so it makes
sense to tax the writers and leave the many dominant readers unburdened,

so

(3) should be better than (2) for the majority of users who want such a
configuration. (Note also that it's not a requirement to tax every

write;

with this proposal you can set causal_reads to off for those transactions
where you know there is no possibility of a causally dependent read).

As for (1), my thinking was that most application developers would

probably

prefer not to have to deal with that type of interface. For users who do
want to do that, it would be comparatively simple to make that possible,

and

would not conflict with this proposal. This proposal could be used by
people retrofitting load balancing to an existing applications with

relative

ease, or simply not wanting to have to deal with LSNs and complexity. (I
have considered proposing pg_wait_for_xlog_replay_location(lsn, timeout)
separately, which could be called on a standby with the lsn obtained from
pg_current_xlog_location() on the primary any time after a COMMIT

completes,

but I was thinking of that as a different feature addressing a different
user base: people prepared to do more work to squeeze out some extra
performance.)

Although I still think that 1) is the correct long term solution I
must say that I agree with the reasoning presented. I think we should
review the API in the light that in the future we might have a mix of
clients, some clients that are able to keep track of causality tokens
and either want to wait when a read request arrives, or pick a host to
use based on the token, and then there are "dumb" clients that want to
use write side waits.

Exactly!

I see the causality tokens approach (thank you for that terminology) not so
much as a "long term" solution, but rather as an expert feature likely to
interest a small number of sophisticated users willing to take on more
responsibility in exchange for greater control. We should definitely add
support for that, and I expect the patch would be fairly simple and short.

But I believe the vast majority of users would like to be able to run new
and existing plain SQL on any node and see the data they just wrote, with
graceful failure modes, and without extra conceptual load or invasive code
changes. So I think we should cater for that mode of usage that too.

Also, it should be possible to configure which standbys are considered

for waiting on. Otherwise a reporting slave will occasionally catch up
enough to be considered "available" and then cause a latency peak when
a long query blocks apply again.

Good point. Here's a new version which adds the GUC
causal_reads_standby_names, defaulting to '*' (but as before, the feature
is not activated until you set causal_reads_timeout). Now you can list
standby names explicitly if you want a way to exclude certain standbys.
Also, I noticed that cascaded standbys shouldn't be available for causal
reads, so I added a check for that.

--
Thomas Munro
http://www.enterprisedb.com

Attachments:

causal-reads-v2.patchapplication/octet-stream; name=causal-reads-v2.patchDownload
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 47312f6..535b5a9 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,10 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
+	{
+		CausalReadsWaitForLSN(XactLastRecEnd);
 		SyncRepWaitForLSN(XactLastRecEnd);
+	}
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5117,6 +5120,13 @@ XactLogCommitRecord(TimestampTz commit_time,
 		xl_xinfo.xinfo |= XACT_COMPLETION_FORCE_SYNC_COMMIT;
 
 	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this commit is applied.
+	 */
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY || causal_reads)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
+	/*
 	 * Relcache invalidations requires information about the current database
 	 * and so does logical decoding.
 	 */
@@ -5452,6 +5462,19 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 	if (XactCompletionForceSyncCommit(parsed->xinfo))
 		XLogFlush(lsn);
 
+	/*
+	 * Record the primary's timestamp for the commit record, so it can be used
+	 * for tracking replay lag.
+	 */
+	SetXLogReplayTimestamp(parsed->xact_time);
+
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit or causal reads), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 /*
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 08d1682..7779c34 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -80,6 +80,8 @@ extern uint32 bootstrap_data_checksum_version;
 #define PROMOTE_SIGNAL_FILE		"promote"
 #define FALLBACK_PROMOTE_SIGNAL_FILE "fallback_promote"
 
+/* Size of the circular buffer of timestamped LSNs. */
+#define MAX_TIMESTAMPED_LSNS 256
 
 /* User-settable parameters */
 int			max_wal_size = 64;	/* 1 GB */
@@ -346,6 +348,12 @@ static XLogRecPtr RedoRecPtr;
 static bool doPageWrites;
 
 /*
+ * doRequestWalReceiverReply is used by recovery code to ask the main recovery
+ * loop to trigger a walreceiver reply.
+ */
+static bool doRequestWalReceiverReply;
+
+/*
  * RedoStartLSN points to the checkpoint's REDO location which is specified
  * in a backup label file, backup history file or control file. In standby
  * mode, XLOG streaming usually starts from the position where an invalid
@@ -357,6 +365,13 @@ static bool doPageWrites;
  */
 static XLogRecPtr RedoStartLSN = InvalidXLogRecPtr;
 
+/*
+ * LastReplayedTimestamp can be set by redo handlers when they apply a record
+ * that carries a timestamp, by calling SetXLogReplayedTimestamp.  The xlog
+ * apply loop can then update the value in shared memory.
+ */
+static TimestampTz LastReplayedTimestamp = 0;
+
 /*----------
  * Shared-memory data structures for XLOG control
  *
@@ -632,6 +647,21 @@ typedef struct XLogCtlData
 	/* current effective recovery target timeline */
 	TimeLineID	RecoveryTargetTLI;
 
+	/* timestamp from the most recently applied record that carried a timestamp. */
+	TimestampTz lastReplayedTimestamp;
+
+	/*
+	 * We maintain a circular buffer of LSNs and associated timestamps.
+	 * Walreceiver writes into it using information from timestamps, and the
+	 * startup recovery process reads from it and notifies walreceiver when
+	 * LSNs are replayed so that the timestamps can be fed back to the
+	 * upstream server, to track lag.
+	 */
+	Index		timestampedLsnRead;
+	Index		timestampedLsnWrite;
+	XLogRecPtr	timestampedLsn[MAX_TIMESTAMPED_LSNS];
+	TimestampTz	timestampedLsnTime[MAX_TIMESTAMPED_LSNS];
+
 	/*
 	 * timestamp of when we started replaying the current chunk of WAL data,
 	 * only relevant for replication or archive recovery
@@ -6844,14 +6874,57 @@ StartupXLOG(void)
 				error_context_stack = errcallback.previous;
 
 				/*
-				 * Update lastReplayedEndRecPtr after this record has been
-				 * successfully replayed.
+				 * Update lastReplayedEndRecPtr and lastReplayedTimestamp
+				 * after this record has been successfully replayed.
 				 */
 				SpinLockAcquire(&XLogCtl->info_lck);
 				XLogCtl->lastReplayedEndRecPtr = EndRecPtr;
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
+				if (LastReplayedTimestamp != 0)
+				{
+					/* If replaying a record produced a timestamp, use that. */
+					XLogCtl->lastReplayedTimestamp = LastReplayedTimestamp;
+					LastReplayedTimestamp = 0;
+				}
+				else
+				{
+					/*
+					 * If we have applied LSNs associated with timestamps
+					 * received by walreceiver, then use the recorded
+					 * timestamp.  We consume from the read end of the
+					 * circular buffer.
+					 */
+					while (XLogCtl->timestampedLsnRead !=
+						   XLogCtl->timestampedLsnWrite &&
+						   XLogCtl->timestampedLsn[XLogCtl->timestampedLsnRead]
+						   <= EndRecPtr)
+					{
+						if (XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead] >
+							XLogCtl->lastReplayedTimestamp)
+						{
+							XLogCtl->lastReplayedTimestamp =
+								XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead];
+							doRequestWalReceiverReply = true;
+						}
+						++XLogCtl->timestampedLsnRead;
+					}
+				}
 				SpinLockRelease(&XLogCtl->info_lck);
 
+				/*
+				 * If rm_redo reported that it applied a commit record that
+				 * the master is waiting for by calling
+				 * XLogRequestWalReceiverReply, or we encountered a WAL
+				 * location that was associated with a timestamp above, then
+				 * we wake up the receiver so that it notices the updated
+				 * lastReplayedEndRecPtr and sends a reply to the master.
+				 */
+				if (doRequestWalReceiverReply)
+				{
+					doRequestWalReceiverReply = false;
+					WalRcvWakeup();
+				}
+
 				/* Remember this record as the last-applied one */
 				LastRec = ReadRecPtr;
 
@@ -11580,3 +11653,103 @@ SetWalWriterSleeping(bool sleeping)
 	XLogCtl->WalWriterSleeping = sleeping;
 	SpinLockRelease(&XLogCtl->info_lck);
 }
+
+/*
+ * Called by redo code to indicate that the xlog replay loop should wake up
+ * the walreceiver process so that a reply can be sent to the primary.
+ */
+void
+XLogRequestWalReceiverReply(void)
+{
+	doRequestWalReceiverReply = true;
+}
+
+/*
+ * Merge timestamps from keepalive messages with the timestamps from WAL
+ * records, so that we can track lag while idle or while replaying large
+ * amounts of WAL without commit records.  In the former case there is no lag,
+ * and in the latter case we will remember a timestamp that goes with an
+ * arbitrary LSN, and wait for that LSN to be replayed before using the
+ * timestamp.
+ *
+ * This is called by walreceiver on standby servers when keepalive messages
+ * arrive.
+ */
+void
+SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn)
+{
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn == XLogCtl->lastReplayedEndRecPtr)
+	{
+		/*
+		 * That is the last replayed LSN: we are fully replayed, so we can
+		 * update the replay timestamp immediately.
+		 */
+		XLogCtl->lastReplayedTimestamp = timestamp;
+	}
+	else
+	{
+		/*
+		 * There is WAL still to be applied.  We will associate the timestamp
+		 * with this WAL position and wait for it to be replayed.  We add it
+		 * at the 'write' end of the circular buffer of LSN/timestamp
+		 * mappings, which the replay loop will eventually read.
+		 */
+		Index w = XLogCtl->timestampedLsnWrite;
+		Index r = XLogCtl->timestampedLsnRead;
+
+		XLogCtl->timestampedLsn[w] = lsn;
+		XLogCtl->timestampedLsnTime[w] = timestamp;
+
+		/* Advance the write point. */
+		w = (w + 1) % MAX_TIMESTAMPED_LSNS;
+		XLogCtl->timestampedLsnWrite = w;
+		if (w == r)
+		{
+			/*
+			 * The buffer is full.  Advance the read point (throwing away
+			 * oldest values; we will begin to oversestimate replay lag, until
+			 * lag decreases to a size our buffer can manage, or the next
+			 * commit record is replayed).
+			 */
+			r = (r + 1) % MAX_TIMESTAMPED_LSNS;
+			XLogCtl->timestampedLsnRead = r;
+		}
+	}
+	SpinLockRelease(&XLogCtl->info_lck);
+}
+
+/*
+ * Set the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary.  This can be called by redo handlers that have
+ * an appropriate timestamp (currently only commit records).  Updating the
+ * shared memory value is deferred until after the redo handler returns.
+ */
+void
+SetXLogReplayTimestamp(TimestampTz timestamp)
+{
+	LastReplayedTimestamp = timestamp;
+}
+
+/*
+ * Get the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the master, and also the most recently applied LSN.  (Note
+ * that the timestamp and the LSN don't necessarily relate to the same
+ * record.)
+ *
+ * This is similar to GetLatestXTime, except that it is not only advanced by
+ * commit records (see SetXLogReplayTimestampAtLsn).
+ */
+TimestampTz
+GetXLogReplayTimestamp(XLogRecPtr *lsn)
+{
+	TimestampTz result;
+
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn)
+		*lsn = XLogCtl->lastReplayedEndRecPtr;
+	result = XLogCtl->lastReplayedTimestamp;
+	SpinLockRelease(&XLogCtl->info_lck);
+
+	return result;
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index ccc030f..f9b0e53 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -647,8 +647,10 @@ CREATE VIEW pg_stat_replication AS
             W.write_location,
             W.flush_location,
             W.replay_location,
+            W.replay_lag,
             W.sync_priority,
-            W.sync_state
+            W.sync_state,
+            W.causal_reads_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
             pg_stat_get_wal_senders() AS W
     WHERE S.usesysid = U.oid AND
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 325239d..91dbd35 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -57,6 +57,11 @@
 #include "utils/builtins.h"
 #include "utils/ps_status.h"
 
+/* GUC variables */
+int causal_reads_timeout;
+bool causal_reads;
+char *causal_reads_standby_names;
+
 /* User-settable parameters for sync rep */
 char	   *SyncRepStandbyNames;
 
@@ -69,7 +74,7 @@ static int	SyncRepWaitMode = SYNC_REP_NO_WAIT;
 
 static void SyncRepQueueInsert(int mode);
 static void SyncRepCancelWait(void);
-static int	SyncRepWakeQueue(bool all, int mode);
+static int	SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn);
 
 static int	SyncRepGetStandbyPriority(void);
 
@@ -83,6 +88,239 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * ===========================================================
  */
 
+static bool
+SyncRepCheckEarlyExit(void)
+{
+	/*
+	 * If a wait for synchronous replication is pending, we can neither
+	 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
+	 * lead the client to believe that that the transaction aborted, which
+	 * is not true: it's already committed locally. The former is no good
+	 * either: the client has requested synchronous replication, and is
+	 * entitled to assume that an acknowledged commit is also replicated,
+	 * which might not be true. So in this case we issue a WARNING (which
+	 * some clients may be able to interpret) and shut off further output.
+	 * We do NOT reset ProcDiePending, so that the process will die after
+	 * the commit is cleaned up.
+	 */
+	if (ProcDiePending)
+	{
+		ereport(WARNING,
+				(errcode(ERRCODE_ADMIN_SHUTDOWN),
+				 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * It's unclear what to do if a query cancel interrupt arrives.  We
+	 * can't actually abort at this point, but ignoring the interrupt
+	 * altogether is not helpful, so we just terminate the wait with a
+	 * suitable warning.
+	 */
+	if (QueryCancelPending)
+	{
+		QueryCancelPending = false;
+		ereport(WARNING,
+				(errmsg("canceling wait for synchronous replication due to user request"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * If the postmaster dies, we'll probably never get an
+	 * acknowledgement, because all the wal sender processes will exit. So
+	 * just bail out.
+	 */
+	if (!PostmasterIsAlive())
+	{
+		ProcDiePending = true;
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	return false;
+}
+
+/*
+ * Check if we can stop waiting for causal consistency.  We can stop waiting
+ * when the following conditions are met:
+ *
+ * 1.  All walsenders currently in 'joining' or 'available' state have
+ * applied the target LSN.
+ *
+ * 2.  Any stall periods caused by standbys dropping out of 'available' state
+ * have passed, so that we can be sure that any causalReadsUntil authorization
+ * has expired.
+ *
+ * The output parameter 'waitingFor' is set to the number of nodes we are
+ * currently waiting for.  The output parameters 'stallTimeMillis' is set to
+ * the number of milliseconds we need to wait for to observe any current
+ * commit stall.
+ *
+ * Returns true if commit can return control, because every standby has either
+ * applied the LSN or started rejecting causal_reads transactions.
+ */
+static bool
+CausalReadsCommitCanReturn(XLogRecPtr XactCommitLSN,
+						   int *waitingFor,
+						   long *stallTimeMillis)
+{
+	int i;
+	TimestampTz now;
+
+	/* Count how many joining/available nodes we are waiting for. */
+	*waitingFor = 0;
+	for (i = 0; i < max_wal_senders; ++i)
+	{
+		WalSnd *walsnd = &WalSndCtl->walsnds[i];
+
+		/*
+		 * Assuming atomic read of pid_t, we can check walsnd->pid without
+		 * acquiring the spinlock to avoid memory synchronization costs for
+		 * unused walsender slots.  We see a value that existed sometime at
+		 * least as recently as the last memory barrier.
+		 */
+		if (walsnd->pid != 0)
+		{
+			/*
+			 * We need to hold the spinlock to read LSNs, because we can't be
+			 * sure they can be read atomically.
+			 */
+			SpinLockAcquire(&walsnd->mutex);
+			if (walsnd->pid != 0 && walsnd->causal_reads_state >= WALSNDCRSTATE_JOINING)
+			{
+				if (walsnd->apply < XactCommitLSN)
+					++*waitingFor;
+			}
+			SpinLockRelease(&walsnd->mutex);
+		}
+	}
+
+	/* Check if there is a stall in progress that we need to observe. */
+	now = GetCurrentTimestamp();
+	LWLockAcquire(SyncRepLock, LW_SHARED);
+	if (WalSndCtl->stall_causal_reads_until > now)
+	{
+		long seconds;
+		int usecs;
+
+		/* Compute how long we have to wait, rounded up to nearest ms. */
+		TimestampDifference(now, WalSndCtl->stall_causal_reads_until,
+							&seconds, &usecs);
+		*stallTimeMillis = seconds * 1000 + (usecs + 999) / 1000;
+	}
+	else
+		*stallTimeMillis = 0;
+	LWLockRelease(SyncRepLock);
+
+	/* We are done if we are not waiting for any nodes or stalls. */
+	return *waitingFor == 0 && *stallTimeMillis == 0;
+}
+
+/*
+ * Wait for causal consistency in causal_reads mode, if requested by user.
+ */
+void
+CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN)
+{
+	long stallTimeMillis;
+	int waitingFor;
+
+	/* Leave if we aren't in causal_reads mode. */
+	if (!causal_reads)
+		return;
+
+	for (;;)
+	{
+		/* Reset latch before checking state. */
+		ResetLatch(MyLatch);
+
+		/*
+		 * Join the queue to be woken up if any causal reads joining/available
+		 * standby applies XactCommitLSN, if we aren't already in it.  We
+		 * don't actually know if we need to wait for any peers yet, but we
+		 * have to register just in case before checking the walsenders' state
+		 * to avoid a race condition that could occur if we did it after
+		 * calling CausalReadsCommitCanReturn.  (SyncRepWaitForLSN doesn't
+		 * have to do this because it can check the highest-seen LSN in
+		 * walsndctl->lsn[mode] which is protected by SyncRepLock, the same
+		 * lock as the queues.  We can't do that here, because there is no
+		 * single highest-seen LSN that is useful.  We must check
+		 * walsnd->apply for all relevant walsenders.  Therefore we must
+		 * register for notifications first, so that we can be notified via
+		 * our latch of any standby applying the LSN we're interested in after
+		 * we check but before we start waiting, or we could wait forever for
+		 * something that has already happened.)
+		 */
+		LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+		if (MyProc->syncRepState != SYNC_REP_WAITING)
+		{
+			MyProc->waitLSN = XactCommitLSN;
+			MyProc->syncRepState = SYNC_REP_WAITING;
+			SyncRepQueueInsert(SYNC_REP_WAIT_CAUSAL_READS_APPLY);
+			Assert(SyncRepQueueIsOrderedByLSN(SYNC_REP_WAIT_CAUSAL_READS_APPLY));
+		}
+		LWLockRelease(SyncRepLock);
+
+		/* Check if we're done. */
+		if (CausalReadsCommitCanReturn(XactCommitLSN, &waitingFor, &stallTimeMillis))
+		{
+			SyncRepCancelWait();
+			break;
+		}
+
+		Assert(waitingFor > 0 || stallTimeMillis > 0);
+
+		/* If we aren't actually waiting for any standbys, leave the queue. */
+		if (waitingFor == 0)
+			SyncRepCancelWait();
+
+		/* Update the ps title. */
+		if (update_process_title)
+		{
+			char buffer[80];
+
+			snprintf(buffer, sizeof(buffer),
+					 "waiting for %d peer(s) to apply %X/%X%s",
+					 waitingFor,
+					 (uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN,
+					 stallTimeMillis > 0 ? " (stalling)" : "");
+			set_ps_display(buffer, false);
+		}
+
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit()) /* Calls SyncRepCancelWait() if true. */
+			break;
+
+		/*
+		 * If are still waiting for peers, then we wait for any joining or
+		 * available peer to reach the LSN (or possibly stop being in one of
+		 * those states or go away).
+		 *
+		 * If not, there must be a non-zero stall time, so we wait for that to
+		 * elapse.
+		 */
+		if (waitingFor > 0)
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH, -1);
+		else
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH | WL_TIMEOUT,
+					  stallTimeMillis);
+	}
+
+	/* There is no way out of the loop that could leave us in the queue. */
+	Assert(SHMQueueIsDetached(&(MyProc->syncRepLinks)));
+	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
+	MyProc->waitLSN = 0;
+
+	if (update_process_title)
+		set_ps_display("", false); /* TODO: restore what was there */
+}
+
 /*
  * Wait for synchronous replication, if requested by user.
  *
@@ -180,57 +418,9 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 		if (syncRepState == SYNC_REP_WAIT_COMPLETE)
 			break;
 
-		/*
-		 * If a wait for synchronous replication is pending, we can neither
-		 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
-		 * lead the client to believe that that the transaction aborted, which
-		 * is not true: it's already committed locally. The former is no good
-		 * either: the client has requested synchronous replication, and is
-		 * entitled to assume that an acknowledged commit is also replicated,
-		 * which might not be true. So in this case we issue a WARNING (which
-		 * some clients may be able to interpret) and shut off further output.
-		 * We do NOT reset ProcDiePending, so that the process will die after
-		 * the commit is cleaned up.
-		 */
-		if (ProcDiePending)
-		{
-			ereport(WARNING,
-					(errcode(ERRCODE_ADMIN_SHUTDOWN),
-					 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * It's unclear what to do if a query cancel interrupt arrives.  We
-		 * can't actually abort at this point, but ignoring the interrupt
-		 * altogether is not helpful, so we just terminate the wait with a
-		 * suitable warning.
-		 */
-		if (QueryCancelPending)
-		{
-			QueryCancelPending = false;
-			ereport(WARNING,
-					(errmsg("canceling wait for synchronous replication due to user request"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * If the postmaster dies, we'll probably never get an
-		 * acknowledgement, because all the wal sender processes will exit. So
-		 * just bail out.
-		 */
-		if (!PostmasterIsAlive())
-		{
-			ProcDiePending = true;
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit())
 			break;
-		}
 
 		/*
 		 * Wait on latch.  Any condition that should wake us up will set the
@@ -403,6 +593,49 @@ SyncRepGetSynchronousStandby(void)
 }
 
 /*
+ * Check if the current WALSender process's application_name matches a name in
+ * causal_reads_standby_names (including '*' for wildcard).
+ */
+bool
+CausalReadsPotentialStandby(void)
+{
+	char *rawstring;
+	List	   *elemlist;
+	ListCell   *l;
+	bool		found = false;
+
+	/* Need a modifiable copy of string */
+	rawstring = pstrdup(causal_reads_standby_names);
+
+	/* Parse string into list of identifiers */
+	if (!SplitIdentifierString(rawstring, ',', &elemlist))
+	{
+		/* syntax error in list */
+		pfree(rawstring);
+		list_free(elemlist);
+		/* GUC machinery will have already complained - no need to do again */
+		return 0;
+	}
+
+	foreach(l, elemlist)
+	{
+		char	   *standby_name = (char *) lfirst(l);
+
+		if (pg_strcasecmp(standby_name, application_name) == 0 ||
+			pg_strcasecmp(standby_name, "*") == 0)
+		{
+			found = true;
+			break;
+		}
+	}
+
+	pfree(rawstring);
+	list_free(elemlist);
+
+	return found;
+}
+
+/*
  * Update the LSNs on each queue based upon our latest state. This
  * implements a simple policy of first-valid-standby-releases-waiter.
  *
@@ -410,22 +643,27 @@ SyncRepGetSynchronousStandby(void)
  * perhaps also which information we store as well.
  */
 void
-SyncRepReleaseWaiters(void)
+SyncRepReleaseWaiters(bool walsender_cr_available_or_joining)
 {
 	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
+	int			numapply = 0;
+	int			numcausalreadsapply = 0;
+	bool		is_highest_priority_sync_standby;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
-	 * potential standbys then we have nothing to do. If we are still starting
-	 * up, still running base backup or the current flush position is still
-	 * invalid, then leave quickly also.
+	 * potential standbys and not in a state that causal_reads waits for, then
+	 * we have nothing to do. If we are still starting up, still running base
+	 * backup or the current flush position is still invalid, then leave
+	 * quickly also.
 	 */
-	if (MyWalSnd->sync_standby_priority == 0 ||
-		MyWalSnd->state < WALSNDSTATE_STREAMING ||
-		XLogRecPtrIsInvalid(MyWalSnd->flush))
+	if (!walsender_cr_available_or_joining &&
+		(MyWalSnd->sync_standby_priority == 0 ||
+		 MyWalSnd->state < WALSNDSTATE_STREAMING ||
+		 XLogRecPtrIsInvalid(MyWalSnd->flush)))
 		return;
 
 	/*
@@ -435,45 +673,77 @@ SyncRepReleaseWaiters(void)
 	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
 	syncWalSnd = SyncRepGetSynchronousStandby();
 
-	/* We should have found ourselves at least */
-	Assert(syncWalSnd != NULL);
+	/*
+	 * If we aren't managing the highest priority standby then make a note of
+	 * that so we can announce a takeover in the log if we ever get that job.
+	 */
+	is_highest_priority_sync_standby = syncWalSnd == MyWalSnd;
+	if (!is_highest_priority_sync_standby)
+		announce_next_takeover = true;
 
 	/*
-	 * If we aren't managing the highest priority standby then just leave.
+	 * If we aren't managing the highest priority standby or a standby in
+	 * causal reads 'joining' or 'available' state, then just leave.
 	 */
-	if (syncWalSnd != MyWalSnd)
+	if (!is_highest_priority_sync_standby && !walsender_cr_available_or_joining)
 	{
 		LWLockRelease(SyncRepLock);
-		announce_next_takeover = true;
 		return;
 	}
 
 	/*
 	 * Set the lsn first so that when we wake backends they will release up to
-	 * this location.
+	 * this location.  For the single-standby synchronous commit levels, we
+	 * only do this if we are the current synchronous standby and we are
+	 * advancing the LSN further than it has been advanced before, so that
+	 * SyncRepWaitForLSN can skip waiting in some cases.
 	 */
-	if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+	if (is_highest_priority_sync_standby)
 	{
-		walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
-		numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
-		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
+		if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
+			numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE,
+										MyWalSnd->write);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->write)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
+			numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH,
+										MyWalSnd->flush);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+			numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY,
+										MyWalSnd->apply);
+		}
 	}
+	/*
+	 * For causal_reads, all walsenders currently in available or joining
+	 * state must reach the LSN on their own, and standbys will reach LSNs in
+	 * any order.  It doesn't make sense to keep the highest seen LSN in a
+	 * single walsndctl->lsn element.  (CausalReadsWaitForLSN has handling for
+	 * LSNs that have already been reached).
+	 */
+	if (walsender_cr_available_or_joining)
+		numcausalreadsapply =
+			SyncRepWakeQueue(false, SYNC_REP_WAIT_CAUSAL_READS_APPLY,
+							 MyWalSnd->apply);
 
 	LWLockRelease(SyncRepLock);
 
-	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X",
+	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%X, %d procs to causal_reads apply",
 		 numwrite, (uint32) (MyWalSnd->write >> 32), (uint32) MyWalSnd->write,
-	   numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush);
+		 numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush,
+		 numapply, (uint32) (MyWalSnd->apply >> 32), (uint32) MyWalSnd->apply,
+		 numcausalreadsapply);
 
 	/*
 	 * If we are managing the highest priority standby, though we weren't
 	 * prior to this, then announce we are now the sync standby.
 	 */
-	if (announce_next_takeover)
+	if (is_highest_priority_sync_standby && announce_next_takeover)
 	{
 		announce_next_takeover = false;
 		ereport(LOG,
@@ -548,9 +818,8 @@ SyncRepGetStandbyPriority(void)
  * Must hold SyncRepLock.
  */
 static int
-SyncRepWakeQueue(bool all, int mode)
+SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn)
 {
-	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	PGPROC	   *proc = NULL;
 	PGPROC	   *thisproc = NULL;
 	int			numprocs = 0;
@@ -567,7 +836,7 @@ SyncRepWakeQueue(bool all, int mode)
 		/*
 		 * Assume the queue is ordered by LSN
 		 */
-		if (!all && walsndctl->lsn[mode] < proc->waitLSN)
+		if (!all && lsn < proc->waitLSN)
 			return numprocs;
 
 		/*
@@ -627,7 +896,7 @@ SyncRepUpdateSyncStandbysDefined(void)
 			int			i;
 
 			for (i = 0; i < NUM_SYNC_REP_WAIT_MODE; i++)
-				SyncRepWakeQueue(true, i);
+				SyncRepWakeQueue(true, i, InvalidXLogRecPtr);
 		}
 
 		/*
@@ -679,13 +948,35 @@ SyncRepQueueIsOrderedByLSN(int mode)
 #endif
 
 /*
+ * Make sure that CausalReadsWaitForLSN can't return until after
+ * any 'causalReadsUntil' time that walsender could possibly have sent to any
+ * standby in a keepalive message.  This maintains the causal consistency
+ * guarantee: in causal_reads mode, we will not return control until any
+ * standby we have lost contact with has started generating 'standby not
+ * available for causal reads' errors.
+ *
+ * Also, wake up all backends waiting in CausalReadsWaitForLSN, because the
+ * set of available/joining peers has changed, and there is a new stall time
+ * they need to observe.
+ */
+void
+CausalReadsBeginStall(void)
+{
+	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+	WalSndCtl->stall_causal_reads_until =
+		TimestampTzPlusMilliseconds(GetCurrentTimestamp(), causal_reads_timeout);
+	SyncRepWakeQueue(true, SYNC_REP_WAIT_CAUSAL_READS_APPLY, InvalidXLogRecPtr);
+	LWLockRelease(SyncRepLock);
+}
+
+/*
  * ===========================================================
  * Synchronous Replication functions executed by any process
  * ===========================================================
  */
 
 bool
-check_synchronous_standby_names(char **newval, void **extra, GucSource source)
+check_standby_names(char **newval, void **extra, GucSource source)
 {
 	char	   *rawstring;
 	List	   *elemlist;
@@ -728,6 +1019,9 @@ assign_synchronous_commit(int newval, void *extra)
 		case SYNCHRONOUS_COMMIT_REMOTE_FLUSH:
 			SyncRepWaitMode = SYNC_REP_WAIT_FLUSH;
 			break;
+		case SYNCHRONOUS_COMMIT_REMOTE_APPLY:
+			SyncRepWaitMode = SYNC_REP_WAIT_APPLY;
+			break;
 		default:
 			SyncRepWaitMode = SYNC_REP_NO_WAIT;
 			break;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 183a3a5..94dd334 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -52,6 +52,7 @@
 #include "libpq/pqformat.h"
 #include "libpq/pqsignal.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "replication/walsender.h"
 #include "storage/ipc.h"
@@ -96,6 +97,7 @@ static uint32 recvOff = 0;
  */
 static volatile sig_atomic_t got_SIGHUP = false;
 static volatile sig_atomic_t got_SIGTERM = false;
+static volatile sig_atomic_t got_SIGUSR1 = false;
 
 /*
  * LogstreamResult indicates the byte positions that we have already
@@ -140,7 +142,8 @@ static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
 static void XLogWalRcvSendReply(bool force, bool requestReply);
 static void XLogWalRcvSendHSFeedback(bool immed);
-static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
+static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+								  TimestampTz *causalReadsUntil);
 
 /* Signal handlers */
 static void WalRcvSigHupHandler(SIGNAL_ARGS);
@@ -195,6 +198,7 @@ WalReceiverMain(void)
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz last_recv_timestamp;
 	bool		ping_sent;
+	bool		forceReply;
 
 	/*
 	 * WalRcv should be set up already (if we are a backend, we inherit this
@@ -246,6 +250,7 @@ WalReceiverMain(void)
 
 	/* Initialise to a sanish value */
 	walrcv->lastMsgSendTime = walrcv->lastMsgReceiptTime = walrcv->latestWalEndTime = GetCurrentTimestamp();
+	walrcv->causalReadsUntil = 0;
 
 	SpinLockRelease(&walrcv->mutex);
 
@@ -410,7 +415,7 @@ WalReceiverMain(void)
 					 * Process the received data, and any subsequent data we
 					 * can read without blocking.
 					 */
-					for (;;)
+					while (!got_SIGUSR1)
 					{
 						if (len > 0)
 						{
@@ -437,8 +442,16 @@ WalReceiverMain(void)
 						len = walrcv_receive(0, &buf);
 					}
 
+					if (got_SIGUSR1)
+					{
+						/* The recovery process asked us to force a reply. */
+						got_SIGUSR1 = false;
+						forceReply = true;
+					}
+
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(false, false);
+					XLogWalRcvSendReply(forceReply, false);
+					forceReply = false;
 
 					/*
 					 * If we've written some records, flush them to disk and
@@ -493,7 +506,15 @@ WalReceiverMain(void)
 						}
 					}
 
-					XLogWalRcvSendReply(requestReply, requestReply);
+					/* Check if the startup process has signaled us. */
+					if (got_SIGUSR1)
+					{
+						got_SIGUSR1 = false;
+						forceReply = true;
+					}
+
+					XLogWalRcvSendReply(requestReply || forceReply, requestReply);
+					forceReply = false;
 					XLogWalRcvSendHSFeedback(false);
 				}
 			}
@@ -730,6 +751,7 @@ WalRcvSigUsr1Handler(SIGNAL_ARGS)
 {
 	int			save_errno = errno;
 
+	got_SIGUSR1 = true;
 	latch_sigusr1_handler();
 
 	errno = save_errno;
@@ -795,6 +817,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 	XLogRecPtr	walEnd;
 	TimestampTz sendTime;
 	bool		replyRequested;
+	TimestampTz causalReadsUntil;
 
 	resetStringInfo(&incoming_message);
 
@@ -815,7 +838,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				walEnd = pq_getmsgint64(&incoming_message);
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, NULL);
 
 				buf += hdrlen;
 				len -= hdrlen;
@@ -825,7 +848,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 		case 'k':				/* Keepalive */
 			{
 				/* copy message to StringInfo */
-				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
+				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char) + sizeof(int64);
 				if (len != hdrlen)
 					ereport(ERROR,
 							(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -837,8 +860,12 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
 				replyRequested = pq_getmsgbyte(&incoming_message);
+				causalReadsUntil = IntegerTimestampToTimestampTz(
+					pq_getmsgint64(&incoming_message));
+				ProcessWalSndrMessage(walEnd, sendTime, &causalReadsUntil);
 
-				ProcessWalSndrMessage(walEnd, sendTime);
+				/* Remember primary's timestamp at this WAL location. */
+				SetXLogReplayTimestampAtLsn(sendTime, walEnd);
 
 				/* If the primary requested a reply, send one immediately */
 				if (replyRequested)
@@ -1032,6 +1059,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	XLogRecPtr	applyPtr;
 	static TimestampTz sendTime = 0;
 	TimestampTz now;
+	TimestampTz applyTimestamp = 0;
 
 	/*
 	 * If the user doesn't want status to be reported to the master, be sure
@@ -1063,7 +1091,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	/* Construct a new message */
 	writePtr = LogstreamResult.Write;
 	flushPtr = LogstreamResult.Flush;
-	applyPtr = GetXLogReplayRecPtr(NULL);
+	applyTimestamp = GetXLogReplayTimestamp(&applyPtr);
 
 	resetStringInfo(&reply_message);
 	pq_sendbyte(&reply_message, 'r');
@@ -1071,6 +1099,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	pq_sendint64(&reply_message, flushPtr);
 	pq_sendint64(&reply_message, applyPtr);
 	pq_sendint64(&reply_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&reply_message, TimestampTzToIntegerTimestamp(applyTimestamp));
 	pq_sendbyte(&reply_message, requestReply ? 1 : 0);
 
 	/* Send it */
@@ -1169,15 +1198,52 @@ XLogWalRcvSendHSFeedback(bool immed)
  * Update shared memory status upon receiving a message from primary.
  *
  * 'walEnd' and 'sendTime' are the end-of-WAL and timestamp of the latest
- * message, reported by primary.
+ * message, reported by primary.  'causalReadsUntil' is a pointer to
+ * the time the primary promises that this standby can safely claim to be
+ * causally consistent, to 0 if it cannot, or a NULL pointer for no change.
  */
 static void
-ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
+ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+					  TimestampTz *causalReadsUntil)
 {
 	WalRcvData *walrcv = WalRcv;
 
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
 
+	/* Sanity check for the causalReadsUntil time. */
+	if (causalReadsUntil != NULL && *causalReadsUntil != 0)
+	{
+		/* Deduce max_clock_skew from the causalReadsUntil and sendTime. */
+#ifdef HAVE_INT64_TIMESTAMP
+		int64 diffMillis = (*causalReadsUntil - sendTime) / 1000;
+#else
+		int64 diffMillis = (*causalReadsUntil - sendTime) * 1000;
+#endif
+		int64 max_clock_skew = diffMillis / (CAUSAL_READS_CLOCK_SKEW_RATIO - 1);
+
+		if (sendTime > TimestampTzPlusMilliseconds(lastMsgReceiptTime, max_clock_skew))
+		{
+			/*
+			 * The primary's clock is more than max_clock_skew + network
+			 * latency ahead of the standby's clock.  (If the primary's clock
+			 * is more than max_clock_skew ahead of the standby's clock, but
+			 * by less than the network latency, then there isn't much we can
+			 * do to detect that; but it still seems useful to have this basic
+			 * sanity check for wildly misconfigured servers.)
+			 */
+			elog(LOG, "the primary server's clock time is too far ahead");
+			causalReadsUntil = NULL;
+		}
+		/*
+		 * We could also try to detect cases where sendTime is more than
+		 * max_clock_skew in the past according to the standby's clock, but
+		 * that is indistinguishable from network latency/buffering, so we
+		 * could produce misleading error messages; if we do nothing, the
+		 * consequence is 'standby is not available for causal reads' errors
+		 * which should cause the user to investigate.
+		 */
+	}
+
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
 	if (walrcv->latestWalEnd < walEnd)
@@ -1185,6 +1251,8 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->latestWalEnd = walEnd;
 	walrcv->lastMsgSendTime = sendTime;
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
+	if (causalReadsUntil != NULL)
+		walrcv->causalReadsUntil = *causalReadsUntil;
 	SpinLockRelease(&walrcv->mutex);
 
 	if (log_min_messages <= DEBUG2)
@@ -1215,3 +1283,23 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 		pfree(receipttime);
 	}
 }
+
+/*
+ * Wake up the walreceiver if it happens to be blocked in walrcv_receive,
+ * and tell it that a commit record has been applied.
+ *
+ * This is called by the startup process whenever interesting xlog records
+ * are applied, so that walreceiver can check if it needs to send an apply
+ * notification back to the master which may be waiting in a COMMIT with
+ * synchronous_commit = apply or causal_reads = on.
+ *
+ * TODO: This may change -- Simon Riggs suggested latches for this.  Maybe
+ * pipes would work too (and avoid interrupting systems calls and allow for
+ * multiplexed IO with the replication socket).
+ */
+void
+WalRcvWakeup(void)
+{
+	if (WalRcv->pid != 0)
+		kill(WalRcv->pid, SIGUSR1);
+}
diff --git a/src/backend/replication/walreceiverfuncs.c b/src/backend/replication/walreceiverfuncs.c
index 4452f25..db9c397 100644
--- a/src/backend/replication/walreceiverfuncs.c
+++ b/src/backend/replication/walreceiverfuncs.c
@@ -25,9 +25,11 @@
 
 #include "access/xlog_internal.h"
 #include "postmaster/startup.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "storage/pmsignal.h"
 #include "storage/shmem.h"
+#include "utils/guc.h"
 #include "utils/timestamp.h"
 
 WalRcvData *WalRcv = NULL;
@@ -374,3 +376,23 @@ GetReplicationTransferLatency(void)
 
 	return ms;
 }
+
+/*
+ * Used by snapmgr to check if this standby has been authorized by the primary
+ * server to consider itself available for causal reads.  That is, to have
+ * applied all commits for which the COMMIT command has returned control on
+ * the primary server.
+ */
+bool
+WalRcvCausalReadsAvailable(void)
+{
+	WalRcvData *walrcv = WalRcv;
+	TimestampTz now = GetCurrentTimestamp();
+	bool result;
+
+	SpinLockAcquire(&walrcv->mutex);
+	result = walrcv->causalReadsUntil != 0 && now <= walrcv->causalReadsUntil;
+	SpinLockRelease(&walrcv->mutex);
+
+	return result;
+}
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 4a4643e..6c9f7ec 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -153,9 +153,17 @@ static StringInfoData tmpbuf;
  */
 static TimestampTz last_reply_timestamp = 0;
 
+static TimestampTz last_keepalive_timestamp = 0;
+
 /* Have we sent a heartbeat message asking for reply, since last reply? */
 static bool waiting_for_ping_response = false;
 
+/* How long do need to stay in JOINING state? */
+static TimestampTz causal_reads_joining_until = 0;
+
+/* Is this WALSender listed in causal_reads_standby_names? */
+static bool am_potential_causal_reads_standby = false;
+
 /*
  * While streaming WAL in Copy mode, streamingDoneSending is set to true
  * after we have sent CopyDone. We should not send any more CopyData messages
@@ -242,6 +250,57 @@ InitWalSender(void)
 }
 
 /*
+ * If we are exiting unexpectedly, we may need to communicate with concurrent
+ * causal_reads commits to maintain the causal consistency guarantee.
+ */
+static void
+PrepareUncleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * We've lost contact with the standby, but it may still be alive.  We
+		 * can't let any causal_reads transactions return until we've stalled
+		 * for long enough for a zombie standby to start raising errors due
+		 * to lack of keepalives with early enough timestamps.
+		 */
+		elog(LOG, "standby \"%s\" is lost (no longer available for causal reads)", application_name);
+		CausalReadsBeginStall();
+
+		/*
+		 * We set the state to a lower level _after_ beginning the stall,
+		 * otherwise there would be a tiny window where commits could return
+		 * without observing the stall.
+		 */
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
+ * We are shutting down because we received a goodbye message from the
+ * walreceiver.
+ */
+static void
+PrepareCleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * The standby is shutting down, so it won't be running any more
+		 * transations.  It is therefore safe to stop waiting for it, and no
+		 * stall is necessary.
+		 */
+		elog(LOG, "standby \"%s\" is leaving (no longer available for causal reads)", application_name);
+
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
  * Clean up after an error.
  *
  * WAL sender processes don't use transactions like regular backends do.
@@ -264,7 +323,10 @@ WalSndErrorCleanup(void)
 
 	replication_active = false;
 	if (walsender_ready_to_stop)
+	{
+		PrepareUncleanExit();
 		proc_exit(0);
+	}
 
 	/* Revert back to startup state */
 	WalSndSetState(WALSNDSTATE_STARTUP);
@@ -276,6 +338,8 @@ WalSndErrorCleanup(void)
 static void
 WalSndShutdown(void)
 {
+	PrepareUncleanExit();
+
 	/*
 	 * Reset whereToSendOutput to prevent ereport from attempting to send any
 	 * more messages to the standby.
@@ -1386,6 +1450,7 @@ ProcessRepliesIfAny(void)
 		if (r < 0)
 		{
 			/* unexpected error or EOF */
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1402,6 +1467,7 @@ ProcessRepliesIfAny(void)
 		resetStringInfo(&reply_message);
 		if (pq_getmessage(&reply_message, 0))
 		{
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1451,6 +1517,7 @@ ProcessRepliesIfAny(void)
 				 * 'X' means that the standby is closing down the socket.
 				 */
 			case 'X':
+				PrepareCleanExit();
 				proc_exit(0);
 
 			default:
@@ -1543,15 +1610,29 @@ ProcessStandbyReplyMessage(void)
 	XLogRecPtr	writePtr,
 				flushPtr,
 				applyPtr;
+	int			applyLagMs;
 	bool		replyRequested;
+	TimestampTz now = GetCurrentTimestamp();
+	TimestampTz applyTimestamp;
 
 	/* the caller already consumed the msgtype byte */
 	writePtr = pq_getmsgint64(&reply_message);
 	flushPtr = pq_getmsgint64(&reply_message);
 	applyPtr = pq_getmsgint64(&reply_message);
 	(void) pq_getmsgint64(&reply_message);		/* sendTime; not used ATM */
+	applyTimestamp = IntegerTimestampToTimestampTz(pq_getmsgint64(&reply_message));
 	replyRequested = pq_getmsgbyte(&reply_message);
 
+	/* Compute the apply lag in milliseconds. */
+	if (applyTimestamp == 0)
+		applyLagMs = -1;
+	else
+#ifdef HAVE_INT64_TIMESTAMP
+		applyLagMs = (now - applyTimestamp) / 1000;
+#else
+		applyLagMs = (now - applyTimestamp) * 1000.0;
+#endif
+
 	elog(DEBUG2, "write %X/%X flush %X/%X apply %X/%X%s",
 		 (uint32) (writePtr >> 32), (uint32) writePtr,
 		 (uint32) (flushPtr >> 32), (uint32) flushPtr,
@@ -1568,16 +1649,84 @@ ProcessStandbyReplyMessage(void)
 	 */
 	{
 		WalSnd *walsnd = MyWalSnd;
+		WalSndCausalReadsState causal_reads_state = walsnd->causal_reads_state;
+		bool causal_reads_state_changed = false;
+
+		/*
+		 * Handle causal reads state transitions, if a causal_reads_timeout is
+		 * configured, this standby is listed in causal_reads_standby_names,
+		 * and we are a primary database (not a cascading standby).
+		 */
+		if (causal_reads_timeout != 0 &&
+			am_potential_causal_reads_standby &&
+			!am_cascading_walsender)
+		{
+			if (applyLagMs >= 0 && applyLagMs < causal_reads_timeout)
+			{
+				if (causal_reads_state == WALSNDCRSTATE_UNAVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_JOINING;
+					causal_reads_joining_until =
+						TimestampTzPlusMilliseconds(now, causal_reads_timeout);
+					causal_reads_state_changed = true;
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING &&
+						 now >= causal_reads_joining_until)
+				{
+					causal_reads_state = WALSNDCRSTATE_AVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+			else
+			{
+				if (causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+					/*
+					 * We are dropping a causal reads available standby, so we
+					 * mustn't let any commit command that is waiting in
+					 * CausalReadsWaitForLSN return until we are sure that the
+					 * standby definitely knows that it's not available and
+					 * starts raising errors for causal_reads transactions.
+					 */
+					CausalReadsBeginStall();
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING)
+				{
+					/*
+					 * Dropping a joining standby doesn't require a stall,
+					 * because the standby doesn't think it's available, so
+					 * it's already raising the error for causal_reads
+					 * transactions.
+					 */
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+		}
 
 		SpinLockAcquire(&walsnd->mutex);
 		walsnd->write = writePtr;
 		walsnd->flush = flushPtr;
 		walsnd->apply = applyPtr;
+		walsnd->applyLagMs = applyLagMs;
+		walsnd->causal_reads_state = causal_reads_state;
 		SpinLockRelease(&walsnd->mutex);
+
+		if (causal_reads_state_changed)
+		{
+			WalSndKeepalive(true);
+			elog(LOG, "standby \"%s\" is %s", application_name,
+				 causal_reads_state == WALSNDCRSTATE_UNAVAILABLE ? "unavailable for causal reads" :
+				 causal_reads_state == WALSNDCRSTATE_JOINING ? "joining as a causal reads standby..." :
+				 causal_reads_state == WALSNDCRSTATE_AVAILABLE ? "available for causal reads" :
+				 "UNKNOWN");
+		}
 	}
 
 	if (!am_cascading_walsender)
-		SyncRepReleaseWaiters();
+		SyncRepReleaseWaiters(MyWalSnd->causal_reads_state >= WALSNDCRSTATE_JOINING);
 
 	/*
 	 * Advance our local xmin horizon when the client confirmed a flush.
@@ -1724,27 +1873,34 @@ WalSndComputeSleeptime(TimestampTz now)
 {
 	long		sleeptime = 10000;		/* 10 s */
 
-	if (wal_sender_timeout > 0 && last_reply_timestamp > 0)
+	if ((wal_sender_timeout > 0 || causal_reads_timeout > 0) && last_reply_timestamp > 0)
 	{
 		TimestampTz wakeup_time;
 		long		sec_to_timeout;
 		int			microsec_to_timeout;
 
-		/*
-		 * At the latest stop sleeping once wal_sender_timeout has been
-		 * reached.
-		 */
-		wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-												  wal_sender_timeout);
-
-		/*
-		 * If no ping has been sent yet, wakeup when it's time to do so.
-		 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
-		 * the timeout passed without a response.
-		 */
-		if (!waiting_for_ping_response)
+		if (causal_reads_timeout != 0)
+			wakeup_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+													  causal_reads_timeout /
+													  CAUSAL_READS_KEEPALIVE_RATIO);
+		else
+		{
+			/*
+			 * At the latest stop sleeping once wal_sender_timeout has been
+			 * reached.
+			 */
 			wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-													  wal_sender_timeout / 2);
+													  wal_sender_timeout);
+
+			/*
+			 * If no ping has been sent yet, wakeup when it's time to do so.
+			 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
+			 * the timeout passed without a response.
+			 */
+			if (!waiting_for_ping_response)
+				wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+														  wal_sender_timeout / 2);
+		}
 
 		/* Compute relative time until wakeup. */
 		TimestampDifference(now, wakeup_time,
@@ -1765,15 +1921,28 @@ static void
 WalSndCheckTimeOut(TimestampTz now)
 {
 	TimestampTz timeout;
+	int allowed_time;
 
 	/* don't bail out if we're doing something that doesn't require timeouts */
 	if (last_reply_timestamp <= 0)
 		return;
 
+	/*
+	 * If a causal_reads_timeout is configured, it is used instead of
+	 * wal_sender_timeout.  Ideally we'd use causal_reads_timeout / 2 +
+	 * allowance for network latency, but since walreceiver can become quite
+	 * bogged down fsyncing WAL we allow more tolerance.  (This could be
+	 * tightened up once standbys hand writing off to the WAL writer).
+	 */
+	if (causal_reads_timeout != 0)
+		allowed_time = causal_reads_timeout;
+	else
+		allowed_time = wal_sender_timeout;
+
 	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
-										  wal_sender_timeout);
+										  allowed_time);
 
-	if (wal_sender_timeout > 0 && now >= timeout)
+	if (allowed_time > 0 && now >= timeout)
 	{
 		/*
 		 * Since typically expiration of replication timeout means
@@ -1806,6 +1975,9 @@ WalSndLoop(WalSndSendDataCallback send_data)
 	last_reply_timestamp = GetCurrentTimestamp();
 	waiting_for_ping_response = false;
 
+	/* Check if we are managing potential causal_reads standby. */
+	am_potential_causal_reads_standby = CausalReadsPotentialStandby();
+
 	/*
 	 * Loop until we reach the end of this timeline or the client requests to
 	 * stop streaming.
@@ -1963,6 +2135,7 @@ InitWalSenderSlot(void)
 			walsnd->pid = MyProcPid;
 			walsnd->sentPtr = InvalidXLogRecPtr;
 			walsnd->state = WALSNDSTATE_STARTUP;
+			walsnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
 			walsnd->latch = &MyProc->procLatch;
 			SpinLockRelease(&walsnd->mutex);
 			/* don't need the lock anymore */
@@ -2732,6 +2905,24 @@ WalSndGetStateString(WalSndState state)
 	return "UNKNOWN";
 }
 
+/*
+ * Return a string constant representing the causal reads state. This is used
+ * in system views, and should *not* be translated.
+ */
+static const char *
+WalSndGetCausalReadsStateString(WalSndCausalReadsState causal_reads_state)
+{
+	switch (causal_reads_state)
+	{
+		case WALSNDCRSTATE_UNAVAILABLE:
+			return "unavailable";
+		case WALSNDCRSTATE_JOINING:
+			return "joining";
+		case WALSNDCRSTATE_AVAILABLE:
+			return "available";
+	}
+	return "UNKNOWN";
+}
 
 /*
  * Returns activity of walsenders, including pids and xlog locations sent to
@@ -2740,7 +2931,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	8
+#define PG_STAT_GET_WAL_SENDERS_COLS	10
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2788,8 +2979,10 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		XLogRecPtr	write;
 		XLogRecPtr	flush;
 		XLogRecPtr	apply;
+		int			applyLagMs;
 		int			priority;
 		WalSndState state;
+		WalSndCausalReadsState causalReadsState;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
 		bool		nulls[PG_STAT_GET_WAL_SENDERS_COLS];
 
@@ -2799,9 +2992,11 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		SpinLockAcquire(&walsnd->mutex);
 		sentPtr = walsnd->sentPtr;
 		state = walsnd->state;
+		causalReadsState = walsnd->causal_reads_state;
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
+		applyLagMs = walsnd->applyLagMs;
 		priority = walsnd->sync_standby_priority;
 		SpinLockRelease(&walsnd->mutex);
 
@@ -2833,6 +3028,23 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				nulls[5] = true;
 			values[5] = LSNGetDatum(apply);
 
+			if (applyLagMs < 0)
+				nulls[6] = true;
+			else
+			{
+				Interval *applyLagInterval = palloc(sizeof(Interval));
+
+				applyLagInterval->month = 0;
+				applyLagInterval->day = 0;
+#ifdef HAVE_INT64_TIMESTAMP
+				applyLagInterval->time = applyLagMs * 1000;
+#else
+				applyLagInterval->time = applyLagMs / 1000.0;
+#endif
+				nulls[6] = false;
+				values[6] = IntervalPGetDatum(applyLagInterval);
+			}
+
 			/*
 			 * Treat a standby such as a pg_basebackup background process
 			 * which always returns an invalid flush location, as an
@@ -2840,18 +3052,21 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 			 */
 			priority = XLogRecPtrIsInvalid(walsnd->flush) ? 0 : priority;
 
-			values[6] = Int32GetDatum(priority);
+			values[7] = Int32GetDatum(priority);
 
 			/*
 			 * More easily understood version of standby state. This is purely
 			 * informational, not different from priority.
 			 */
 			if (priority == 0)
-				values[7] = CStringGetTextDatum("async");
+				values[8] = CStringGetTextDatum("async");
 			else if (walsnd == sync_standby)
-				values[7] = CStringGetTextDatum("sync");
+				values[8] = CStringGetTextDatum("sync");
 			else
-				values[7] = CStringGetTextDatum("potential");
+				values[8] = CStringGetTextDatum("potential");
+
+			values[9] =
+				CStringGetTextDatum(WalSndGetCausalReadsStateString(causalReadsState));
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
@@ -2871,14 +3086,50 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 static void
 WalSndKeepalive(bool requestReply)
 {
+	TimestampTz now;
+	TimestampTz causal_reads_until;
+
 	elog(DEBUG2, "sending replication keepalive");
 
+	/*
+	 * If the walsender currently deems the standby to be available for causal
+	 * reads, then we authorize the standby to consider itself avialable until
+	 * a certain time in the future.  If we lose contact with the standby or
+	 * drop it from the set of standbys we wait for in causal_reads mode
+	 * because of excessive lag, then we'll stall until after that time to
+	 * maintain our causal consistency guarantee.
+	 */
+	now = GetCurrentTimestamp();
+	if (MyWalSnd->causal_reads_state < WALSNDCRSTATE_AVAILABLE)
+		causal_reads_until = 0; /* Not available */
+	else
+	{
+		/*
+		 * Since this timestamp is being sent to the standby where it will be
+		 * compared against a time generated by the standby's system clock, we
+		 * must consider clock skew.  First, we decide on a maximum tolerable
+		 * difference between system clocks.  If the primary's clock is ahead
+		 * of the standby's by more than this, then all bets are off (the
+		 * standby could falsely believe it is available).  If the primary's
+		 * clock is behind the standby's by more than this, then the standby
+		 * will err the other way and generate spurious errors in
+		 * causal_reads mode.  Rather than having a separate GUC for this,
+		 * we derive it from causal_reads_timeout.
+		 */
+		int max_clock_skew = causal_reads_timeout / CAUSAL_READS_CLOCK_SKEW_RATIO;
+
+		causal_reads_until =
+			TimestampTzPlusMilliseconds(now,
+										causal_reads_timeout - max_clock_skew);
+	}
+
 	/* construct the message... */
 	resetStringInfo(&output_message);
 	pq_sendbyte(&output_message, 'k');
 	pq_sendint64(&output_message, sentPtr);
-	pq_sendint64(&output_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(now));
 	pq_sendbyte(&output_message, requestReply ? 1 : 0);
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(causal_reads_until));
 
 	/* ... and send it wrapped in CopyData */
 	pq_putmessage_noblock('d', output_message.data, output_message.len);
@@ -2896,23 +3147,32 @@ WalSndKeepaliveIfNecessary(TimestampTz now)
 	 * Don't send keepalive messages if timeouts are globally disabled or
 	 * we're doing something not partaking in timeouts.
 	 */
-	if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
+	if ((wal_sender_timeout <= 0 && causal_reads_timeout == 0) || last_reply_timestamp <= 0)
 		return;
 
-	if (waiting_for_ping_response)
+	if (waiting_for_ping_response && causal_reads_timeout == 0)
 		return;
 
 	/*
 	 * If half of wal_sender_timeout has lapsed without receiving any reply
 	 * from the standby, send a keep-alive message to the standby requesting
 	 * an immediate reply.
+	 *
+	 * If causal_reads_timeout has been configured, use it to control
+	 * keepalive intervals rather than wal_sender_timeout.
 	 */
-	ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-											wal_sender_timeout / 2);
+	if (causal_reads_timeout != 0)
+		ping_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+												causal_reads_timeout /
+												CAUSAL_READS_KEEPALIVE_RATIO);
+	else
+		ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+												wal_sender_timeout / 2);
 	if (now >= ping_time)
 	{
 		WalSndKeepalive(true);
 		waiting_for_ping_response = true;
+		last_keepalive_timestamp = now;
 
 		/* Try to flush pending output to the client */
 		if (pq_flush_if_writable() != 0)
diff --git a/src/backend/utils/adt/timestamp.c b/src/backend/utils/adt/timestamp.c
index 8fbb310..12c8b88 100644
--- a/src/backend/utils/adt/timestamp.c
+++ b/src/backend/utils/adt/timestamp.c
@@ -1611,6 +1611,20 @@ IntegerTimestampToTimestampTz(int64 timestamp)
 #endif
 
 /*
+ * TimestampTzToIntegerTimestamp -- convert a native timestamp to int64 format
+ *
+ * When compiled with --enable-integer-datetimes, this is implemented as a
+ * no-op macro.
+ */
+#ifndef HAVE_INT64_TIMESTAMP
+int64
+TimestampTzToIntegerTimestamp(TimestampTz timestamp)
+{
+	return timestamp * 1000000;
+}
+#endif
+
+/*
  * TimestampDifference -- convert the difference between two timestamps
  *		into integer seconds and microseconds
  *
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index fda0fb9..10b1ed1 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -351,6 +351,7 @@ static const struct config_enum_entry constraint_exclusion_options[] = {
 static const struct config_enum_entry synchronous_commit_options[] = {
 	{"local", SYNCHRONOUS_COMMIT_LOCAL_FLUSH, false},
 	{"remote_write", SYNCHRONOUS_COMMIT_REMOTE_WRITE, false},
+	{"apply", SYNCHRONOUS_COMMIT_REMOTE_APPLY, false},
 	{"on", SYNCHRONOUS_COMMIT_ON, false},
 	{"off", SYNCHRONOUS_COMMIT_OFF, false},
 	{"true", SYNCHRONOUS_COMMIT_ON, true},
@@ -1618,6 +1619,16 @@ static struct config_bool ConfigureNamesBool[] =
 		NULL, NULL, NULL
 	},
 
+	{
+		{"causal_reads", PGC_USERSET, REPLICATION_STANDBY,
+		 gettext_noop("Enables causal reads."),
+		 NULL
+		},
+		&causal_reads,
+		false,
+		NULL, NULL, NULL
+	},
+
 	/* End-of-list marker */
 	{
 		{NULL, 0, 0, NULL, NULL}, NULL, false, NULL, NULL, NULL
@@ -1776,6 +1787,17 @@ static struct config_int ConfigureNamesInt[] =
 	},
 
 	{
+		{"causal_reads_timeout", PGC_SIGHUP, REPLICATION_STANDBY,
+			gettext_noop("Sets the maximum apply lag before causal reads standbys are no longer available."),
+			NULL,
+			GUC_UNIT_MS
+		},
+		&causal_reads_timeout,
+		0, 0, INT_MAX,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"max_connections", PGC_POSTMASTER, CONN_AUTH_SETTINGS,
 			gettext_noop("Sets the maximum number of concurrent connections."),
 			NULL
@@ -3361,7 +3383,18 @@ static struct config_string ConfigureNamesString[] =
 		},
 		&SyncRepStandbyNames,
 		"",
-		check_synchronous_standby_names, NULL, NULL
+		check_standby_names, NULL, NULL
+	},
+
+	{
+		{"causal_reads_standby_names", PGC_SIGHUP, REPLICATION_MASTER,
+			gettext_noop("List of names of potential causal reads standbys."),
+			NULL,
+			GUC_LIST_INPUT
+		},
+		&causal_reads_standby_names,
+		"*",
+		check_standby_names, NULL, NULL
 	},
 
 	{
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index 074935c..a466732 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -46,8 +46,11 @@
 
 #include "access/transam.h"
 #include "access/xact.h"
+#include "access/xlog.h"
 #include "lib/pairingheap.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
+#include "replication/walreceiver.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
 #include "storage/procarray.h"
@@ -209,6 +212,16 @@ GetTransactionSnapshot(void)
 				 "cannot take query snapshot during a parallel operation");
 
 		/*
+		 * In causal_reads mode on a standby, check if we have definitely
+		 * applied WAL for any COMMIT that returned successfully on the
+		 * primary.
+		 *
+		 * TODO: Machine readable error code?
+		 */
+		if (causal_reads && RecoveryInProgress() && !WalRcvCausalReadsAvailable())
+			elog(ERROR, "standby is not available for causal reads");
+
+		/*
 		 * In transaction-snapshot mode, the first snapshot must live until
 		 * end of xact regardless of what the caller does with it, so we must
 		 * make a copy of it rather than returning CurrentSnapshotData
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index cb1c2db..0f08ff5 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -60,7 +60,11 @@ typedef enum
 	SYNCHRONOUS_COMMIT_LOCAL_FLUSH,		/* wait for local flush only */
 	SYNCHRONOUS_COMMIT_REMOTE_WRITE,	/* wait for local flush and remote
 										 * write */
-	SYNCHRONOUS_COMMIT_REMOTE_FLUSH		/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_FLUSH,	/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_APPLY,	/* wait for local flush and remote
+										 * apply */
+	SYNCHRONOUS_COMMIT_CONSISTENT_APPLY /* wait for local flusha and remote
+										   apply with causal consistency */
 }	SyncCommitLevel;
 
 /* Define the default setting for synchonous_commit */
@@ -144,10 +148,13 @@ typedef void (*SubXactCallback) (SubXactEvent event, SubTransactionId mySubid,
  * EOXact... routines which run at the end of the original transaction
  * completion.
  */
+#define XACT_COMPLETION_SYNC_APPLY_FEEDBACK		(1U << 29)
 #define XACT_COMPLETION_UPDATE_RELCACHE_FILE	(1U << 30)
 #define XACT_COMPLETION_FORCE_SYNC_COMMIT		(1U << 31)
 
 /* Access macros for above flags */
+#define XactCompletionSyncApplyFeedback(xinfo) \
+	(!!(xinfo & XACT_COMPLETION_SYNC_APPLY_FEEDBACK))
 #define XactCompletionRelcacheInitFileInval(xinfo) \
 	(!!(xinfo & XACT_COMPLETION_UPDATE_RELCACHE_FILE))
 #define XactCompletionForceSyncCommit(xinfo) \
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 790ca66..8aeda11 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -235,6 +235,9 @@ extern void GetXLogReceiptTime(TimestampTz *rtime, bool *fromStream);
 extern XLogRecPtr GetXLogReplayRecPtr(TimeLineID *replayTLI);
 extern XLogRecPtr GetXLogInsertRecPtr(void);
 extern XLogRecPtr GetXLogWriteRecPtr(void);
+extern void SetXLogReplayTimestamp(TimestampTz timestamp);
+extern void SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn);
+extern TimestampTz GetXLogReplayTimestamp(XLogRecPtr *lsn);
 extern bool RecoveryIsPaused(void);
 extern void SetRecoveryPause(bool recoveryPause);
 extern TimestampTz GetLatestXTime(void);
@@ -267,6 +270,8 @@ extern bool CheckPromoteSignal(void);
 extern void WakeupRecovery(void);
 extern void SetWalWriterSleeping(bool sleeping);
 
+extern void XLogRequestWalReceiverReply(void);
+
 extern void assign_max_wal_size(int newval, void *extra);
 extern void assign_checkpoint_completion_target(double newval, void *extra);
 
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index d8640db..acb6796 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2783,7 +2783,7 @@ DATA(insert OID = 1936 (  pg_stat_get_backend_idset		PGNSP PGUID 12 1 100 0 0 f
 DESCR("statistics: currently active backend IDs");
 DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f f f t s r 1 0 2249 "23" "{23,26,23,26,25,25,25,16,1184,1184,1184,1184,869,25,23,28,28,16,25,25,23,16,25}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{pid,datid,pid,usesysid,application_name,state,query,waiting,xact_start,query_start,backend_start,state_change,client_addr,client_hostname,client_port,backend_xid,backend_xmin,ssl,sslversion,sslcipher,sslbits,sslcompression,sslclientdn}" _null_ _null_ pg_stat_get_activity _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active backends");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,23,25}" "{o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25,25}" "{o,o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state,causal_reads_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 2026 (  pg_backend_pid				PGNSP PGUID 12 1 0 0 0 f f f f t f s r 0 0 23 "" _null_ _null_ _null_ _null_ _null_ pg_backend_pid _null_ _null_ _null_ ));
 DESCR("statistics: current backend PID");
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index 71e2857..5b92c4f 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -23,14 +23,34 @@
 #define SYNC_REP_NO_WAIT		-1
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
+#define SYNC_REP_WAIT_APPLY		2
+#define SYNC_REP_WAIT_CAUSAL_READS_APPLY 3
 
-#define NUM_SYNC_REP_WAIT_MODE	2
+#define NUM_SYNC_REP_WAIT_MODE	4
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
 #define SYNC_REP_WAITING			1
 #define SYNC_REP_WAIT_COMPLETE		2
 
+/*
+ * ratio of causal_read_timeout to max_clock_skew (4 means than the maximum
+ * tolerated clock difference between primary and standbys using causal_reads
+ * is 1/4 of causal_reads_timeout)
+ */
+#define CAUSAL_READS_CLOCK_SKEW_RATIO 4
+
+/*
+ * ratio of causal_reads_timeout to keepalive time (2 means that the effective
+ * keepalive time is 1/2 of the causal_reads_timeout GUC when it is non-zero)
+ */
+#define CAUSAL_READS_KEEPALIVE_RATIO 2
+
+/* GUC variables */
+extern int causal_reads_timeout;
+extern bool causal_reads;
+extern char *causal_reads_standby_names;
+
 /* user-settable parameters for synchronous replication */
 extern char *SyncRepStandbyNames;
 
@@ -42,16 +62,23 @@ extern void SyncRepCleanupAtProcExit(void);
 
 /* called by wal sender */
 extern void SyncRepInitConfig(void);
-extern void SyncRepReleaseWaiters(void);
+extern void SyncRepReleaseWaiters(bool walsender_cr_available_or_joining);
 
 /* called by checkpointer */
 extern void SyncRepUpdateSyncStandbysDefined(void);
 
+/* called by user backend (xact.c) */
+extern void CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN);
+
+/* called by wal sender */
+extern void CausalReadsBeginStall(void);
+extern bool CausalReadsPotentialStandby(void);
+
 /* forward declaration to avoid pulling in walsender_private.h */
 struct WalSnd;
 extern struct WalSnd *SyncRepGetSynchronousStandby(void);
 
-extern bool check_synchronous_standby_names(char **newval, void **extra, GucSource source);
+extern bool check_standby_names(char **newval, void **extra, GucSource source);
 extern void assign_synchronous_commit(int newval, void *extra);
 
 #endif   /* _SYNCREP_H */
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 61255a9..507af9f 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -79,6 +79,13 @@ typedef struct
 	TimeLineID	receivedTLI;
 
 	/*
+	 * causallyReadsUntil is the time until which the primary has authorized
+	 * this standby to consider itself avialable for causal_reads mode, or 0
+	 * for not authorized.
+	 */
+	TimestampTz causalReadsUntil;
+
+	/*
 	 * latestChunkStart is the starting byte position of the current "batch"
 	 * of received WAL.  It's actually the same as the previous value of
 	 * receivedUpto before the last flush to disk.  Startup process can use
@@ -160,5 +167,8 @@ extern void RequestXLogStreaming(TimeLineID tli, XLogRecPtr recptr,
 extern XLogRecPtr GetWalRcvWriteRecPtr(XLogRecPtr *latestChunkStart, TimeLineID *receiveTLI);
 extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
+extern void WalRcvWakeup(void);
+
+extern bool WalRcvCausalReadsAvailable(void);
 
 #endif   /* _WALRECEIVER_H */
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 6dae480..deeb277 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -27,6 +27,13 @@ typedef enum WalSndState
 	WALSNDSTATE_STREAMING
 } WalSndState;
 
+typedef enum WalSndCausalReadsState
+{
+	WALSNDCRSTATE_UNAVAILABLE = 0,
+	WALSNDCRSTATE_JOINING,
+	WALSNDCRSTATE_AVAILABLE
+} WalSndCausalReadsState;
+
 /*
  * Each walsender has a WalSnd struct in shared memory.
  */
@@ -34,6 +41,7 @@ typedef struct WalSnd
 {
 	pid_t		pid;			/* this walsender's process id, or 0 */
 	WalSndState state;			/* this walsender's state */
+	WalSndCausalReadsState causal_reads_state; /* the walsender's causal reads state */
 	XLogRecPtr	sentPtr;		/* WAL has been sent up to this point */
 	bool		needreload;		/* does currently-open file need to be
 								 * reloaded? */
@@ -46,6 +54,7 @@ typedef struct WalSnd
 	XLogRecPtr	write;
 	XLogRecPtr	flush;
 	XLogRecPtr	apply;
+	int			applyLagMs;
 
 	/* Protects shared variables shown above. */
 	slock_t		mutex;
@@ -88,6 +97,14 @@ typedef struct
 	 */
 	bool		sync_standbys_defined;
 
+	/*
+	 * Until when must commits in causal_reads stall?  This is set to a time
+	 * in the future whenever a standby is dropped from the set of consistent
+	 * standbys, to give standbys time to know that they are not able to
+	 * provide causal consistency guarantees.
+	 */
+	TimestampTz	stall_causal_reads_until;
+
 	WalSnd		walsnds[FLEXIBLE_ARRAY_MEMBER];
 } WalSndCtlData;
 
diff --git a/src/include/utils/timestamp.h b/src/include/utils/timestamp.h
index 530fef1..0f4b166 100644
--- a/src/include/utils/timestamp.h
+++ b/src/include/utils/timestamp.h
@@ -227,9 +227,11 @@ extern bool TimestampDifferenceExceeds(TimestampTz start_time,
 #ifndef HAVE_INT64_TIMESTAMP
 extern int64 GetCurrentIntegerTimestamp(void);
 extern TimestampTz IntegerTimestampToTimestampTz(int64 timestamp);
+extern int64 TimestampTzToIntegerTimestamp(TimestampTz timestamp);
 #else
 #define GetCurrentIntegerTimestamp()	GetCurrentTimestamp()
 #define IntegerTimestampToTimestampTz(timestamp) (timestamp)
+#define TimestampTzToIntegerTimestamp(timestamp) (timestamp)
 #endif
 
 extern TimestampTz time_t_to_timestamptz(pg_time_t tm);
#10Simon Riggs
simon@2ndQuadrant.com
In reply to: Thomas Munro (#5)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On 11 November 2015 at 09:22, Thomas Munro <thomas.munro@enterprisedb.com>
wrote:

1. Reader waits with exposed LSNs, as Heikki suggests. This is what
BerkeleyDB does in "read-your-writes" mode. It means that application
developers have the responsibility for correctly identifying transactions
with causal dependencies and dealing with LSNs (or whatever equivalent
tokens), potentially even passing them to other processes where the
transactions are causally dependent but run by multiple communicating
clients (for example, communicating microservices). This makes it
difficult to retrofit load balancing to pre-existing applications and (like
anything involving concurrency) difficult to reason about as applications
grow in size and complexity. It is efficient if done correctly, but it is
a tax on application complexity.

Agreed. This works if you have a single transaction connected thru a pool
that does statement-level load balancing, so it works in both session and
transaction mode.

I was in favour of a scheme like this myself, earlier, but have more
thoughts now.

We must also consider the need for serialization across sessions or
transactions.

In transaction pooling mode, an application could get assigned a different
session, so a token would be much harder to pass around.

2. Reader waits for a conservatively chosen LSN. This is roughly what

MySQL derivatives do in their "causal_reads = on" and "wsrep_sync_wait =
1" modes. Read transactions would start off by finding the current end
of WAL on the primary, since that must be later than any commit that
already completed, and then waiting for that to apply locally. That means
every read transaction waits for a complete replication lag period,
potentially unnecessarily. This is tax on readers with unnecessary waiting.

This tries to make it easier for users by forcing all users to experience a
causality delay. Given the whole purpose of multi-node load balancing is
performance, referencing the master again simply defeats any performance
gain, so you couldn't ever use it for all sessions. It could be a USERSET
parameter, so could be turned off in most cases that didn't need it. But
its easier to use than (1).

Though this should be implemented in the pooler.

3. Writer waits, as proposed. In this model, there is no tax on readers

(they have zero overhead, aside from the added complexity of dealing with
the possibility of transactions being rejected when a standby falls behind
and is dropped from 'available' status; but database clients must already
deal with certain types of rare rejected queries/failures such as
deadlocks, serialization failures, server restarts etc). This is a tax on
writers.

This would seem to require that all readers must first check with the
master as to which standbys are now considered available, so it looks like
(2).

The alternative is that we simply send readers to any standby and allow the
pool to work out separately whether the standby is still available, which
mostly works, but it doesn't handle sporadic slow downs on particular
standbys very well (if at all).

I think we need to look at whether this does actually give us anything, or
whether we are missing the underlying Heisenberg reality.

More later.

--
Simon Riggs http://www.2ndQuadrant.com/
<http://www.2ndquadrant.com/&gt;
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#11Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Simon Riggs (#10)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Fri, Nov 13, 2015 at 1:16 AM, Simon Riggs <simon@2ndquadrant.com> wrote:

On 11 November 2015 at 09:22, Thomas Munro <thomas.munro@enterprisedb.com>
wrote:

1. Reader waits with exposed LSNs, as Heikki suggests. This is what
BerkeleyDB does in "read-your-writes" mode. It means that application
developers have the responsibility for correctly identifying transactions
with causal dependencies and dealing with LSNs (or whatever equivalent
tokens), potentially even passing them to other processes where the
transactions are causally dependent but run by multiple communicating
clients (for example, communicating microservices). This makes it
difficult to retrofit load balancing to pre-existing applications and (like
anything involving concurrency) difficult to reason about as applications
grow in size and complexity. It is efficient if done correctly, but it is
a tax on application complexity.

Agreed. This works if you have a single transaction connected thru a pool
that does statement-level load balancing, so it works in both session and
transaction mode.

I was in favour of a scheme like this myself, earlier, but have more
thoughts now.

We must also consider the need for serialization across sessions or
transactions.

In transaction pooling mode, an application could get assigned a different
session, so a token would be much harder to pass around.

2. Reader waits for a conservatively chosen LSN. This is roughly what

MySQL derivatives do in their "causal_reads = on" and "wsrep_sync_wait =
1" modes. Read transactions would start off by finding the current end
of WAL on the primary, since that must be later than any commit that
already completed, and then waiting for that to apply locally. That means
every read transaction waits for a complete replication lag period,
potentially unnecessarily. This is tax on readers with unnecessary waiting.

This tries to make it easier for users by forcing all users to experience
a causality delay. Given the whole purpose of multi-node load balancing is
performance, referencing the master again simply defeats any performance
gain, so you couldn't ever use it for all sessions. It could be a USERSET
parameter, so could be turned off in most cases that didn't need it. But
its easier to use than (1).

Though this should be implemented in the pooler.

3. Writer waits, as proposed. In this model, there is no tax on readers

(they have zero overhead, aside from the added complexity of dealing with
the possibility of transactions being rejected when a standby falls behind
and is dropped from 'available' status; but database clients must already
deal with certain types of rare rejected queries/failures such as
deadlocks, serialization failures, server restarts etc). This is a tax on
writers.

This would seem to require that all readers must first check with the
master as to which standbys are now considered available, so it looks like
(2).

No -- in (3), that is this proposal, standbys don't check with the primary
when you run a transaction. Instead, the primary sends a constant stream
of authorizations (in the form of keepalives sent every
causal_reads_timeout / 2 in the current patch) to the standby, allowing it
to consider itself available for a short time into the future (currently
now + causal_reads_timeout - max_tolerable_clock_skew to be specific -- I
can elaborate on that logic in a separate email). At the start of a
transaction in causal reads mode (the first call to GetTransaction to be
specific), the standby knows immediately without communicating with the
primary whether it can proceed or must raise the error. In the happy case,
the reader simply compares the most recently received authorization's
expiry time with the system clock and proceeds. In the worst case, when
contact is lost between primary and standby, the primary must stall
causal_reads commits for causal_reads_timeout (see CausalReadsBeginStall).
Doing that makes sure that no causal reads commit can return (see
CausalReadsCommitCanReturn) before the lost standby has definitely started
raising the error for causal_reads queries (because its most recent
authorization has expired), in case it is still alive and handling requests
from clients.

It is not at all like (2), which introduces a conservative wait at the
start of every read transaction, slowing all readers down. In (3), readers
don't wait, they run (or are rejected) as fast as possible, but instead the
primary has to do extra things. Hence my categorization of (2) as a 'tax
on readers', and of (3) as a 'tax on writers'. The idea is that a site
with a high ratio of reads to writes would prefer zero-overhead reads.

The alternative is that we simply send readers to any standby and allow
the pool to work out separately whether the standby is still available,
which mostly works, but it doesn't handle sporadic slow downs on particular
standbys very well (if at all).

This proposal does handle sporadic slowdowns on standbys: it drops them
from the set of available standbys if they don't apply fast enough, all the
while maintaining the guarantee. Though occurs to me that it probably
needs some kind of defence against too much flapping between available and
unavailable (maybe some kind of back off on the 'joining' phase that
standbys go through when they transition from unavailable to available in
the current patch, which I realize I haven't described yet -- but I don't
want to get bogged down in details, while we're talking about the 30,000
foot view).

I think we need to look at whether this does actually give us anything, or
whether we are missing the underlying Heisenberg reality.

--
Thomas Munro
http://www.enterprisedb.com

#12Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Simon Riggs (#10)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Fri, Nov 13, 2015 at 1:16 AM, Simon Riggs <simon@2ndquadrant.com> wrote:

On 11 November 2015 at 09:22, Thomas Munro <thomas.munro@enterprisedb.com>
wrote:

1. Reader waits with exposed LSNs, as Heikki suggests. This is what
BerkeleyDB does in "read-your-writes" mode. It means that application
developers have the responsibility for correctly identifying transactions
with causal dependencies and dealing with LSNs (or whatever equivalent
tokens), potentially even passing them to other processes where the
transactions are causally dependent but run by multiple communicating
clients (for example, communicating microservices). This makes it
difficult to retrofit load balancing to pre-existing applications and (like
anything involving concurrency) difficult to reason about as applications
grow in size and complexity. It is efficient if done correctly, but it is
a tax on application complexity.

Agreed. This works if you have a single transaction connected thru a pool
that does statement-level load balancing, so it works in both session and
transaction mode.

I was in favour of a scheme like this myself, earlier, but have more
thoughts now.

We must also consider the need for serialization across sessions or
transactions.

In transaction pooling mode, an application could get assigned a different
session, so a token would be much harder to pass around.

Sorry for the double reply, I just wanted to add a couple more thoughts.

As discussed elsewhere in the thread, I think it makes absolute sense to
offer some kind of support for causality tokens, I don't see that on its
own as enough for most users. (At the least, it would be good to have
pg_wait_for_xlog_replay_location(lsn, timeout), but perhaps explicit BEGIN
syntax as suggested by Heikki, or a new field in the libpq protocol which
can be attached to any statement, and likewise for the commit LSN of
results).

It's true that a pooling system/middleware could spy on your sessions and
insert causality token handling imposing a global ordering of visibility
for you, so that naive users don't have to deal with them. Whenever it
sees a COMMIT result (assuming they are taught to return LSNs), it could
update a highest-LSN-seen variable, and transparently insert a wait for
that LSN into every transaction that it sees beginning. But then you would
have to push all your queries through a single point that can see
everything across all Postgres servers, and maintain this global high LSN.

In contrast, my writer-waits proposal makes different trade-offs to provide
causal reads as a built-in feature without an external single point
observer of all transactions.

--
Thomas Munro
http://www.enterprisedb.com

#13Simon Riggs
simon@2ndQuadrant.com
In reply to: Thomas Munro (#11)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On 12 November 2015 at 18:25, Thomas Munro <thomas.munro@enterprisedb.com>
wrote:

I don't want to get bogged down in details, while we're talking about the
30,000 foot view).

Hmm, if that's where we're at, I'll summarize my thoughts.

All of this discussion presupposes we are distributing/load balancing
queries so that reads and writes might occur on different nodes.

We need a good balancer. Any discussion of this that ignores the balancer
component is only talking about half the solution. What we need to do is
decide whether functionality should live in the balancer or the core.

Your option (1) is viable, but only in certain cases. We could add support
for some token/wait mechanism but as you say, this would require
application changes not pooler changes.

Your option (2) is wider but also worse in some ways. It can be implemented
in a pooler.

Your option (3) doesn't excite me much. You've got a load of stuff that
really should happen in a pooler. And at its core we have
synchronous_commit = apply but with a timeout rather than a wait. So
anyway, consider me nudged to finish my patch to provide capability for
that by 1 Jan.

On a related note, any further things like "GUC causal_reads_standby_names"
should be implemented by Node Registry as a named group of nodes. We can
have as many arbitrary groups of nodes as we want. If that sounds strange
look back at exactly why GUCs are called GUCs.

--
Simon Riggs http://www.2ndQuadrant.com/
<http://www.2ndquadrant.com/&gt;
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#14Robert Haas
robertmhaas@gmail.com
In reply to: Simon Riggs (#13)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Sun, Nov 15, 2015 at 5:41 AM, Simon Riggs <simon@2ndquadrant.com> wrote:

Hmm, if that's where we're at, I'll summarize my thoughts.

All of this discussion presupposes we are distributing/load balancing
queries so that reads and writes might occur on different nodes.

Agreed. I think that's a pretty common pattern, though certainly not
the only one.

We need a good balancer. Any discussion of this that ignores the balancer
component is only talking about half the solution. What we need to do is
decide whether functionality should live in the balancer or the core.

I'm all in favor of having a load-balancer in core, but that seems
completely unrelated to the patch at hand.

Your option (1) is viable, but only in certain cases. We could add support
for some token/wait mechanism but as you say, this would require application
changes not pooler changes.

Agreed.

Your option (2) is wider but also worse in some ways. It can be implemented
in a pooler.

Your option (3) doesn't excite me much. You've got a load of stuff that
really should happen in a pooler. And at its core we have synchronous_commit
= apply but with a timeout rather than a wait. So anyway, consider me nudged
to finish my patch to provide capability for that by 1 Jan.

I don't see how either option (2) or option (3) could be implemented
in a pooler. How would that work?

To be frank, it's starting to seem to me like you are just trying to
block this patch so you can have time to develop your own version
instead. I hope that's not the case, because it would be quite
unfair. When Thomas originally posted the patch, you complained that
"This causes every writer to wait. What we want is to isolate the wait
only to people performing a write-read sequence, so I think it should
be readers that wait. Let's have that debate up front before we start
reviewing the patch." Now, you seem to be saying that's OK, because
you want to post a patch to do exactly the same thing under the name
synchronous_commit=apply, but you want it to be your own patch,
leaving out the other stuff that Thomas has put into this one.

That could be the right thing to do, but how about we discuss it a
bit? The timeout stuff that Thomas has added here is really useful.
Without that, if a machine goes down, we wait forever. That's the
right thing to do if we're replicating to make sure transactions can
never be lost, but it's a bad idea if we're replicating for load
balancing. In the load balancing case, you want to drop sync slaves
quickly to ensure the cluster remains available, and you need them to
know they are out of sync so that the load balancer doesn't get
confused. That's exactly what is implemented here.

If you have an idea for a simpler implementation, great, but I think
we need something. I don't see how it's going to work to make it
entirely the pooler's job to figure out whether the cluster is in sync
- it needs a push from the core server. Here, that push is easy to
find: if a particular replica starts returning the "i'm out of sync"
error when you query it, then stop routing queries to that replica
until the error clears (which the pooler can find out by polling it
with some trivial query). That's a great deal more useful than
synchronous_commit=apply without such a feature.

On a related note, any further things like "GUC causal_reads_standby_names"
should be implemented by Node Registry as a named group of nodes. We can
have as many arbitrary groups of nodes as we want. If that sounds strange
look back at exactly why GUCs are called GUCs.

I think a node registry is a good idea, and my impression from the
session in Vienna is quite a few other hackers do, too. But I also
don't think it's remotely reasonable to make that a precondition for
accepting this patch.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#15Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Simon Riggs (#13)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Sun, Nov 15, 2015 at 11:41 PM, Simon Riggs <simon@2ndquadrant.com> wrote:

On 12 November 2015 at 18:25, Thomas Munro <thomas.munro@enterprisedb.com>
wrote:

I don't want to get bogged down in details, while we're talking about
the 30,000 foot view).

Hmm, if that's where we're at, I'll summarize my thoughts.

All of this discussion presupposes we are distributing/load balancing
queries so that reads and writes might occur on different nodes.

We need a good balancer. Any discussion of this that ignores the balancer
component is only talking about half the solution. What we need to do is
decide whether functionality should live in the balancer or the core.

Your option (1) is viable, but only in certain cases. We could add support
for some token/wait mechanism but as you say, this would require
application changes not pooler changes.

Your option (2) is wider but also worse in some ways. It can be
implemented in a pooler.

Your option (3) doesn't excite me much. You've got a load of stuff that
really should happen in a pooler. And at its core we have
synchronous_commit = apply but with a timeout rather than a wait. So
anyway, consider me nudged to finish my patch to provide capability for
that by 1 Jan.

Just to be clear, this patch doesn't use a "timeout rather than a wait".
It always waits for the current set of available causal reads standbys to
apply the commit. It's just that nodes get kicked out of that set pretty
soon if they don't keep up, a bit like a RAID controller dropping a failing
disk. And it does so using a protocol that ensures that the dropped
standby starts raising the error, even if contact has been lost with it, so
the causal reads guarantee is maintained at all times for all clients.

On a related note, any further things like "GUC causal_reads_standby_names"

should be implemented by Node Registry as a named group of nodes. We can
have as many arbitrary groups of nodes as we want. If that sounds strange
look back at exactly why GUCs are called GUCs.

Agreed, the application_name whitelist stuff is clunky. I left it out of
the first version I posted, not wanting the focus of this proposal to be
side-tracked. But as Ants Aasma pointed out, some users might need
something like that, so I posted a 2nd version that follows the established
example, again not wanting to distract with anything new in that area. Of
course that would eventually be replaced/improved as part of a future node
topology management project.

--
Thomas Munro
http://www.enterprisedb.com

#16Simon Riggs
simon@2ndQuadrant.com
In reply to: Robert Haas (#14)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On 15 November 2015 at 14:50, Robert Haas <robertmhaas@gmail.com> wrote:

On Sun, Nov 15, 2015 at 5:41 AM, Simon Riggs <simon@2ndquadrant.com>
wrote:

Hmm, if that's where we're at, I'll summarize my thoughts.

All of this discussion presupposes we are distributing/load balancing
queries so that reads and writes might occur on different nodes.

Agreed. I think that's a pretty common pattern, though certainly not
the only one.

It looks to me this functionality is only of use in a pooler. Please
explain how else this would be used.

Your option (2) is wider but also worse in some ways. It can be

implemented

in a pooler.

Your option (3) doesn't excite me much. You've got a load of stuff that
really should happen in a pooler. And at its core we have

synchronous_commit

= apply but with a timeout rather than a wait.

I don't see how either option (2) or option (3) could be implemented
in a pooler. How would that work?

My starting thought was that (1) was the only way forwards. Through
discussion, I now see that its not the best solution for the general case.

The pooler knows which statements are reads and writes, it also knows about
transaction boundaries, so it is possible for it to perform the waits for
either (2) or (3). The pooler *needs* to know which nodes it can route
queries to, so it looks to me that the pooler is the best place to put
waits and track status of nodes, no matter when we wait. I don't see any
benefit in having other nodes keep track of node status since that will
just replicate work that *must* be performed in the pooler.

I would like to see a load balancing pooler in Postgres.

--
Simon Riggs http://www.2ndQuadrant.com/
<http://www.2ndquadrant.com/&gt;
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#17Craig Ringer
craig@2ndquadrant.com
In reply to: Simon Riggs (#16)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On 16 November 2015 at 18:44, Simon Riggs <simon@2ndquadrant.com> wrote:

The pooler knows which statements are reads and writes

I think that's an iffy assumption. It's one we tend to make because
otherwise read/write pooling won't work, but in PostgreSQL there's really
no way to know when calling a function.

What does

SELECT get_user_stats()

do? The pooler has _no_ _idea_ unless manually configured with knowledge
about particular user defined functions.

In the absence of such knowledge it can:

- send the work to a replica and report the ERROR to the user if it fails
due to an attempted write;
- send the work to a replica, capture an ERROR due to attempted write, and
retry on the master;
- send everything it's not sure about to the master

If a pooler had insight into the catalogs and if we had readonly /
readwrite attributes on functions, it could be smarter.

I would like to see a load balancing pooler in Postgres.

Given the number of times I say "no, no, don't raise max_connections to
2000 to solve your performance problems, lower it to around 100 and put
pgbouncer in front if your application doesn't support connection pooling
internally" .... yes!

--
Craig Ringer http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#18Simon Riggs
simon@2ndQuadrant.com
In reply to: Craig Ringer (#17)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On 16 November 2015 at 11:01, Craig Ringer <craig@2ndquadrant.com> wrote:

On 16 November 2015 at 18:44, Simon Riggs <simon@2ndquadrant.com> wrote:

The pooler knows which statements are reads and writes

I think that's an iffy assumption.

It's not an assumption, its a requirement. If it can't do this in some
manner then you can't use a load balancing pooler.

Randomly submitting things works as well, since it leads to a write error
when you try to write data on a read only server, so you do then learn
whether it is a read or a write. Once you know its a write, you submit to
master. But you still need to be careful of other effects, so that isn't
recommended.

It's one we tend to make because otherwise read/write pooling won't work,

but in PostgreSQL there's really no way to know when calling a function.

What does

SELECT get_user_stats()

do? The pooler has _no_ _idea_ unless manually configured with knowledge
about particular user defined functions.

In the absence of such knowledge it can:

- send the work to a replica and report the ERROR to the user if it fails
due to an attempted write;
- send the work to a replica, capture an ERROR due to attempted write, and
retry on the master;
- send everything it's not sure about to the master

If a pooler had insight into the catalogs and if we had readonly /
readwrite attributes on functions, it could be smarter.

pgpool supports white/black function listing for exactly this reason.

--
Simon Riggs http://www.2ndQuadrant.com/
<http://www.2ndquadrant.com/&gt;
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#19Simon Riggs
simon@2ndQuadrant.com
In reply to: Simon Riggs (#13)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On 15 November 2015 at 10:41, Simon Riggs <simon@2ndquadrant.com> wrote:

So anyway, consider me nudged to finish my patch to provide capability
for that by 1 Jan.

My earlier patch aimed to allow WALReceiver to wait on both a latch and a
socket as well as allow WALWriter to be active, so that WALReceiver could
reply more quickly and handle greater workload. As I explained previously
when we discussed that in recent posts, it is necessary infrastructure to
have anybody wait on anything higher than remote-fsync. I aim to complete
that by 1 Jan.

--
Simon Riggs http://www.2ndQuadrant.com/
<http://www.2ndquadrant.com/&gt;
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#20Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Simon Riggs (#19)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Tue, Nov 17, 2015 at 12:44 AM, Simon Riggs <simon@2ndquadrant.com> wrote:

On 15 November 2015 at 10:41, Simon Riggs <simon@2ndquadrant.com> wrote:

So anyway, consider me nudged to finish my patch to provide capability
for that by 1 Jan.

My earlier patch aimed to allow WALReceiver to wait on both a latch and a
socket as well as allow WALWriter to be active, so that WALReceiver could
reply more quickly and handle greater workload. As I explained previously
when we discussed that in recent posts, it is necessary infrastructure to
have anybody wait on anything higher than remote-fsync. I aim to complete
that by 1 Jan.

Right, handing write/fsync work off to WALWriter in standbys makes a lot of
sense for any kind of writer-waits system, so that WALReceiver doesn't
spend time in long syscalls which wouldn't play nicely with signals
(whether from 'kill' or SetLatch) and can deal with network IO with the
lowest possible latency. I would like to help test/review that, if that
could be useful.

The SIGUSR1 code in the WalReceiverMain and WalRecvWakeup in this patch
works well enough for now for proof-of-concept purposes until then.

--
Thomas Munro
http://www.enterprisedb.com

#21Robert Haas
robertmhaas@gmail.com
In reply to: Simon Riggs (#16)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Mon, Nov 16, 2015 at 5:44 AM, Simon Riggs <simon@2ndquadrant.com> wrote:

On 15 November 2015 at 14:50, Robert Haas <robertmhaas@gmail.com> wrote:

On Sun, Nov 15, 2015 at 5:41 AM, Simon Riggs <simon@2ndquadrant.com>
wrote:

Hmm, if that's where we're at, I'll summarize my thoughts.

All of this discussion presupposes we are distributing/load balancing
queries so that reads and writes might occur on different nodes.

Agreed. I think that's a pretty common pattern, though certainly not
the only one.

It looks to me this functionality is only of use in a pooler. Please explain
how else this would be used.

I think you're right. I mean, you could have the pooling built into
the application, but some place connections have to be being farmed
out to different nodes, or there's no point to using this feature.
Some people may not want to use this feature, but those who do are
using some form of pooling at some level.

Your option (2) is wider but also worse in some ways. It can be
implemented
in a pooler.

Your option (3) doesn't excite me much. You've got a load of stuff that
really should happen in a pooler. And at its core we have
synchronous_commit
= apply but with a timeout rather than a wait.

I don't see how either option (2) or option (3) could be implemented
in a pooler. How would that work?

My starting thought was that (1) was the only way forwards. Through
discussion, I now see that its not the best solution for the general case.

The pooler knows which statements are reads and writes, it also knows about
transaction boundaries, so it is possible for it to perform the waits for
either (2) or (3).

As Craig says, it may not: pgbouncer, for example, won't. pgpool
will, except when it's wrong because some apparently read-only
function is actually writing data. But even if the pooler does know,
that isn't enough for it to perform the waits for (2) or (3) without
some support for the server. If it wants to wait for a particular
transaction to be applied on the standby, it needs to know how long to
wait, and without some support for the server, it has no way of
knowing. Now that could be done by doing (1) and then having the
pooler perform the waits, but now every pooler has to be taught how to
do that. pgpool needs to know, pgbouncer needs to know, every
JDBC-based connection pooler needs to know. Uggh. Thomas's solution
is nice because it works with any pooler.

The other point I'd make, which I think may be what you said before
but I think is worth making very explicit, is that (1) supposes that
we know which reads are dependent on which previous writes. In the
real world, that's probably frequently untrue. If someone does SELECT
sum(salary) FROM emp on the standby, there's no particular write to
the emp table that they want to wait for: they want to wait for ALL
such writes previously acknowledged as committed. Now, even when the
dependent writes can be identified, it may be convenient to just wait
for all of them instead of a particular subset that we know are
related. But I bet there will be many cases where identification is
impractical or impossible, and thus I suspect (1) won't be very
useful.

I think (2) and (3) both have good prospects for being useful, but I
suspect that the performance consequences of (3), which is what Thomas
actually implemented, although possibly severe, are still likely to be
only a fraction of the cost of (2). Having to potentially wait every
time a standby takes a snapshot just sounds awful to me.

I would like to see a load balancing pooler in Postgres.

Me, too, but I don't expect that to be possible in the near future,
and I think this is awfully useful until it does.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#22Jim Nasby
Jim.Nasby@BlueTreble.com
In reply to: Thomas Munro (#12)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On 11/12/15 1:11 PM, Thomas Munro wrote:

It's true that a pooling system/middleware could spy on your sessions
and insert causality token handling imposing a global ordering of
visibility for you, so that naive users don't have to deal with them.
Whenever it sees a COMMIT result (assuming they are taught to return
LSNs), it could update a highest-LSN-seen variable, and transparently
insert a wait for that LSN into every transaction that it sees
beginning. But then you would have to push all your queries through a
single point that can see everything across all Postgres servers, and
maintain this global high LSN.

I think that depends on what you're doing. Frequently you don't care
about anyone elses writes, just your own. In that case, there's no need
for a shared connection pooler, you just have to come back to the same one.

There's also a 4th option: until a commit has made it out to some number
of slaves, re-direct all reads from a session back to the master. That
might sound horrible for master performance, but in reality I think it'd
normally be fine. Generally, you only care about this when you're going
to read data that you've just written, which means the data's still in
shared buffers.
--
Jim Nasby, Data Architect, Blue Treble Consulting, Austin TX
Experts in Analytics, Data Architecture and PostgreSQL
Data in Trouble? Get it in Treble! http://BlueTreble.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#23Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Jim Nasby (#22)
3 attachment(s)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

Hi,

Here is a new version of the patch with a few small improvements:

1. Adopted the term '[read] lease', replacing various hand-wavy language
in the comments and code. That seems to be the established term for this
approach[1]Originally from a well known 1989 paper on caching, but in the context of databases and synchronous replication see for example the recent papers on "Niobe" and "Paxos Quorum Leases" (especially the reference to Google Megastore). Of course a *lot* more is going on in those very different algorithms, but at some level "read leases" are being used to allow local-node-only reads for a limited time while upholding some kind of global consistency guarantee, in some of those consensus database systems. I spent a bit of time talking about consistency levels to database guru and former colleague Alex Scotti who works on a Paxos-based system, and he gave me the initial idea to try out a lease-based consistency system for Postgres streaming rep. It seems like a very useful point in the space of trade-offs to me..

2. Reduced the stalling time on failure. When things go wrong with a
standby (such as losing contact with it), instead of stalling for a
conservative amount of time longer than any lease that might have been
granted, the primary now stalls only until the expiry of the last lease
that actually was granted to a given dropped standby, which should be
sooner.

3. Fixed a couple of bugs that showed up in testing and review (some bad
flow control in the signal handling, and a bug in a circular buffer), and
changed the recovery->walreceiver wakeup signal handling to block the
signal except while waiting in walrcv_receive (it didn't seem a good idea
to interrupt arbitrary syscalls in walreceiver so I thought that would be a
improvement; but of course that area's going to be reworked by Simon's
patch anyway, as discussed elsewhere).

Restating the central idea using the new terminology: So long as they are
replaying fast enough, the primary grants a series of causal reads leases
to standbys allowing them to handle causal reads queries locally without
any inter-node communication for a limited time. Leases are promises that
the primary will wait for the standby to apply commit records OR be dropped
from the set of available causal reads standbys and know that it has been
dropped, before the primary returns from commit, in order to uphold the
causal reads guarantee. In the worst case it can do that by waiting for
the most recently granted lease to expire.

I've also attached a couple of things which might be useful when trying the
patch out: test-causal-reads.c which can be used to test performance and
causality under various conditions, and test-causal-reads.sh which can be
used to bring up a primary and a bunch of local hot standbys to talk to.
(In the hope of encouraging people to take the patch for a spin...)

[1]: Originally from a well known 1989 paper on caching, but in the context of databases and synchronous replication see for example the recent papers on "Niobe" and "Paxos Quorum Leases" (especially the reference to Google Megastore). Of course a *lot* more is going on in those very different algorithms, but at some level "read leases" are being used to allow local-node-only reads for a limited time while upholding some kind of global consistency guarantee, in some of those consensus database systems. I spent a bit of time talking about consistency levels to database guru and former colleague Alex Scotti who works on a Paxos-based system, and he gave me the initial idea to try out a lease-based consistency system for Postgres streaming rep. It seems like a very useful point in the space of trade-offs to me.
of databases and synchronous replication see for example the recent papers
on "Niobe" and "Paxos Quorum Leases" (especially the reference to Google
Megastore). Of course a *lot* more is going on in those very different
algorithms, but at some level "read leases" are being used to allow
local-node-only reads for a limited time while upholding some kind of
global consistency guarantee, in some of those consensus database systems.
I spent a bit of time talking about consistency levels to database guru and
former colleague Alex Scotti who works on a Paxos-based system, and he gave
me the initial idea to try out a lease-based consistency system for
Postgres streaming rep. It seems like a very useful point in the space of
trade-offs to me.

--
Thomas Munro
http://www.enterprisedb.com

Attachments:

test-causal-reads.shapplication/x-sh; name=test-causal-reads.shDownload
test-causal-reads.ctext/x-csrc; charset=US-ASCII; name=test-causal-reads.cDownload
causal-reads-v3.patchapplication/octet-stream; name=causal-reads-v3.patchDownload
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 47312f6..535b5a9 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,10 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
+	{
+		CausalReadsWaitForLSN(XactLastRecEnd);
 		SyncRepWaitForLSN(XactLastRecEnd);
+	}
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5117,6 +5120,13 @@ XactLogCommitRecord(TimestampTz commit_time,
 		xl_xinfo.xinfo |= XACT_COMPLETION_FORCE_SYNC_COMMIT;
 
 	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this commit is applied.
+	 */
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY || causal_reads)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
+	/*
 	 * Relcache invalidations requires information about the current database
 	 * and so does logical decoding.
 	 */
@@ -5452,6 +5462,19 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 	if (XactCompletionForceSyncCommit(parsed->xinfo))
 		XLogFlush(lsn);
 
+	/*
+	 * Record the primary's timestamp for the commit record, so it can be used
+	 * for tracking replay lag.
+	 */
+	SetXLogReplayTimestamp(parsed->xact_time);
+
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit or causal reads), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 /*
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index f17f834..531f512 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -80,6 +80,8 @@ extern uint32 bootstrap_data_checksum_version;
 #define PROMOTE_SIGNAL_FILE		"promote"
 #define FALLBACK_PROMOTE_SIGNAL_FILE "fallback_promote"
 
+/* Size of the circular buffer of timestamped LSNs. */
+#define MAX_TIMESTAMPED_LSNS 8192
 
 /* User-settable parameters */
 int			max_wal_size = 64;	/* 1 GB */
@@ -346,6 +348,12 @@ static XLogRecPtr RedoRecPtr;
 static bool doPageWrites;
 
 /*
+ * doRequestWalReceiverReply is used by recovery code to ask the main recovery
+ * loop to trigger a walreceiver reply.
+ */
+static bool doRequestWalReceiverReply;
+
+/*
  * RedoStartLSN points to the checkpoint's REDO location which is specified
  * in a backup label file, backup history file or control file. In standby
  * mode, XLOG streaming usually starts from the position where an invalid
@@ -357,6 +365,13 @@ static bool doPageWrites;
  */
 static XLogRecPtr RedoStartLSN = InvalidXLogRecPtr;
 
+/*
+ * LastReplayedTimestamp can be set by redo handlers when they apply a record
+ * that carries a timestamp, by calling SetXLogReplayedTimestamp.  The xlog
+ * apply loop can then update the value in shared memory.
+ */
+static TimestampTz LastReplayedTimestamp = 0;
+
 /*----------
  * Shared-memory data structures for XLOG control
  *
@@ -632,6 +647,21 @@ typedef struct XLogCtlData
 	/* current effective recovery target timeline */
 	TimeLineID	RecoveryTargetTLI;
 
+	/* timestamp from the most recently applied record that carried a timestamp. */
+	TimestampTz lastReplayedTimestamp;
+
+	/*
+	 * We maintain a circular buffer of LSNs and associated timestamps.
+	 * Walreceiver writes into it using information from timestamps, and the
+	 * startup recovery process reads from it and notifies walreceiver when
+	 * LSNs are replayed so that the timestamps can be fed back to the
+	 * upstream server, to track lag.
+	 */
+	Index		timestampedLsnRead;
+	Index		timestampedLsnWrite;
+	XLogRecPtr	timestampedLsn[MAX_TIMESTAMPED_LSNS];
+	TimestampTz	timestampedLsnTime[MAX_TIMESTAMPED_LSNS];
+
 	/*
 	 * timestamp of when we started replaying the current chunk of WAL data,
 	 * only relevant for replication or archive recovery
@@ -6847,14 +6877,58 @@ StartupXLOG(void)
 				error_context_stack = errcallback.previous;
 
 				/*
-				 * Update lastReplayedEndRecPtr after this record has been
-				 * successfully replayed.
+				 * Update lastReplayedEndRecPtr and lastReplayedTimestamp
+				 * after this record has been successfully replayed.
 				 */
 				SpinLockAcquire(&XLogCtl->info_lck);
 				XLogCtl->lastReplayedEndRecPtr = EndRecPtr;
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
+				if (LastReplayedTimestamp != 0)
+				{
+					/* If replaying a record produced a timestamp, use that. */
+					XLogCtl->lastReplayedTimestamp = LastReplayedTimestamp;
+					LastReplayedTimestamp = 0;
+				}
+				else
+				{
+					/*
+					 * If we have applied LSNs associated with timestamps
+					 * received by walreceiver, then use the recorded
+					 * timestamp.  We consume from the read end of the
+					 * circular buffer.
+					 */
+					while (XLogCtl->timestampedLsnRead !=
+						   XLogCtl->timestampedLsnWrite &&
+						   XLogCtl->timestampedLsn[XLogCtl->timestampedLsnRead]
+						   <= EndRecPtr)
+					{
+						if (XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead] >
+							XLogCtl->lastReplayedTimestamp)
+						{
+							XLogCtl->lastReplayedTimestamp =
+								XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead];
+							doRequestWalReceiverReply = true;
+						}
+						XLogCtl->timestampedLsnRead =
+							(XLogCtl->timestampedLsnRead + 1) % MAX_TIMESTAMPED_LSNS;
+					}
+				}
 				SpinLockRelease(&XLogCtl->info_lck);
 
+				/*
+				 * If rm_redo reported that it applied a commit record that
+				 * the master is waiting for by calling
+				 * XLogRequestWalReceiverReply, or we encountered a WAL
+				 * location that was associated with a timestamp above, then
+				 * we wake up the receiver so that it notices the updated
+				 * lastReplayedEndRecPtr and sends a reply to the master.
+				 */
+				if (doRequestWalReceiverReply)
+				{
+					doRequestWalReceiverReply = false;
+					WalRcvWakeup();
+				}
+
 				/* Remember this record as the last-applied one */
 				LastRec = ReadRecPtr;
 
@@ -11583,3 +11657,103 @@ SetWalWriterSleeping(bool sleeping)
 	XLogCtl->WalWriterSleeping = sleeping;
 	SpinLockRelease(&XLogCtl->info_lck);
 }
+
+/*
+ * Called by redo code to indicate that the xlog replay loop should wake up
+ * the walreceiver process so that a reply can be sent to the primary.
+ */
+void
+XLogRequestWalReceiverReply(void)
+{
+	doRequestWalReceiverReply = true;
+}
+
+/*
+ * Merge timestamps from keepalive messages with the timestamps from WAL
+ * records, so that we can track lag while idle or while replaying large
+ * amounts of WAL without commit records.  In the former case there is no lag,
+ * and in the latter case we will remember a timestamp that goes with an
+ * arbitrary LSN, and wait for that LSN to be replayed before using the
+ * timestamp.
+ *
+ * This is called by walreceiver on standby servers when keepalive messages
+ * arrive.
+ */
+void
+SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn)
+{
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn == XLogCtl->lastReplayedEndRecPtr)
+	{
+		/*
+		 * That is the last replayed LSN: we are fully replayed, so we can
+		 * update the replay timestamp immediately.
+		 */
+		XLogCtl->lastReplayedTimestamp = timestamp;
+	}
+	else
+	{
+		/*
+		 * There is WAL still to be applied.  We will associate the timestamp
+		 * with this WAL position and wait for it to be replayed.  We add it
+		 * at the 'write' end of the circular buffer of LSN/timestamp
+		 * mappings, which the replay loop will eventually read.
+		 */
+		Index w = XLogCtl->timestampedLsnWrite;
+		Index r = XLogCtl->timestampedLsnRead;
+
+		XLogCtl->timestampedLsn[w] = lsn;
+		XLogCtl->timestampedLsnTime[w] = timestamp;
+
+		/* Advance the write point. */
+		w = (w + 1) % MAX_TIMESTAMPED_LSNS;
+		XLogCtl->timestampedLsnWrite = w;
+		if (w == r)
+		{
+			/*
+			 * The buffer is full.  Advance the read point (throwing away
+			 * oldest values; we will begin to overestimate replay lag, until
+			 * lag decreases to a size our buffer can manage, or the next
+			 * commit record is replayed).
+			 */
+			r = (r + 1) % MAX_TIMESTAMPED_LSNS;
+			XLogCtl->timestampedLsnRead = r;
+		}
+	}
+	SpinLockRelease(&XLogCtl->info_lck);
+}
+
+/*
+ * Set the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary.  This can be called by redo handlers that have
+ * an appropriate timestamp (currently only commit records).  Updating the
+ * shared memory value is deferred until after the redo handler returns.
+ */
+void
+SetXLogReplayTimestamp(TimestampTz timestamp)
+{
+	LastReplayedTimestamp = timestamp;
+}
+
+/*
+ * Get the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the master, and also the most recently applied LSN.  (Note
+ * that the timestamp and the LSN don't necessarily relate to the same
+ * record.)
+ *
+ * This is similar to GetLatestXTime, except that it is not only advanced by
+ * commit records (see SetXLogReplayTimestampAtLsn).
+ */
+TimestampTz
+GetXLogReplayTimestamp(XLogRecPtr *lsn)
+{
+	TimestampTz result;
+
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn)
+		*lsn = XLogCtl->lastReplayedEndRecPtr;
+	result = XLogCtl->lastReplayedTimestamp;
+	SpinLockRelease(&XLogCtl->info_lck);
+
+	return result;
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index ccc030f..f9b0e53 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -647,8 +647,10 @@ CREATE VIEW pg_stat_replication AS
             W.write_location,
             W.flush_location,
             W.replay_location,
+            W.replay_lag,
             W.sync_priority,
-            W.sync_state
+            W.sync_state,
+            W.causal_reads_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
             pg_stat_get_wal_senders() AS W
     WHERE S.usesysid = U.oid AND
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 325239d..4fc5fce 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -57,6 +57,11 @@
 #include "utils/builtins.h"
 #include "utils/ps_status.h"
 
+/* GUC variables */
+int causal_reads_timeout;
+bool causal_reads;
+char *causal_reads_standby_names;
+
 /* User-settable parameters for sync rep */
 char	   *SyncRepStandbyNames;
 
@@ -69,7 +74,7 @@ static int	SyncRepWaitMode = SYNC_REP_NO_WAIT;
 
 static void SyncRepQueueInsert(int mode);
 static void SyncRepCancelWait(void);
-static int	SyncRepWakeQueue(bool all, int mode);
+static int	SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn);
 
 static int	SyncRepGetStandbyPriority(void);
 
@@ -83,6 +88,239 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * ===========================================================
  */
 
+static bool
+SyncRepCheckEarlyExit(void)
+{
+	/*
+	 * If a wait for synchronous replication is pending, we can neither
+	 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
+	 * lead the client to believe that that the transaction aborted, which
+	 * is not true: it's already committed locally. The former is no good
+	 * either: the client has requested synchronous replication, and is
+	 * entitled to assume that an acknowledged commit is also replicated,
+	 * which might not be true. So in this case we issue a WARNING (which
+	 * some clients may be able to interpret) and shut off further output.
+	 * We do NOT reset ProcDiePending, so that the process will die after
+	 * the commit is cleaned up.
+	 */
+	if (ProcDiePending)
+	{
+		ereport(WARNING,
+				(errcode(ERRCODE_ADMIN_SHUTDOWN),
+				 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * It's unclear what to do if a query cancel interrupt arrives.  We
+	 * can't actually abort at this point, but ignoring the interrupt
+	 * altogether is not helpful, so we just terminate the wait with a
+	 * suitable warning.
+	 */
+	if (QueryCancelPending)
+	{
+		QueryCancelPending = false;
+		ereport(WARNING,
+				(errmsg("canceling wait for synchronous replication due to user request"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * If the postmaster dies, we'll probably never get an
+	 * acknowledgement, because all the wal sender processes will exit. So
+	 * just bail out.
+	 */
+	if (!PostmasterIsAlive())
+	{
+		ProcDiePending = true;
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	return false;
+}
+
+/*
+ * Check if we can stop waiting for causal consistency.  We can stop waiting
+ * when the following conditions are met:
+ *
+ * 1.  All walsenders currently in 'joining' or 'available' state have
+ * applied the target LSN.
+ *
+ * 2.  Any stall periods caused by standbys dropping out of 'available' state
+ * have passed, so that we can be sure that their leases have expired and they
+ * have started rejecting causal reads transactions.
+ *
+ * The output parameter 'waitingFor' is set to the number of nodes we are
+ * currently waiting for.  The output parameters 'stallTimeMillis' is set to
+ * the number of milliseconds we need to wait for to observe any current
+ * commit stall.
+ *
+ * Returns true if commit can return control, because every standby has either
+ * applied the LSN or started rejecting causal_reads transactions.
+ */
+static bool
+CausalReadsCommitCanReturn(XLogRecPtr XactCommitLSN,
+						   int *waitingFor,
+						   long *stallTimeMillis)
+{
+	int i;
+	TimestampTz now;
+
+	/* Count how many joining/available nodes we are waiting for. */
+	*waitingFor = 0;
+	for (i = 0; i < max_wal_senders; ++i)
+	{
+		WalSnd *walsnd = &WalSndCtl->walsnds[i];
+
+		/*
+		 * Assuming atomic read of pid_t, we can check walsnd->pid without
+		 * acquiring the spinlock to avoid memory synchronization costs for
+		 * unused walsender slots.  We see a value that existed sometime at
+		 * least as recently as the last memory barrier.
+		 */
+		if (walsnd->pid != 0)
+		{
+			/*
+			 * We need to hold the spinlock to read LSNs, because we can't be
+			 * sure they can be read atomically.
+			 */
+			SpinLockAcquire(&walsnd->mutex);
+			if (walsnd->pid != 0 && walsnd->causal_reads_state >= WALSNDCRSTATE_JOINING)
+			{
+				if (walsnd->apply < XactCommitLSN)
+					++*waitingFor;
+			}
+			SpinLockRelease(&walsnd->mutex);
+		}
+	}
+
+	/* Check if there is a stall in progress that we need to observe. */
+	now = GetCurrentTimestamp();
+	LWLockAcquire(SyncRepLock, LW_SHARED);
+	if (WalSndCtl->stall_causal_reads_until > now)
+	{
+		long seconds;
+		int usecs;
+
+		/* Compute how long we have to wait, rounded up to nearest ms. */
+		TimestampDifference(now, WalSndCtl->stall_causal_reads_until,
+							&seconds, &usecs);
+		*stallTimeMillis = seconds * 1000 + (usecs + 999) / 1000;
+	}
+	else
+		*stallTimeMillis = 0;
+	LWLockRelease(SyncRepLock);
+
+	/* We are done if we are not waiting for any nodes or stalls. */
+	return *waitingFor == 0 && *stallTimeMillis == 0;
+}
+
+/*
+ * Wait for causal consistency in causal_reads mode, if requested by user.
+ */
+void
+CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN)
+{
+	long stallTimeMillis;
+	int waitingFor;
+
+	/* Leave if we aren't in causal_reads mode. */
+	if (!causal_reads)
+		return;
+
+	for (;;)
+	{
+		/* Reset latch before checking state. */
+		ResetLatch(MyLatch);
+
+		/*
+		 * Join the queue to be woken up if any causal reads joining/available
+		 * standby applies XactCommitLSN, if we aren't already in it.  We
+		 * don't actually know if we need to wait for any peers yet, but we
+		 * have to register just in case before checking the walsenders' state
+		 * to avoid a race condition that could occur if we did it after
+		 * calling CausalReadsCommitCanReturn.  (SyncRepWaitForLSN doesn't
+		 * have to do this because it can check the highest-seen LSN in
+		 * walsndctl->lsn[mode] which is protected by SyncRepLock, the same
+		 * lock as the queues.  We can't do that here, because there is no
+		 * single highest-seen LSN that is useful.  We must check
+		 * walsnd->apply for all relevant walsenders.  Therefore we must
+		 * register for notifications first, so that we can be notified via
+		 * our latch of any standby applying the LSN we're interested in after
+		 * we check but before we start waiting, or we could wait forever for
+		 * something that has already happened.)
+		 */
+		LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+		if (MyProc->syncRepState != SYNC_REP_WAITING)
+		{
+			MyProc->waitLSN = XactCommitLSN;
+			MyProc->syncRepState = SYNC_REP_WAITING;
+			SyncRepQueueInsert(SYNC_REP_WAIT_CAUSAL_READS_APPLY);
+			Assert(SyncRepQueueIsOrderedByLSN(SYNC_REP_WAIT_CAUSAL_READS_APPLY));
+		}
+		LWLockRelease(SyncRepLock);
+
+		/* Check if we're done. */
+		if (CausalReadsCommitCanReturn(XactCommitLSN, &waitingFor, &stallTimeMillis))
+		{
+			SyncRepCancelWait();
+			break;
+		}
+
+		Assert(waitingFor > 0 || stallTimeMillis > 0);
+
+		/* If we aren't actually waiting for any standbys, leave the queue. */
+		if (waitingFor == 0)
+			SyncRepCancelWait();
+
+		/* Update the ps title. */
+		if (update_process_title)
+		{
+			char buffer[80];
+
+			snprintf(buffer, sizeof(buffer),
+					 "waiting for %d peer(s) to apply %X/%X%s",
+					 waitingFor,
+					 (uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN,
+					 stallTimeMillis > 0 ? " (stalling)" : "");
+			set_ps_display(buffer, false);
+		}
+
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit()) /* Calls SyncRepCancelWait() if true. */
+			break;
+
+		/*
+		 * If are still waiting for peers, then we wait for any joining or
+		 * available peer to reach the LSN (or possibly stop being in one of
+		 * those states or go away).
+		 *
+		 * If not, there must be a non-zero stall time, so we wait for that to
+		 * elapse.
+		 */
+		if (waitingFor > 0)
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH, -1);
+		else
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH | WL_TIMEOUT,
+					  stallTimeMillis);
+	}
+
+	/* There is no way out of the loop that could leave us in the queue. */
+	Assert(SHMQueueIsDetached(&(MyProc->syncRepLinks)));
+	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
+	MyProc->waitLSN = 0;
+
+	if (update_process_title)
+		set_ps_display("", false); /* TODO: restore what was there */
+}
+
 /*
  * Wait for synchronous replication, if requested by user.
  *
@@ -180,57 +418,9 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 		if (syncRepState == SYNC_REP_WAIT_COMPLETE)
 			break;
 
-		/*
-		 * If a wait for synchronous replication is pending, we can neither
-		 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
-		 * lead the client to believe that that the transaction aborted, which
-		 * is not true: it's already committed locally. The former is no good
-		 * either: the client has requested synchronous replication, and is
-		 * entitled to assume that an acknowledged commit is also replicated,
-		 * which might not be true. So in this case we issue a WARNING (which
-		 * some clients may be able to interpret) and shut off further output.
-		 * We do NOT reset ProcDiePending, so that the process will die after
-		 * the commit is cleaned up.
-		 */
-		if (ProcDiePending)
-		{
-			ereport(WARNING,
-					(errcode(ERRCODE_ADMIN_SHUTDOWN),
-					 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * It's unclear what to do if a query cancel interrupt arrives.  We
-		 * can't actually abort at this point, but ignoring the interrupt
-		 * altogether is not helpful, so we just terminate the wait with a
-		 * suitable warning.
-		 */
-		if (QueryCancelPending)
-		{
-			QueryCancelPending = false;
-			ereport(WARNING,
-					(errmsg("canceling wait for synchronous replication due to user request"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * If the postmaster dies, we'll probably never get an
-		 * acknowledgement, because all the wal sender processes will exit. So
-		 * just bail out.
-		 */
-		if (!PostmasterIsAlive())
-		{
-			ProcDiePending = true;
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit())
 			break;
-		}
 
 		/*
 		 * Wait on latch.  Any condition that should wake us up will set the
@@ -403,6 +593,49 @@ SyncRepGetSynchronousStandby(void)
 }
 
 /*
+ * Check if the current WALSender process's application_name matches a name in
+ * causal_reads_standby_names (including '*' for wildcard).
+ */
+bool
+CausalReadsPotentialStandby(void)
+{
+	char *rawstring;
+	List	   *elemlist;
+	ListCell   *l;
+	bool		found = false;
+
+	/* Need a modifiable copy of string */
+	rawstring = pstrdup(causal_reads_standby_names);
+
+	/* Parse string into list of identifiers */
+	if (!SplitIdentifierString(rawstring, ',', &elemlist))
+	{
+		/* syntax error in list */
+		pfree(rawstring);
+		list_free(elemlist);
+		/* GUC machinery will have already complained - no need to do again */
+		return 0;
+	}
+
+	foreach(l, elemlist)
+	{
+		char	   *standby_name = (char *) lfirst(l);
+
+		if (pg_strcasecmp(standby_name, application_name) == 0 ||
+			pg_strcasecmp(standby_name, "*") == 0)
+		{
+			found = true;
+			break;
+		}
+	}
+
+	pfree(rawstring);
+	list_free(elemlist);
+
+	return found;
+}
+
+/*
  * Update the LSNs on each queue based upon our latest state. This
  * implements a simple policy of first-valid-standby-releases-waiter.
  *
@@ -410,22 +643,27 @@ SyncRepGetSynchronousStandby(void)
  * perhaps also which information we store as well.
  */
 void
-SyncRepReleaseWaiters(void)
+SyncRepReleaseWaiters(bool walsender_cr_available_or_joining)
 {
 	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
+	int			numapply = 0;
+	int			numcausalreadsapply = 0;
+	bool		is_highest_priority_sync_standby;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
-	 * potential standbys then we have nothing to do. If we are still starting
-	 * up, still running base backup or the current flush position is still
-	 * invalid, then leave quickly also.
+	 * potential standbys and not in a state that causal_reads waits for, then
+	 * we have nothing to do. If we are still starting up, still running base
+	 * backup or the current flush position is still invalid, then leave
+	 * quickly also.
 	 */
-	if (MyWalSnd->sync_standby_priority == 0 ||
-		MyWalSnd->state < WALSNDSTATE_STREAMING ||
-		XLogRecPtrIsInvalid(MyWalSnd->flush))
+	if (!walsender_cr_available_or_joining &&
+		(MyWalSnd->sync_standby_priority == 0 ||
+		 MyWalSnd->state < WALSNDSTATE_STREAMING ||
+		 XLogRecPtrIsInvalid(MyWalSnd->flush)))
 		return;
 
 	/*
@@ -435,45 +673,77 @@ SyncRepReleaseWaiters(void)
 	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
 	syncWalSnd = SyncRepGetSynchronousStandby();
 
-	/* We should have found ourselves at least */
-	Assert(syncWalSnd != NULL);
+	/*
+	 * If we aren't managing the highest priority standby then make a note of
+	 * that so we can announce a takeover in the log if we ever get that job.
+	 */
+	is_highest_priority_sync_standby = syncWalSnd == MyWalSnd;
+	if (!is_highest_priority_sync_standby)
+		announce_next_takeover = true;
 
 	/*
-	 * If we aren't managing the highest priority standby then just leave.
+	 * If we aren't managing the highest priority standby or a standby in
+	 * causal reads 'joining' or 'available' state, then just leave.
 	 */
-	if (syncWalSnd != MyWalSnd)
+	if (!is_highest_priority_sync_standby && !walsender_cr_available_or_joining)
 	{
 		LWLockRelease(SyncRepLock);
-		announce_next_takeover = true;
 		return;
 	}
 
 	/*
 	 * Set the lsn first so that when we wake backends they will release up to
-	 * this location.
+	 * this location.  For the single-standby synchronous commit levels, we
+	 * only do this if we are the current synchronous standby and we are
+	 * advancing the LSN further than it has been advanced before, so that
+	 * SyncRepWaitForLSN can skip waiting in some cases.
 	 */
-	if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+	if (is_highest_priority_sync_standby)
 	{
-		walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
-		numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
-		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
+		if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
+			numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE,
+										MyWalSnd->write);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->write)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
+			numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH,
+										MyWalSnd->flush);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+			numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY,
+										MyWalSnd->apply);
+		}
 	}
+	/*
+	 * For causal_reads, all walsenders currently in available or joining
+	 * state must reach the LSN on their own, and standbys will reach LSNs in
+	 * any order.  It doesn't make sense to keep the highest seen LSN in a
+	 * single walsndctl->lsn element.  (CausalReadsWaitForLSN has handling for
+	 * LSNs that have already been reached).
+	 */
+	if (walsender_cr_available_or_joining)
+		numcausalreadsapply =
+			SyncRepWakeQueue(false, SYNC_REP_WAIT_CAUSAL_READS_APPLY,
+							 MyWalSnd->apply);
 
 	LWLockRelease(SyncRepLock);
 
-	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X",
+	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%X, %d procs to causal_reads apply",
 		 numwrite, (uint32) (MyWalSnd->write >> 32), (uint32) MyWalSnd->write,
-	   numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush);
+		 numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush,
+		 numapply, (uint32) (MyWalSnd->apply >> 32), (uint32) MyWalSnd->apply,
+		 numcausalreadsapply);
 
 	/*
 	 * If we are managing the highest priority standby, though we weren't
 	 * prior to this, then announce we are now the sync standby.
 	 */
-	if (announce_next_takeover)
+	if (is_highest_priority_sync_standby && announce_next_takeover)
 	{
 		announce_next_takeover = false;
 		ereport(LOG,
@@ -548,9 +818,8 @@ SyncRepGetStandbyPriority(void)
  * Must hold SyncRepLock.
  */
 static int
-SyncRepWakeQueue(bool all, int mode)
+SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn)
 {
-	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	PGPROC	   *proc = NULL;
 	PGPROC	   *thisproc = NULL;
 	int			numprocs = 0;
@@ -567,7 +836,7 @@ SyncRepWakeQueue(bool all, int mode)
 		/*
 		 * Assume the queue is ordered by LSN
 		 */
-		if (!all && walsndctl->lsn[mode] < proc->waitLSN)
+		if (!all && lsn < proc->waitLSN)
 			return numprocs;
 
 		/*
@@ -627,7 +896,7 @@ SyncRepUpdateSyncStandbysDefined(void)
 			int			i;
 
 			for (i = 0; i < NUM_SYNC_REP_WAIT_MODE; i++)
-				SyncRepWakeQueue(true, i);
+				SyncRepWakeQueue(true, i, InvalidXLogRecPtr);
 		}
 
 		/*
@@ -679,13 +948,31 @@ SyncRepQueueIsOrderedByLSN(int mode)
 #endif
 
 /*
+ * Make sure that CausalReadsWaitForLSN can't return until after the given
+ * lease expiry time has been reached.
+ *
+ * Wake up all backends waiting in CausalReadsWaitForLSN, because the set of
+ * available/joining peers has changed, and there is a new stall time they
+ * need to observe.
+ */
+void
+CausalReadsBeginStall(TimestampTz lease_expiry_time)
+{
+	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+	WalSndCtl->stall_causal_reads_until =
+		Max(WalSndCtl->stall_causal_reads_until, lease_expiry_time);
+	SyncRepWakeQueue(true, SYNC_REP_WAIT_CAUSAL_READS_APPLY, InvalidXLogRecPtr);
+	LWLockRelease(SyncRepLock);
+}
+
+/*
  * ===========================================================
  * Synchronous Replication functions executed by any process
  * ===========================================================
  */
 
 bool
-check_synchronous_standby_names(char **newval, void **extra, GucSource source)
+check_standby_names(char **newval, void **extra, GucSource source)
 {
 	char	   *rawstring;
 	List	   *elemlist;
@@ -728,6 +1015,9 @@ assign_synchronous_commit(int newval, void *extra)
 		case SYNCHRONOUS_COMMIT_REMOTE_FLUSH:
 			SyncRepWaitMode = SYNC_REP_WAIT_FLUSH;
 			break;
+		case SYNCHRONOUS_COMMIT_REMOTE_APPLY:
+			SyncRepWaitMode = SYNC_REP_WAIT_APPLY;
+			break;
 		default:
 			SyncRepWaitMode = SYNC_REP_NO_WAIT;
 			break;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 183a3a5..ab61b3f 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -52,6 +52,7 @@
 #include "libpq/pqformat.h"
 #include "libpq/pqsignal.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "replication/walsender.h"
 #include "storage/ipc.h"
@@ -96,6 +97,7 @@ static uint32 recvOff = 0;
  */
 static volatile sig_atomic_t got_SIGHUP = false;
 static volatile sig_atomic_t got_SIGTERM = false;
+static volatile sig_atomic_t got_SIGUSR2 = false;
 
 /*
  * LogstreamResult indicates the byte positions that we have already
@@ -140,14 +142,33 @@ static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
 static void XLogWalRcvSendReply(bool force, bool requestReply);
 static void XLogWalRcvSendHSFeedback(bool immed);
-static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
+static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+								  TimestampTz *causalReadsUntil);
 
 /* Signal handlers */
 static void WalRcvSigHupHandler(SIGNAL_ARGS);
 static void WalRcvSigUsr1Handler(SIGNAL_ARGS);
+static void WalRcvSigUsr2Handler(SIGNAL_ARGS);
 static void WalRcvShutdownHandler(SIGNAL_ARGS);
 static void WalRcvQuickDieHandler(SIGNAL_ARGS);
 
+static void WalRcvBlockSigUsr2(void)
+{
+	sigset_t mask;
+
+	sigemptyset(&mask);
+	sigaddset(&mask, SIGUSR2);
+	sigprocmask(SIG_BLOCK, &mask, NULL);
+}
+
+static void WalRcvUnblockSigUsr2(void)
+{
+	sigset_t mask;
+
+	sigemptyset(&mask);
+	sigaddset(&mask, SIGUSR2);
+	sigprocmask(SIG_UNBLOCK, &mask, NULL);
+}
 
 static void
 ProcessWalRcvInterrupts(void)
@@ -195,6 +216,7 @@ WalReceiverMain(void)
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz last_recv_timestamp;
 	bool		ping_sent;
+	bool		forceReply;
 
 	/*
 	 * WalRcv should be set up already (if we are a backend, we inherit this
@@ -246,6 +268,7 @@ WalReceiverMain(void)
 
 	/* Initialise to a sanish value */
 	walrcv->lastMsgSendTime = walrcv->lastMsgReceiptTime = walrcv->latestWalEndTime = GetCurrentTimestamp();
+	walrcv->causalReadsLease = 0;
 
 	SpinLockRelease(&walrcv->mutex);
 
@@ -263,7 +286,7 @@ WalReceiverMain(void)
 	pqsignal(SIGALRM, SIG_IGN);
 	pqsignal(SIGPIPE, SIG_IGN);
 	pqsignal(SIGUSR1, WalRcvSigUsr1Handler);
-	pqsignal(SIGUSR2, SIG_IGN);
+	pqsignal(SIGUSR2, WalRcvSigUsr2Handler);
 
 	/* Reset some signals that are accepted by postmaster but not here */
 	pqsignal(SIGCHLD, SIG_DFL);
@@ -294,6 +317,10 @@ WalReceiverMain(void)
 	/* Unblock signals (they were blocked when the postmaster forked us) */
 	PG_SETMASK(&UnBlockSig);
 
+	/* Block SIGUSR2 (we unblock it only during network waits). */
+	WalRcvBlockSigUsr2();
+	got_SIGUSR2 = false;
+
 	/* Establish the connection to the primary for XLOG streaming */
 	EnableWalRcvImmediateExit();
 	walrcv_connect(conninfo);
@@ -403,7 +430,9 @@ WalReceiverMain(void)
 				}
 
 				/* Wait a while for data to arrive */
+				WalRcvUnblockSigUsr2();
 				len = walrcv_receive(NAPTIME_PER_CYCLE, &buf);
+				WalRcvBlockSigUsr2();
 				if (len != 0)
 				{
 					/*
@@ -434,11 +463,21 @@ WalReceiverMain(void)
 							endofwal = true;
 							break;
 						}
+						WalRcvUnblockSigUsr2();
 						len = walrcv_receive(0, &buf);
+						WalRcvBlockSigUsr2();
+					}
+
+					if (got_SIGUSR2)
+					{
+						/* The recovery process asked us to force a reply. */
+						got_SIGUSR2 = false;
+						forceReply = true;
 					}
 
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(false, false);
+					XLogWalRcvSendReply(forceReply, false);
+					forceReply = false;
 
 					/*
 					 * If we've written some records, flush them to disk and
@@ -493,7 +532,15 @@ WalReceiverMain(void)
 						}
 					}
 
-					XLogWalRcvSendReply(requestReply, requestReply);
+					/* Check if the startup process has signaled us. */
+					if (got_SIGUSR2)
+					{
+						got_SIGUSR2 = false;
+						forceReply = true;
+					}
+
+					XLogWalRcvSendReply(requestReply || forceReply, requestReply);
+					forceReply = false;
 					XLogWalRcvSendHSFeedback(false);
 				}
 			}
@@ -735,6 +782,13 @@ WalRcvSigUsr1Handler(SIGNAL_ARGS)
 	errno = save_errno;
 }
 
+/* SIGUSR2: used to receive wakeups from recovery */
+static void
+WalRcvSigUsr2Handler(SIGNAL_ARGS)
+{
+	got_SIGUSR2 = true;
+}
+
 /* SIGTERM: set flag for main loop, or shutdown immediately if safe */
 static void
 WalRcvShutdownHandler(SIGNAL_ARGS)
@@ -795,6 +849,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 	XLogRecPtr	walEnd;
 	TimestampTz sendTime;
 	bool		replyRequested;
+	TimestampTz causalReadsLease;
 
 	resetStringInfo(&incoming_message);
 
@@ -815,7 +870,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				walEnd = pq_getmsgint64(&incoming_message);
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, NULL);
 
 				buf += hdrlen;
 				len -= hdrlen;
@@ -825,7 +880,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 		case 'k':				/* Keepalive */
 			{
 				/* copy message to StringInfo */
-				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
+				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char) + sizeof(int64);
 				if (len != hdrlen)
 					ereport(ERROR,
 							(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -837,8 +892,12 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
 				replyRequested = pq_getmsgbyte(&incoming_message);
+				causalReadsLease = IntegerTimestampToTimestampTz(
+					pq_getmsgint64(&incoming_message));
+				ProcessWalSndrMessage(walEnd, sendTime, &causalReadsLease);
 
-				ProcessWalSndrMessage(walEnd, sendTime);
+				/* Remember primary's timestamp at this WAL location. */
+				SetXLogReplayTimestampAtLsn(sendTime, walEnd);
 
 				/* If the primary requested a reply, send one immediately */
 				if (replyRequested)
@@ -1032,6 +1091,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	XLogRecPtr	applyPtr;
 	static TimestampTz sendTime = 0;
 	TimestampTz now;
+	TimestampTz applyTimestamp = 0;
 
 	/*
 	 * If the user doesn't want status to be reported to the master, be sure
@@ -1063,7 +1123,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	/* Construct a new message */
 	writePtr = LogstreamResult.Write;
 	flushPtr = LogstreamResult.Flush;
-	applyPtr = GetXLogReplayRecPtr(NULL);
+	applyTimestamp = GetXLogReplayTimestamp(&applyPtr);
 
 	resetStringInfo(&reply_message);
 	pq_sendbyte(&reply_message, 'r');
@@ -1071,6 +1131,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	pq_sendint64(&reply_message, flushPtr);
 	pq_sendint64(&reply_message, applyPtr);
 	pq_sendint64(&reply_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&reply_message, TimestampTzToIntegerTimestamp(applyTimestamp));
 	pq_sendbyte(&reply_message, requestReply ? 1 : 0);
 
 	/* Send it */
@@ -1169,15 +1230,52 @@ XLogWalRcvSendHSFeedback(bool immed)
  * Update shared memory status upon receiving a message from primary.
  *
  * 'walEnd' and 'sendTime' are the end-of-WAL and timestamp of the latest
- * message, reported by primary.
+ * message, reported by primary.  'causalReadsLease' is a pointer to
+ * the time the primary promises that this standby can safely claim to be
+ * causally consistent, to 0 if it cannot, or a NULL pointer for no change.
  */
 static void
-ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
+ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+					  TimestampTz *causalReadsLease)
 {
 	WalRcvData *walrcv = WalRcv;
 
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
 
+	/* Sanity check for the causalReadsLease time. */
+	if (causalReadsLease != NULL && *causalReadsLease != 0)
+	{
+		/* Deduce max_clock_skew from the causalReadsLease and sendTime. */
+#ifdef HAVE_INT64_TIMESTAMP
+		int64 diffMillis = (*causalReadsLease - sendTime) / 1000;
+#else
+		int64 diffMillis = (*causalReadsLease - sendTime) * 1000;
+#endif
+		int64 max_clock_skew = diffMillis / (CAUSAL_READS_CLOCK_SKEW_RATIO - 1);
+
+		if (sendTime > TimestampTzPlusMilliseconds(lastMsgReceiptTime, max_clock_skew))
+		{
+			/*
+			 * The primary's clock is more than max_clock_skew + network
+			 * latency ahead of the standby's clock.  (If the primary's clock
+			 * is more than max_clock_skew ahead of the standby's clock, but
+			 * by less than the network latency, then there isn't much we can
+			 * do to detect that; but it still seems useful to have this basic
+			 * sanity check for wildly misconfigured servers.)
+			 */
+			elog(LOG, "the primary server's clock time is too far ahead");
+			causalReadsLease = NULL;
+		}
+		/*
+		 * We could also try to detect cases where sendTime is more than
+		 * max_clock_skew in the past according to the standby's clock, but
+		 * that is indistinguishable from network latency/buffering, so we
+		 * could produce misleading error messages; if we do nothing, the
+		 * consequence is 'standby is not available for causal reads' errors
+		 * which should cause the user to investigate.
+		 */
+	}
+
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
 	if (walrcv->latestWalEnd < walEnd)
@@ -1185,6 +1283,8 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->latestWalEnd = walEnd;
 	walrcv->lastMsgSendTime = sendTime;
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
+	if (causalReadsLease != NULL)
+		walrcv->causalReadsLease = *causalReadsLease;
 	SpinLockRelease(&walrcv->mutex);
 
 	if (log_min_messages <= DEBUG2)
@@ -1215,3 +1315,23 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 		pfree(receipttime);
 	}
 }
+
+/*
+ * Wake up the walreceiver if it happens to be blocked in walrcv_receive,
+ * and tell it that a commit record has been applied.
+ *
+ * This is called by the startup process whenever interesting xlog records
+ * are applied, so that walreceiver can check if it needs to send an apply
+ * notification back to the master which may be waiting in a COMMIT with
+ * synchronous_commit = apply or causal_reads = on.
+ *
+ * TODO: This may change -- Simon Riggs suggested latches for this.  Maybe
+ * pipes would work too (and avoid interrupting systems calls and allow for
+ * multiplexed IO with the replication socket).
+ */
+void
+WalRcvWakeup(void)
+{
+	if (WalRcv->pid != 0)
+		kill(WalRcv->pid, SIGUSR2);
+}
diff --git a/src/backend/replication/walreceiverfuncs.c b/src/backend/replication/walreceiverfuncs.c
index 4452f25..a76d4da 100644
--- a/src/backend/replication/walreceiverfuncs.c
+++ b/src/backend/replication/walreceiverfuncs.c
@@ -25,9 +25,11 @@
 
 #include "access/xlog_internal.h"
 #include "postmaster/startup.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "storage/pmsignal.h"
 #include "storage/shmem.h"
+#include "utils/guc.h"
 #include "utils/timestamp.h"
 
 WalRcvData *WalRcv = NULL;
@@ -374,3 +376,21 @@ GetReplicationTransferLatency(void)
 
 	return ms;
 }
+
+/*
+ * Used by snapmgr to check if this standby has a valid lease, granting it the
+ * right to consider itself available for causal reads.
+ */
+bool
+WalRcvCausalReadsAvailable(void)
+{
+	WalRcvData *walrcv = WalRcv;
+	TimestampTz now = GetCurrentTimestamp();
+	bool result;
+
+	SpinLockAcquire(&walrcv->mutex);
+	result = walrcv->causalReadsLease != 0 && now <= walrcv->causalReadsLease;
+	SpinLockRelease(&walrcv->mutex);
+
+	return result;
+}
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 4a4643e..8405e40 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -153,9 +153,20 @@ static StringInfoData tmpbuf;
  */
 static TimestampTz last_reply_timestamp = 0;
 
+static TimestampTz last_keepalive_timestamp = 0;
+
 /* Have we sent a heartbeat message asking for reply, since last reply? */
 static bool waiting_for_ping_response = false;
 
+/* How long do need to stay in JOINING state? */
+static TimestampTz causal_reads_joining_until = 0;
+
+/* The last causal reads lease sent to the standby. */
+static TimestampTz causal_reads_last_lease = 0;
+
+/* Is this WALSender listed in causal_reads_standby_names? */
+static bool am_potential_causal_reads_standby = false;
+
 /*
  * While streaming WAL in Copy mode, streamingDoneSending is set to true
  * after we have sent CopyDone. We should not send any more CopyData messages
@@ -242,6 +253,57 @@ InitWalSender(void)
 }
 
 /*
+ * If we are exiting unexpectedly, we may need to communicate with concurrent
+ * causal_reads commits to maintain the causal consistency guarantee.
+ */
+static void
+PrepareUncleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * We've lost contact with the standby, but it may still be alive.  We
+		 * can't let any causal_reads transactions return until we've stalled
+		 * for long enough for a zombie standby to start raising errors
+		 * because its lease has expired.
+		 */
+		elog(LOG, "standby \"%s\" is lost (no longer available for causal reads)", application_name);
+		CausalReadsBeginStall(causal_reads_last_lease);
+
+		/*
+		 * We set the state to a lower level _after_ beginning the stall,
+		 * otherwise there would be a tiny window where commits could return
+		 * without observing the stall.
+		 */
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
+ * We are shutting down because we received a goodbye message from the
+ * walreceiver.
+ */
+static void
+PrepareCleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * The standby is shutting down, so it won't be running any more
+		 * transactions.  It is therefore safe to stop waiting for it, and no
+		 * stall is necessary.
+		 */
+		elog(LOG, "standby \"%s\" is leaving (no longer available for causal reads)", application_name);
+
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
  * Clean up after an error.
  *
  * WAL sender processes don't use transactions like regular backends do.
@@ -264,7 +326,10 @@ WalSndErrorCleanup(void)
 
 	replication_active = false;
 	if (walsender_ready_to_stop)
+	{
+		PrepareUncleanExit();
 		proc_exit(0);
+	}
 
 	/* Revert back to startup state */
 	WalSndSetState(WALSNDSTATE_STARTUP);
@@ -276,6 +341,8 @@ WalSndErrorCleanup(void)
 static void
 WalSndShutdown(void)
 {
+	PrepareUncleanExit();
+
 	/*
 	 * Reset whereToSendOutput to prevent ereport from attempting to send any
 	 * more messages to the standby.
@@ -1386,6 +1453,7 @@ ProcessRepliesIfAny(void)
 		if (r < 0)
 		{
 			/* unexpected error or EOF */
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1402,6 +1470,7 @@ ProcessRepliesIfAny(void)
 		resetStringInfo(&reply_message);
 		if (pq_getmessage(&reply_message, 0))
 		{
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1451,6 +1520,7 @@ ProcessRepliesIfAny(void)
 				 * 'X' means that the standby is closing down the socket.
 				 */
 			case 'X':
+				PrepareCleanExit();
 				proc_exit(0);
 
 			default:
@@ -1543,15 +1613,29 @@ ProcessStandbyReplyMessage(void)
 	XLogRecPtr	writePtr,
 				flushPtr,
 				applyPtr;
+	int			applyLagMs;
 	bool		replyRequested;
+	TimestampTz now = GetCurrentTimestamp();
+	TimestampTz applyTimestamp;
 
 	/* the caller already consumed the msgtype byte */
 	writePtr = pq_getmsgint64(&reply_message);
 	flushPtr = pq_getmsgint64(&reply_message);
 	applyPtr = pq_getmsgint64(&reply_message);
 	(void) pq_getmsgint64(&reply_message);		/* sendTime; not used ATM */
+	applyTimestamp = IntegerTimestampToTimestampTz(pq_getmsgint64(&reply_message));
 	replyRequested = pq_getmsgbyte(&reply_message);
 
+	/* Compute the apply lag in milliseconds. */
+	if (applyTimestamp == 0)
+		applyLagMs = -1;
+	else
+#ifdef HAVE_INT64_TIMESTAMP
+		applyLagMs = (now - applyTimestamp) / 1000;
+#else
+		applyLagMs = (now - applyTimestamp) * 1000.0;
+#endif
+
 	elog(DEBUG2, "write %X/%X flush %X/%X apply %X/%X%s",
 		 (uint32) (writePtr >> 32), (uint32) writePtr,
 		 (uint32) (flushPtr >> 32), (uint32) flushPtr,
@@ -1568,16 +1652,91 @@ ProcessStandbyReplyMessage(void)
 	 */
 	{
 		WalSnd *walsnd = MyWalSnd;
+		WalSndCausalReadsState causal_reads_state = walsnd->causal_reads_state;
+		bool causal_reads_state_changed = false;
+
+		/*
+		 * Handle causal reads state transitions, if a causal_reads_timeout is
+		 * configured, this standby is listed in causal_reads_standby_names,
+		 * and we are a primary database (not a cascading standby).
+		 */
+		if (causal_reads_timeout != 0 &&
+			am_potential_causal_reads_standby &&
+			!am_cascading_walsender)
+		{
+			if (applyLagMs >= 0 && applyLagMs < causal_reads_timeout)
+			{
+				if (causal_reads_state == WALSNDCRSTATE_UNAVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_JOINING;
+					causal_reads_joining_until =
+						TimestampTzPlusMilliseconds(now, causal_reads_timeout);
+					causal_reads_state_changed = true;
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING &&
+						 now >= causal_reads_joining_until)
+				{
+					causal_reads_state = WALSNDCRSTATE_AVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+			else
+			{
+				if (causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+					/*
+					 * We are dropping a causal reads available standby, so we
+					 * mustn't let any commit command that is waiting in
+					 * CausalReadsWaitForLSN return until we are sure that the
+					 * standby definitely knows that it's not available and
+					 * starts raising errors for causal_reads transactions.
+					 * TODO: We could just wait until the standby acks that
+					 * its lease has been cancelled, and start numbering
+					 * keepalives and sending the number back in replies, so
+					 * we know it's acking the right message; then lagging
+					 * standbys would be less disruptive, but for now we just
+					 * wait for the lease to expire, as we do when we lose
+					 * contact with a standby, for the sake of simplicity.
+					 */
+					CausalReadsBeginStall(causal_reads_last_lease);
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING)
+				{
+					/*
+					 * Dropping a joining standby doesn't require a stall,
+					 * because the standby doesn't think it's available, so
+					 * it's already raising the error for causal_reads
+					 * transactions.
+					 */
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+		}
 
 		SpinLockAcquire(&walsnd->mutex);
 		walsnd->write = writePtr;
 		walsnd->flush = flushPtr;
 		walsnd->apply = applyPtr;
+		walsnd->applyLagMs = applyLagMs;
+		walsnd->causal_reads_state = causal_reads_state;
 		SpinLockRelease(&walsnd->mutex);
+
+		if (causal_reads_state_changed)
+		{
+			WalSndKeepalive(true);
+			elog(LOG, "standby \"%s\" is %s", application_name,
+				 causal_reads_state == WALSNDCRSTATE_UNAVAILABLE ? "unavailable for causal reads" :
+				 causal_reads_state == WALSNDCRSTATE_JOINING ? "joining as a causal reads standby..." :
+				 causal_reads_state == WALSNDCRSTATE_AVAILABLE ? "available for causal reads" :
+				 "UNKNOWN");
+		}
 	}
 
 	if (!am_cascading_walsender)
-		SyncRepReleaseWaiters();
+		SyncRepReleaseWaiters(MyWalSnd->causal_reads_state >= WALSNDCRSTATE_JOINING);
 
 	/*
 	 * Advance our local xmin horizon when the client confirmed a flush.
@@ -1724,27 +1883,34 @@ WalSndComputeSleeptime(TimestampTz now)
 {
 	long		sleeptime = 10000;		/* 10 s */
 
-	if (wal_sender_timeout > 0 && last_reply_timestamp > 0)
+	if ((wal_sender_timeout > 0 || causal_reads_timeout > 0) && last_reply_timestamp > 0)
 	{
 		TimestampTz wakeup_time;
 		long		sec_to_timeout;
 		int			microsec_to_timeout;
 
-		/*
-		 * At the latest stop sleeping once wal_sender_timeout has been
-		 * reached.
-		 */
-		wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-												  wal_sender_timeout);
-
-		/*
-		 * If no ping has been sent yet, wakeup when it's time to do so.
-		 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
-		 * the timeout passed without a response.
-		 */
-		if (!waiting_for_ping_response)
+		if (causal_reads_timeout != 0)
+			wakeup_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+													  causal_reads_timeout /
+													  CAUSAL_READS_KEEPALIVE_RATIO);
+		else
+		{
+			/*
+			 * At the latest stop sleeping once wal_sender_timeout has been
+			 * reached.
+			 */
 			wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-													  wal_sender_timeout / 2);
+													  wal_sender_timeout);
+
+			/*
+			 * If no ping has been sent yet, wakeup when it's time to do so.
+			 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
+			 * the timeout passed without a response.
+			 */
+			if (!waiting_for_ping_response)
+				wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+														  wal_sender_timeout / 2);
+		}
 
 		/* Compute relative time until wakeup. */
 		TimestampDifference(now, wakeup_time,
@@ -1765,15 +1931,28 @@ static void
 WalSndCheckTimeOut(TimestampTz now)
 {
 	TimestampTz timeout;
+	int allowed_time;
 
 	/* don't bail out if we're doing something that doesn't require timeouts */
 	if (last_reply_timestamp <= 0)
 		return;
 
+	/*
+	 * If a causal_reads_timeout is configured, it is used instead of
+	 * wal_sender_timeout.  Ideally we'd use causal_reads_timeout / 2 +
+	 * allowance for network latency, but since walreceiver can become quite
+	 * bogged down fsyncing WAL we allow more tolerance.  (This could be
+	 * tightened up once standbys hand writing off to the WAL writer).
+	 */
+	if (causal_reads_timeout != 0)
+		allowed_time = causal_reads_timeout;
+	else
+		allowed_time = wal_sender_timeout;
+
 	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
-										  wal_sender_timeout);
+										  allowed_time);
 
-	if (wal_sender_timeout > 0 && now >= timeout)
+	if (allowed_time > 0 && now >= timeout)
 	{
 		/*
 		 * Since typically expiration of replication timeout means
@@ -1806,6 +1985,9 @@ WalSndLoop(WalSndSendDataCallback send_data)
 	last_reply_timestamp = GetCurrentTimestamp();
 	waiting_for_ping_response = false;
 
+	/* Check if we are managing potential causal_reads standby. */
+	am_potential_causal_reads_standby = CausalReadsPotentialStandby();
+
 	/*
 	 * Loop until we reach the end of this timeline or the client requests to
 	 * stop streaming.
@@ -1963,6 +2145,7 @@ InitWalSenderSlot(void)
 			walsnd->pid = MyProcPid;
 			walsnd->sentPtr = InvalidXLogRecPtr;
 			walsnd->state = WALSNDSTATE_STARTUP;
+			walsnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
 			walsnd->latch = &MyProc->procLatch;
 			SpinLockRelease(&walsnd->mutex);
 			/* don't need the lock anymore */
@@ -2732,6 +2915,24 @@ WalSndGetStateString(WalSndState state)
 	return "UNKNOWN";
 }
 
+/*
+ * Return a string constant representing the causal reads state. This is used
+ * in system views, and should *not* be translated.
+ */
+static const char *
+WalSndGetCausalReadsStateString(WalSndCausalReadsState causal_reads_state)
+{
+	switch (causal_reads_state)
+	{
+		case WALSNDCRSTATE_UNAVAILABLE:
+			return "unavailable";
+		case WALSNDCRSTATE_JOINING:
+			return "joining";
+		case WALSNDCRSTATE_AVAILABLE:
+			return "available";
+	}
+	return "UNKNOWN";
+}
 
 /*
  * Returns activity of walsenders, including pids and xlog locations sent to
@@ -2740,7 +2941,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	8
+#define PG_STAT_GET_WAL_SENDERS_COLS	10
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2788,8 +2989,10 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		XLogRecPtr	write;
 		XLogRecPtr	flush;
 		XLogRecPtr	apply;
+		int			applyLagMs;
 		int			priority;
 		WalSndState state;
+		WalSndCausalReadsState causalReadsState;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
 		bool		nulls[PG_STAT_GET_WAL_SENDERS_COLS];
 
@@ -2799,9 +3002,11 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		SpinLockAcquire(&walsnd->mutex);
 		sentPtr = walsnd->sentPtr;
 		state = walsnd->state;
+		causalReadsState = walsnd->causal_reads_state;
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
+		applyLagMs = walsnd->applyLagMs;
 		priority = walsnd->sync_standby_priority;
 		SpinLockRelease(&walsnd->mutex);
 
@@ -2833,6 +3038,23 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				nulls[5] = true;
 			values[5] = LSNGetDatum(apply);
 
+			if (applyLagMs < 0)
+				nulls[6] = true;
+			else
+			{
+				Interval *applyLagInterval = palloc(sizeof(Interval));
+
+				applyLagInterval->month = 0;
+				applyLagInterval->day = 0;
+#ifdef HAVE_INT64_TIMESTAMP
+				applyLagInterval->time = applyLagMs * 1000;
+#else
+				applyLagInterval->time = applyLagMs / 1000.0;
+#endif
+				nulls[6] = false;
+				values[6] = IntervalPGetDatum(applyLagInterval);
+			}
+
 			/*
 			 * Treat a standby such as a pg_basebackup background process
 			 * which always returns an invalid flush location, as an
@@ -2840,18 +3062,21 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 			 */
 			priority = XLogRecPtrIsInvalid(walsnd->flush) ? 0 : priority;
 
-			values[6] = Int32GetDatum(priority);
+			values[7] = Int32GetDatum(priority);
 
 			/*
 			 * More easily understood version of standby state. This is purely
 			 * informational, not different from priority.
 			 */
 			if (priority == 0)
-				values[7] = CStringGetTextDatum("async");
+				values[8] = CStringGetTextDatum("async");
 			else if (walsnd == sync_standby)
-				values[7] = CStringGetTextDatum("sync");
+				values[8] = CStringGetTextDatum("sync");
 			else
-				values[7] = CStringGetTextDatum("potential");
+				values[8] = CStringGetTextDatum("potential");
+
+			values[9] =
+				CStringGetTextDatum(WalSndGetCausalReadsStateString(causalReadsState));
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
@@ -2871,14 +3096,52 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 static void
 WalSndKeepalive(bool requestReply)
 {
+	TimestampTz now;
+	TimestampTz causal_reads_lease;
+
 	elog(DEBUG2, "sending replication keepalive");
 
+	/*
+	 * If the walsender currently deems the standby to be available for causal
+	 * reads, then it grants a causal reads lease.  The lease authorizes the
+	 * standby to consider itself available for causal reads until a short
+	 * time in the future.  The primary promises to uphold the causal reads
+	 * guarantee until that time, by stalling commits until the the lease has
+	 * expired if necessary.
+	 */
+	now = GetCurrentTimestamp();
+	if (MyWalSnd->causal_reads_state < WALSNDCRSTATE_AVAILABLE)
+		causal_reads_lease = 0; /* Not available, no lease granted. */
+	else
+	{
+		/*
+		 * Since this timestamp is being sent to the standby where it will be
+		 * compared against a time generated by the standby's system clock, we
+		 * must consider clock skew.  First, we decide on a maximum tolerable
+		 * difference between system clocks.  If the primary's clock is ahead
+		 * of the standby's by more than this, then all bets are off (the
+		 * standby could falsely believe it has a valid lease).  If the
+		 * primary's clock is behind the standby's by more than this, then the
+		 * standby will err the other way and generate spurious errors in
+		 * causal_reads mode.  Rather than having a separate GUC for this, we
+		 * derive it from causal_reads_timeout.
+		 */
+		int max_clock_skew = causal_reads_timeout / CAUSAL_READS_CLOCK_SKEW_RATIO;
+
+		/* Compute and remember the expiry time of the lease we're granting. */
+		causal_reads_last_lease = TimestampTzPlusMilliseconds(now, causal_reads_timeout);
+		/* The version we'll send to the standby is adjusted to tolerate clock skew. */
+		causal_reads_lease =
+			TimestampTzPlusMilliseconds(causal_reads_last_lease, -max_clock_skew);
+	}
+
 	/* construct the message... */
 	resetStringInfo(&output_message);
 	pq_sendbyte(&output_message, 'k');
 	pq_sendint64(&output_message, sentPtr);
-	pq_sendint64(&output_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(now));
 	pq_sendbyte(&output_message, requestReply ? 1 : 0);
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(causal_reads_lease));
 
 	/* ... and send it wrapped in CopyData */
 	pq_putmessage_noblock('d', output_message.data, output_message.len);
@@ -2896,23 +3159,32 @@ WalSndKeepaliveIfNecessary(TimestampTz now)
 	 * Don't send keepalive messages if timeouts are globally disabled or
 	 * we're doing something not partaking in timeouts.
 	 */
-	if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
+	if ((wal_sender_timeout <= 0 && causal_reads_timeout == 0) || last_reply_timestamp <= 0)
 		return;
 
-	if (waiting_for_ping_response)
+	if (waiting_for_ping_response && causal_reads_timeout == 0)
 		return;
 
 	/*
 	 * If half of wal_sender_timeout has lapsed without receiving any reply
 	 * from the standby, send a keep-alive message to the standby requesting
 	 * an immediate reply.
+	 *
+	 * If causal_reads_timeout has been configured, use it to control
+	 * keepalive intervals rather than wal_sender_timeout.
 	 */
-	ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-											wal_sender_timeout / 2);
+	if (causal_reads_timeout != 0)
+		ping_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+												causal_reads_timeout /
+												CAUSAL_READS_KEEPALIVE_RATIO);
+	else
+		ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+												wal_sender_timeout / 2);
 	if (now >= ping_time)
 	{
 		WalSndKeepalive(true);
 		waiting_for_ping_response = true;
+		last_keepalive_timestamp = now;
 
 		/* Try to flush pending output to the client */
 		if (pq_flush_if_writable() != 0)
diff --git a/src/backend/utils/adt/timestamp.c b/src/backend/utils/adt/timestamp.c
index 8fbb310..12c8b88 100644
--- a/src/backend/utils/adt/timestamp.c
+++ b/src/backend/utils/adt/timestamp.c
@@ -1611,6 +1611,20 @@ IntegerTimestampToTimestampTz(int64 timestamp)
 #endif
 
 /*
+ * TimestampTzToIntegerTimestamp -- convert a native timestamp to int64 format
+ *
+ * When compiled with --enable-integer-datetimes, this is implemented as a
+ * no-op macro.
+ */
+#ifndef HAVE_INT64_TIMESTAMP
+int64
+TimestampTzToIntegerTimestamp(TimestampTz timestamp)
+{
+	return timestamp * 1000000;
+}
+#endif
+
+/*
  * TimestampDifference -- convert the difference between two timestamps
  *		into integer seconds and microseconds
  *
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index a185749..6e0b144 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -351,6 +351,7 @@ static const struct config_enum_entry constraint_exclusion_options[] = {
 static const struct config_enum_entry synchronous_commit_options[] = {
 	{"local", SYNCHRONOUS_COMMIT_LOCAL_FLUSH, false},
 	{"remote_write", SYNCHRONOUS_COMMIT_REMOTE_WRITE, false},
+	{"apply", SYNCHRONOUS_COMMIT_REMOTE_APPLY, false},
 	{"on", SYNCHRONOUS_COMMIT_ON, false},
 	{"off", SYNCHRONOUS_COMMIT_OFF, false},
 	{"true", SYNCHRONOUS_COMMIT_ON, true},
@@ -1618,6 +1619,16 @@ static struct config_bool ConfigureNamesBool[] =
 		NULL, NULL, NULL
 	},
 
+	{
+		{"causal_reads", PGC_USERSET, REPLICATION_STANDBY,
+		 gettext_noop("Enables causal reads."),
+		 NULL
+		},
+		&causal_reads,
+		false,
+		NULL, NULL, NULL
+	},
+
 	/* End-of-list marker */
 	{
 		{NULL, 0, 0, NULL, NULL}, NULL, false, NULL, NULL, NULL
@@ -1776,6 +1787,17 @@ static struct config_int ConfigureNamesInt[] =
 	},
 
 	{
+		{"causal_reads_timeout", PGC_SIGHUP, REPLICATION_STANDBY,
+			gettext_noop("Sets the maximum apply lag before causal reads standbys are no longer available."),
+			NULL,
+			GUC_UNIT_MS
+		},
+		&causal_reads_timeout,
+		0, 0, INT_MAX,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"max_connections", PGC_POSTMASTER, CONN_AUTH_SETTINGS,
 			gettext_noop("Sets the maximum number of concurrent connections."),
 			NULL
@@ -3361,7 +3383,18 @@ static struct config_string ConfigureNamesString[] =
 		},
 		&SyncRepStandbyNames,
 		"",
-		check_synchronous_standby_names, NULL, NULL
+		check_standby_names, NULL, NULL
+	},
+
+	{
+		{"causal_reads_standby_names", PGC_SIGHUP, REPLICATION_MASTER,
+			gettext_noop("List of names of potential causal reads standbys."),
+			NULL,
+			GUC_LIST_INPUT
+		},
+		&causal_reads_standby_names,
+		"*",
+		check_standby_names, NULL, NULL
 	},
 
 	{
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index 074935c..a466732 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -46,8 +46,11 @@
 
 #include "access/transam.h"
 #include "access/xact.h"
+#include "access/xlog.h"
 #include "lib/pairingheap.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
+#include "replication/walreceiver.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
 #include "storage/procarray.h"
@@ -209,6 +212,16 @@ GetTransactionSnapshot(void)
 				 "cannot take query snapshot during a parallel operation");
 
 		/*
+		 * In causal_reads mode on a standby, check if we have definitely
+		 * applied WAL for any COMMIT that returned successfully on the
+		 * primary.
+		 *
+		 * TODO: Machine readable error code?
+		 */
+		if (causal_reads && RecoveryInProgress() && !WalRcvCausalReadsAvailable())
+			elog(ERROR, "standby is not available for causal reads");
+
+		/*
 		 * In transaction-snapshot mode, the first snapshot must live until
 		 * end of xact regardless of what the caller does with it, so we must
 		 * make a copy of it rather than returning CurrentSnapshotData
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index cb1c2db..0f08ff5 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -60,7 +60,11 @@ typedef enum
 	SYNCHRONOUS_COMMIT_LOCAL_FLUSH,		/* wait for local flush only */
 	SYNCHRONOUS_COMMIT_REMOTE_WRITE,	/* wait for local flush and remote
 										 * write */
-	SYNCHRONOUS_COMMIT_REMOTE_FLUSH		/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_FLUSH,	/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_APPLY,	/* wait for local flush and remote
+										 * apply */
+	SYNCHRONOUS_COMMIT_CONSISTENT_APPLY /* wait for local flusha and remote
+										   apply with causal consistency */
 }	SyncCommitLevel;
 
 /* Define the default setting for synchonous_commit */
@@ -144,10 +148,13 @@ typedef void (*SubXactCallback) (SubXactEvent event, SubTransactionId mySubid,
  * EOXact... routines which run at the end of the original transaction
  * completion.
  */
+#define XACT_COMPLETION_SYNC_APPLY_FEEDBACK		(1U << 29)
 #define XACT_COMPLETION_UPDATE_RELCACHE_FILE	(1U << 30)
 #define XACT_COMPLETION_FORCE_SYNC_COMMIT		(1U << 31)
 
 /* Access macros for above flags */
+#define XactCompletionSyncApplyFeedback(xinfo) \
+	(!!(xinfo & XACT_COMPLETION_SYNC_APPLY_FEEDBACK))
 #define XactCompletionRelcacheInitFileInval(xinfo) \
 	(!!(xinfo & XACT_COMPLETION_UPDATE_RELCACHE_FILE))
 #define XactCompletionForceSyncCommit(xinfo) \
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 790ca66..8aeda11 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -235,6 +235,9 @@ extern void GetXLogReceiptTime(TimestampTz *rtime, bool *fromStream);
 extern XLogRecPtr GetXLogReplayRecPtr(TimeLineID *replayTLI);
 extern XLogRecPtr GetXLogInsertRecPtr(void);
 extern XLogRecPtr GetXLogWriteRecPtr(void);
+extern void SetXLogReplayTimestamp(TimestampTz timestamp);
+extern void SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn);
+extern TimestampTz GetXLogReplayTimestamp(XLogRecPtr *lsn);
 extern bool RecoveryIsPaused(void);
 extern void SetRecoveryPause(bool recoveryPause);
 extern TimestampTz GetLatestXTime(void);
@@ -267,6 +270,8 @@ extern bool CheckPromoteSignal(void);
 extern void WakeupRecovery(void);
 extern void SetWalWriterSleeping(bool sleeping);
 
+extern void XLogRequestWalReceiverReply(void);
+
 extern void assign_max_wal_size(int newval, void *extra);
 extern void assign_checkpoint_completion_target(double newval, void *extra);
 
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index d8640db..acb6796 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2783,7 +2783,7 @@ DATA(insert OID = 1936 (  pg_stat_get_backend_idset		PGNSP PGUID 12 1 100 0 0 f
 DESCR("statistics: currently active backend IDs");
 DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f f f t s r 1 0 2249 "23" "{23,26,23,26,25,25,25,16,1184,1184,1184,1184,869,25,23,28,28,16,25,25,23,16,25}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{pid,datid,pid,usesysid,application_name,state,query,waiting,xact_start,query_start,backend_start,state_change,client_addr,client_hostname,client_port,backend_xid,backend_xmin,ssl,sslversion,sslcipher,sslbits,sslcompression,sslclientdn}" _null_ _null_ pg_stat_get_activity _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active backends");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,23,25}" "{o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25,25}" "{o,o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state,causal_reads_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 2026 (  pg_backend_pid				PGNSP PGUID 12 1 0 0 0 f f f f t f s r 0 0 23 "" _null_ _null_ _null_ _null_ _null_ pg_backend_pid _null_ _null_ _null_ ));
 DESCR("statistics: current backend PID");
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index 71e2857..6a090b7 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -23,14 +23,34 @@
 #define SYNC_REP_NO_WAIT		-1
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
+#define SYNC_REP_WAIT_APPLY		2
+#define SYNC_REP_WAIT_CAUSAL_READS_APPLY 3
 
-#define NUM_SYNC_REP_WAIT_MODE	2
+#define NUM_SYNC_REP_WAIT_MODE	4
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
 #define SYNC_REP_WAITING			1
 #define SYNC_REP_WAIT_COMPLETE		2
 
+/*
+ * ratio of causal_read_timeout to max_clock_skew (4 means than the maximum
+ * tolerated clock difference between primary and standbys using causal_reads
+ * is 1/4 of causal_reads_timeout)
+ */
+#define CAUSAL_READS_CLOCK_SKEW_RATIO 4
+
+/*
+ * ratio of causal_reads_timeout to keepalive time (2 means that the effective
+ * keepalive time is 1/2 of the causal_reads_timeout GUC when it is non-zero)
+ */
+#define CAUSAL_READS_KEEPALIVE_RATIO 2
+
+/* GUC variables */
+extern int causal_reads_timeout;
+extern bool causal_reads;
+extern char *causal_reads_standby_names;
+
 /* user-settable parameters for synchronous replication */
 extern char *SyncRepStandbyNames;
 
@@ -42,16 +62,23 @@ extern void SyncRepCleanupAtProcExit(void);
 
 /* called by wal sender */
 extern void SyncRepInitConfig(void);
-extern void SyncRepReleaseWaiters(void);
+extern void SyncRepReleaseWaiters(bool walsender_cr_available_or_joining);
 
 /* called by checkpointer */
 extern void SyncRepUpdateSyncStandbysDefined(void);
 
+/* called by user backend (xact.c) */
+extern void CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN);
+
+/* called by wal sender */
+extern void CausalReadsBeginStall(TimestampTz lease_expiry_time);
+extern bool CausalReadsPotentialStandby(void);
+
 /* forward declaration to avoid pulling in walsender_private.h */
 struct WalSnd;
 extern struct WalSnd *SyncRepGetSynchronousStandby(void);
 
-extern bool check_synchronous_standby_names(char **newval, void **extra, GucSource source);
+extern bool check_standby_names(char **newval, void **extra, GucSource source);
 extern void assign_synchronous_commit(int newval, void *extra);
 
 #endif   /* _SYNCREP_H */
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 61255a9..40d99e6 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -79,6 +79,13 @@ typedef struct
 	TimeLineID	receivedTLI;
 
 	/*
+	 * causallyReadsLease is the time until which the primary has authorized
+	 * this standby to consider itself available for causal_reads mode, or 0
+	 * for not authorized.
+	 */
+	TimestampTz causalReadsLease;
+
+	/*
 	 * latestChunkStart is the starting byte position of the current "batch"
 	 * of received WAL.  It's actually the same as the previous value of
 	 * receivedUpto before the last flush to disk.  Startup process can use
@@ -160,5 +167,8 @@ extern void RequestXLogStreaming(TimeLineID tli, XLogRecPtr recptr,
 extern XLogRecPtr GetWalRcvWriteRecPtr(XLogRecPtr *latestChunkStart, TimeLineID *receiveTLI);
 extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
+extern void WalRcvWakeup(void);
+
+extern bool WalRcvCausalReadsAvailable(void);
 
 #endif   /* _WALRECEIVER_H */
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 6dae480..88b4fe9 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -27,6 +27,13 @@ typedef enum WalSndState
 	WALSNDSTATE_STREAMING
 } WalSndState;
 
+typedef enum WalSndCausalReadsState
+{
+	WALSNDCRSTATE_UNAVAILABLE = 0,
+	WALSNDCRSTATE_JOINING,
+	WALSNDCRSTATE_AVAILABLE
+} WalSndCausalReadsState;
+
 /*
  * Each walsender has a WalSnd struct in shared memory.
  */
@@ -34,6 +41,7 @@ typedef struct WalSnd
 {
 	pid_t		pid;			/* this walsender's process id, or 0 */
 	WalSndState state;			/* this walsender's state */
+	WalSndCausalReadsState causal_reads_state; /* the walsender's causal reads state */
 	XLogRecPtr	sentPtr;		/* WAL has been sent up to this point */
 	bool		needreload;		/* does currently-open file need to be
 								 * reloaded? */
@@ -46,6 +54,7 @@ typedef struct WalSnd
 	XLogRecPtr	write;
 	XLogRecPtr	flush;
 	XLogRecPtr	apply;
+	int			applyLagMs;
 
 	/* Protects shared variables shown above. */
 	slock_t		mutex;
@@ -88,6 +97,12 @@ typedef struct
 	 */
 	bool		sync_standbys_defined;
 
+	/*
+	 * Until when must commits in causal_reads stall?  This is used to wait
+	 * for causal reads leases to expire.
+	 */
+	TimestampTz	stall_causal_reads_until;
+
 	WalSnd		walsnds[FLEXIBLE_ARRAY_MEMBER];
 } WalSndCtlData;
 
diff --git a/src/include/utils/timestamp.h b/src/include/utils/timestamp.h
index 530fef1..0f4b166 100644
--- a/src/include/utils/timestamp.h
+++ b/src/include/utils/timestamp.h
@@ -227,9 +227,11 @@ extern bool TimestampDifferenceExceeds(TimestampTz start_time,
 #ifndef HAVE_INT64_TIMESTAMP
 extern int64 GetCurrentIntegerTimestamp(void);
 extern TimestampTz IntegerTimestampToTimestampTz(int64 timestamp);
+extern int64 TimestampTzToIntegerTimestamp(TimestampTz timestamp);
 #else
 #define GetCurrentIntegerTimestamp()	GetCurrentTimestamp()
 #define IntegerTimestampToTimestampTz(timestamp) (timestamp)
+#define TimestampTzToIntegerTimestamp(timestamp) (timestamp)
 #endif
 
 extern TimestampTz time_t_to_timestamptz(pg_time_t tm);
#24Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Thomas Munro (#23)
1 attachment(s)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Wed, Nov 18, 2015 at 11:50 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

Here is a new version of the patch with a few small improvements:
...
[causal-reads-v3.patch]

That didn't apply after 6e7b3359 (which fix a typo in a comment that I
moved). Here is a new version that does.

--
Thomas Munro
http://www.enterprisedb.com

Attachments:

causal-reads-v4.patchapplication/octet-stream; name=causal-reads-v4.patchDownload
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 47312f6..535b5a9 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,10 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
+	{
+		CausalReadsWaitForLSN(XactLastRecEnd);
 		SyncRepWaitForLSN(XactLastRecEnd);
+	}
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5117,6 +5120,13 @@ XactLogCommitRecord(TimestampTz commit_time,
 		xl_xinfo.xinfo |= XACT_COMPLETION_FORCE_SYNC_COMMIT;
 
 	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this commit is applied.
+	 */
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY || causal_reads)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
+	/*
 	 * Relcache invalidations requires information about the current database
 	 * and so does logical decoding.
 	 */
@@ -5452,6 +5462,19 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 	if (XactCompletionForceSyncCommit(parsed->xinfo))
 		XLogFlush(lsn);
 
+	/*
+	 * Record the primary's timestamp for the commit record, so it can be used
+	 * for tracking replay lag.
+	 */
+	SetXLogReplayTimestamp(parsed->xact_time);
+
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit or causal reads), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 /*
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index ce2e074..d5876b2 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -80,6 +80,8 @@ extern uint32 bootstrap_data_checksum_version;
 #define PROMOTE_SIGNAL_FILE		"promote"
 #define FALLBACK_PROMOTE_SIGNAL_FILE "fallback_promote"
 
+/* Size of the circular buffer of timestamped LSNs. */
+#define MAX_TIMESTAMPED_LSNS 8192
 
 /* User-settable parameters */
 int			max_wal_size = 64;	/* 1 GB */
@@ -346,6 +348,12 @@ static XLogRecPtr RedoRecPtr;
 static bool doPageWrites;
 
 /*
+ * doRequestWalReceiverReply is used by recovery code to ask the main recovery
+ * loop to trigger a walreceiver reply.
+ */
+static bool doRequestWalReceiverReply;
+
+/*
  * RedoStartLSN points to the checkpoint's REDO location which is specified
  * in a backup label file, backup history file or control file. In standby
  * mode, XLOG streaming usually starts from the position where an invalid
@@ -357,6 +365,13 @@ static bool doPageWrites;
  */
 static XLogRecPtr RedoStartLSN = InvalidXLogRecPtr;
 
+/*
+ * LastReplayedTimestamp can be set by redo handlers when they apply a record
+ * that carries a timestamp, by calling SetXLogReplayedTimestamp.  The xlog
+ * apply loop can then update the value in shared memory.
+ */
+static TimestampTz LastReplayedTimestamp = 0;
+
 /*----------
  * Shared-memory data structures for XLOG control
  *
@@ -631,6 +646,21 @@ typedef struct XLogCtlData
 	/* current effective recovery target timeline */
 	TimeLineID	RecoveryTargetTLI;
 
+	/* timestamp from the most recently applied record that carried a timestamp. */
+	TimestampTz lastReplayedTimestamp;
+
+	/*
+	 * We maintain a circular buffer of LSNs and associated timestamps.
+	 * Walreceiver writes into it using information from timestamps, and the
+	 * startup recovery process reads from it and notifies walreceiver when
+	 * LSNs are replayed so that the timestamps can be fed back to the
+	 * upstream server, to track lag.
+	 */
+	Index		timestampedLsnRead;
+	Index		timestampedLsnWrite;
+	XLogRecPtr	timestampedLsn[MAX_TIMESTAMPED_LSNS];
+	TimestampTz	timestampedLsnTime[MAX_TIMESTAMPED_LSNS];
+
 	/*
 	 * timestamp of when we started replaying the current chunk of WAL data,
 	 * only relevant for replication or archive recovery
@@ -6850,14 +6880,58 @@ StartupXLOG(void)
 				error_context_stack = errcallback.previous;
 
 				/*
-				 * Update lastReplayedEndRecPtr after this record has been
-				 * successfully replayed.
+				 * Update lastReplayedEndRecPtr and lastReplayedTimestamp
+				 * after this record has been successfully replayed.
 				 */
 				SpinLockAcquire(&XLogCtl->info_lck);
 				XLogCtl->lastReplayedEndRecPtr = EndRecPtr;
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
+				if (LastReplayedTimestamp != 0)
+				{
+					/* If replaying a record produced a timestamp, use that. */
+					XLogCtl->lastReplayedTimestamp = LastReplayedTimestamp;
+					LastReplayedTimestamp = 0;
+				}
+				else
+				{
+					/*
+					 * If we have applied LSNs associated with timestamps
+					 * received by walreceiver, then use the recorded
+					 * timestamp.  We consume from the read end of the
+					 * circular buffer.
+					 */
+					while (XLogCtl->timestampedLsnRead !=
+						   XLogCtl->timestampedLsnWrite &&
+						   XLogCtl->timestampedLsn[XLogCtl->timestampedLsnRead]
+						   <= EndRecPtr)
+					{
+						if (XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead] >
+							XLogCtl->lastReplayedTimestamp)
+						{
+							XLogCtl->lastReplayedTimestamp =
+								XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead];
+							doRequestWalReceiverReply = true;
+						}
+						XLogCtl->timestampedLsnRead =
+							(XLogCtl->timestampedLsnRead + 1) % MAX_TIMESTAMPED_LSNS;
+					}
+				}
 				SpinLockRelease(&XLogCtl->info_lck);
 
+				/*
+				 * If rm_redo reported that it applied a commit record that
+				 * the master is waiting for by calling
+				 * XLogRequestWalReceiverReply, or we encountered a WAL
+				 * location that was associated with a timestamp above, then
+				 * we wake up the receiver so that it notices the updated
+				 * lastReplayedEndRecPtr and sends a reply to the master.
+				 */
+				if (doRequestWalReceiverReply)
+				{
+					doRequestWalReceiverReply = false;
+					WalRcvWakeup();
+				}
+
 				/* Remember this record as the last-applied one */
 				LastRec = ReadRecPtr;
 
@@ -11585,3 +11659,103 @@ SetWalWriterSleeping(bool sleeping)
 	XLogCtl->WalWriterSleeping = sleeping;
 	SpinLockRelease(&XLogCtl->info_lck);
 }
+
+/*
+ * Called by redo code to indicate that the xlog replay loop should wake up
+ * the walreceiver process so that a reply can be sent to the primary.
+ */
+void
+XLogRequestWalReceiverReply(void)
+{
+	doRequestWalReceiverReply = true;
+}
+
+/*
+ * Merge timestamps from keepalive messages with the timestamps from WAL
+ * records, so that we can track lag while idle or while replaying large
+ * amounts of WAL without commit records.  In the former case there is no lag,
+ * and in the latter case we will remember a timestamp that goes with an
+ * arbitrary LSN, and wait for that LSN to be replayed before using the
+ * timestamp.
+ *
+ * This is called by walreceiver on standby servers when keepalive messages
+ * arrive.
+ */
+void
+SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn)
+{
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn == XLogCtl->lastReplayedEndRecPtr)
+	{
+		/*
+		 * That is the last replayed LSN: we are fully replayed, so we can
+		 * update the replay timestamp immediately.
+		 */
+		XLogCtl->lastReplayedTimestamp = timestamp;
+	}
+	else
+	{
+		/*
+		 * There is WAL still to be applied.  We will associate the timestamp
+		 * with this WAL position and wait for it to be replayed.  We add it
+		 * at the 'write' end of the circular buffer of LSN/timestamp
+		 * mappings, which the replay loop will eventually read.
+		 */
+		Index w = XLogCtl->timestampedLsnWrite;
+		Index r = XLogCtl->timestampedLsnRead;
+
+		XLogCtl->timestampedLsn[w] = lsn;
+		XLogCtl->timestampedLsnTime[w] = timestamp;
+
+		/* Advance the write point. */
+		w = (w + 1) % MAX_TIMESTAMPED_LSNS;
+		XLogCtl->timestampedLsnWrite = w;
+		if (w == r)
+		{
+			/*
+			 * The buffer is full.  Advance the read point (throwing away
+			 * oldest values; we will begin to overestimate replay lag, until
+			 * lag decreases to a size our buffer can manage, or the next
+			 * commit record is replayed).
+			 */
+			r = (r + 1) % MAX_TIMESTAMPED_LSNS;
+			XLogCtl->timestampedLsnRead = r;
+		}
+	}
+	SpinLockRelease(&XLogCtl->info_lck);
+}
+
+/*
+ * Set the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary.  This can be called by redo handlers that have
+ * an appropriate timestamp (currently only commit records).  Updating the
+ * shared memory value is deferred until after the redo handler returns.
+ */
+void
+SetXLogReplayTimestamp(TimestampTz timestamp)
+{
+	LastReplayedTimestamp = timestamp;
+}
+
+/*
+ * Get the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the master, and also the most recently applied LSN.  (Note
+ * that the timestamp and the LSN don't necessarily relate to the same
+ * record.)
+ *
+ * This is similar to GetLatestXTime, except that it is not only advanced by
+ * commit records (see SetXLogReplayTimestampAtLsn).
+ */
+TimestampTz
+GetXLogReplayTimestamp(XLogRecPtr *lsn)
+{
+	TimestampTz result;
+
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn)
+		*lsn = XLogCtl->lastReplayedEndRecPtr;
+	result = XLogCtl->lastReplayedTimestamp;
+	SpinLockRelease(&XLogCtl->info_lck);
+
+	return result;
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 536c805..13cca9f 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -655,8 +655,10 @@ CREATE VIEW pg_stat_replication AS
             W.write_location,
             W.flush_location,
             W.replay_location,
+            W.replay_lag,
             W.sync_priority,
-            W.sync_state
+            W.sync_state,
+            W.causal_reads_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
             pg_stat_get_wal_senders() AS W
     WHERE S.usesysid = U.oid AND
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 1bc584f..adaf466 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -57,6 +57,11 @@
 #include "utils/builtins.h"
 #include "utils/ps_status.h"
 
+/* GUC variables */
+int causal_reads_timeout;
+bool causal_reads;
+char *causal_reads_standby_names;
+
 /* User-settable parameters for sync rep */
 char	   *SyncRepStandbyNames;
 
@@ -69,7 +74,7 @@ static int	SyncRepWaitMode = SYNC_REP_NO_WAIT;
 
 static void SyncRepQueueInsert(int mode);
 static void SyncRepCancelWait(void);
-static int	SyncRepWakeQueue(bool all, int mode);
+static int	SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn);
 
 static int	SyncRepGetStandbyPriority(void);
 
@@ -83,6 +88,239 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * ===========================================================
  */
 
+static bool
+SyncRepCheckEarlyExit(void)
+{
+	/*
+	 * If a wait for synchronous replication is pending, we can neither
+	 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
+	 * lead the client to believe that the transaction aborted, which
+	 * is not true: it's already committed locally. The former is no good
+	 * either: the client has requested synchronous replication, and is
+	 * entitled to assume that an acknowledged commit is also replicated,
+	 * which might not be true. So in this case we issue a WARNING (which
+	 * some clients may be able to interpret) and shut off further output.
+	 * We do NOT reset ProcDiePending, so that the process will die after
+	 * the commit is cleaned up.
+	 */
+	if (ProcDiePending)
+	{
+		ereport(WARNING,
+				(errcode(ERRCODE_ADMIN_SHUTDOWN),
+				 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * It's unclear what to do if a query cancel interrupt arrives.  We
+	 * can't actually abort at this point, but ignoring the interrupt
+	 * altogether is not helpful, so we just terminate the wait with a
+	 * suitable warning.
+	 */
+	if (QueryCancelPending)
+	{
+		QueryCancelPending = false;
+		ereport(WARNING,
+				(errmsg("canceling wait for synchronous replication due to user request"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * If the postmaster dies, we'll probably never get an
+	 * acknowledgement, because all the wal sender processes will exit. So
+	 * just bail out.
+	 */
+	if (!PostmasterIsAlive())
+	{
+		ProcDiePending = true;
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	return false;
+}
+
+/*
+ * Check if we can stop waiting for causal consistency.  We can stop waiting
+ * when the following conditions are met:
+ *
+ * 1.  All walsenders currently in 'joining' or 'available' state have
+ * applied the target LSN.
+ *
+ * 2.  Any stall periods caused by standbys dropping out of 'available' state
+ * have passed, so that we can be sure that their leases have expired and they
+ * have started rejecting causal reads transactions.
+ *
+ * The output parameter 'waitingFor' is set to the number of nodes we are
+ * currently waiting for.  The output parameters 'stallTimeMillis' is set to
+ * the number of milliseconds we need to wait for to observe any current
+ * commit stall.
+ *
+ * Returns true if commit can return control, because every standby has either
+ * applied the LSN or started rejecting causal_reads transactions.
+ */
+static bool
+CausalReadsCommitCanReturn(XLogRecPtr XactCommitLSN,
+						   int *waitingFor,
+						   long *stallTimeMillis)
+{
+	int i;
+	TimestampTz now;
+
+	/* Count how many joining/available nodes we are waiting for. */
+	*waitingFor = 0;
+	for (i = 0; i < max_wal_senders; ++i)
+	{
+		WalSnd *walsnd = &WalSndCtl->walsnds[i];
+
+		/*
+		 * Assuming atomic read of pid_t, we can check walsnd->pid without
+		 * acquiring the spinlock to avoid memory synchronization costs for
+		 * unused walsender slots.  We see a value that existed sometime at
+		 * least as recently as the last memory barrier.
+		 */
+		if (walsnd->pid != 0)
+		{
+			/*
+			 * We need to hold the spinlock to read LSNs, because we can't be
+			 * sure they can be read atomically.
+			 */
+			SpinLockAcquire(&walsnd->mutex);
+			if (walsnd->pid != 0 && walsnd->causal_reads_state >= WALSNDCRSTATE_JOINING)
+			{
+				if (walsnd->apply < XactCommitLSN)
+					++*waitingFor;
+			}
+			SpinLockRelease(&walsnd->mutex);
+		}
+	}
+
+	/* Check if there is a stall in progress that we need to observe. */
+	now = GetCurrentTimestamp();
+	LWLockAcquire(SyncRepLock, LW_SHARED);
+	if (WalSndCtl->stall_causal_reads_until > now)
+	{
+		long seconds;
+		int usecs;
+
+		/* Compute how long we have to wait, rounded up to nearest ms. */
+		TimestampDifference(now, WalSndCtl->stall_causal_reads_until,
+							&seconds, &usecs);
+		*stallTimeMillis = seconds * 1000 + (usecs + 999) / 1000;
+	}
+	else
+		*stallTimeMillis = 0;
+	LWLockRelease(SyncRepLock);
+
+	/* We are done if we are not waiting for any nodes or stalls. */
+	return *waitingFor == 0 && *stallTimeMillis == 0;
+}
+
+/*
+ * Wait for causal consistency in causal_reads mode, if requested by user.
+ */
+void
+CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN)
+{
+	long stallTimeMillis;
+	int waitingFor;
+
+	/* Leave if we aren't in causal_reads mode. */
+	if (!causal_reads)
+		return;
+
+	for (;;)
+	{
+		/* Reset latch before checking state. */
+		ResetLatch(MyLatch);
+
+		/*
+		 * Join the queue to be woken up if any causal reads joining/available
+		 * standby applies XactCommitLSN, if we aren't already in it.  We
+		 * don't actually know if we need to wait for any peers yet, but we
+		 * have to register just in case before checking the walsenders' state
+		 * to avoid a race condition that could occur if we did it after
+		 * calling CausalReadsCommitCanReturn.  (SyncRepWaitForLSN doesn't
+		 * have to do this because it can check the highest-seen LSN in
+		 * walsndctl->lsn[mode] which is protected by SyncRepLock, the same
+		 * lock as the queues.  We can't do that here, because there is no
+		 * single highest-seen LSN that is useful.  We must check
+		 * walsnd->apply for all relevant walsenders.  Therefore we must
+		 * register for notifications first, so that we can be notified via
+		 * our latch of any standby applying the LSN we're interested in after
+		 * we check but before we start waiting, or we could wait forever for
+		 * something that has already happened.)
+		 */
+		LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+		if (MyProc->syncRepState != SYNC_REP_WAITING)
+		{
+			MyProc->waitLSN = XactCommitLSN;
+			MyProc->syncRepState = SYNC_REP_WAITING;
+			SyncRepQueueInsert(SYNC_REP_WAIT_CAUSAL_READS_APPLY);
+			Assert(SyncRepQueueIsOrderedByLSN(SYNC_REP_WAIT_CAUSAL_READS_APPLY));
+		}
+		LWLockRelease(SyncRepLock);
+
+		/* Check if we're done. */
+		if (CausalReadsCommitCanReturn(XactCommitLSN, &waitingFor, &stallTimeMillis))
+		{
+			SyncRepCancelWait();
+			break;
+		}
+
+		Assert(waitingFor > 0 || stallTimeMillis > 0);
+
+		/* If we aren't actually waiting for any standbys, leave the queue. */
+		if (waitingFor == 0)
+			SyncRepCancelWait();
+
+		/* Update the ps title. */
+		if (update_process_title)
+		{
+			char buffer[80];
+
+			snprintf(buffer, sizeof(buffer),
+					 "waiting for %d peer(s) to apply %X/%X%s",
+					 waitingFor,
+					 (uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN,
+					 stallTimeMillis > 0 ? " (stalling)" : "");
+			set_ps_display(buffer, false);
+		}
+
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit()) /* Calls SyncRepCancelWait() if true. */
+			break;
+
+		/*
+		 * If are still waiting for peers, then we wait for any joining or
+		 * available peer to reach the LSN (or possibly stop being in one of
+		 * those states or go away).
+		 *
+		 * If not, there must be a non-zero stall time, so we wait for that to
+		 * elapse.
+		 */
+		if (waitingFor > 0)
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH, -1);
+		else
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH | WL_TIMEOUT,
+					  stallTimeMillis);
+	}
+
+	/* There is no way out of the loop that could leave us in the queue. */
+	Assert(SHMQueueIsDetached(&(MyProc->syncRepLinks)));
+	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
+	MyProc->waitLSN = 0;
+
+	if (update_process_title)
+		set_ps_display("", false); /* TODO: restore what was there */
+}
+
 /*
  * Wait for synchronous replication, if requested by user.
  *
@@ -180,57 +418,9 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 		if (syncRepState == SYNC_REP_WAIT_COMPLETE)
 			break;
 
-		/*
-		 * If a wait for synchronous replication is pending, we can neither
-		 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
-		 * lead the client to believe that the transaction aborted, which
-		 * is not true: it's already committed locally. The former is no good
-		 * either: the client has requested synchronous replication, and is
-		 * entitled to assume that an acknowledged commit is also replicated,
-		 * which might not be true. So in this case we issue a WARNING (which
-		 * some clients may be able to interpret) and shut off further output.
-		 * We do NOT reset ProcDiePending, so that the process will die after
-		 * the commit is cleaned up.
-		 */
-		if (ProcDiePending)
-		{
-			ereport(WARNING,
-					(errcode(ERRCODE_ADMIN_SHUTDOWN),
-					 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * It's unclear what to do if a query cancel interrupt arrives.  We
-		 * can't actually abort at this point, but ignoring the interrupt
-		 * altogether is not helpful, so we just terminate the wait with a
-		 * suitable warning.
-		 */
-		if (QueryCancelPending)
-		{
-			QueryCancelPending = false;
-			ereport(WARNING,
-					(errmsg("canceling wait for synchronous replication due to user request"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * If the postmaster dies, we'll probably never get an
-		 * acknowledgement, because all the wal sender processes will exit. So
-		 * just bail out.
-		 */
-		if (!PostmasterIsAlive())
-		{
-			ProcDiePending = true;
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit())
 			break;
-		}
 
 		/*
 		 * Wait on latch.  Any condition that should wake us up will set the
@@ -403,6 +593,49 @@ SyncRepGetSynchronousStandby(void)
 }
 
 /*
+ * Check if the current WALSender process's application_name matches a name in
+ * causal_reads_standby_names (including '*' for wildcard).
+ */
+bool
+CausalReadsPotentialStandby(void)
+{
+	char *rawstring;
+	List	   *elemlist;
+	ListCell   *l;
+	bool		found = false;
+
+	/* Need a modifiable copy of string */
+	rawstring = pstrdup(causal_reads_standby_names);
+
+	/* Parse string into list of identifiers */
+	if (!SplitIdentifierString(rawstring, ',', &elemlist))
+	{
+		/* syntax error in list */
+		pfree(rawstring);
+		list_free(elemlist);
+		/* GUC machinery will have already complained - no need to do again */
+		return 0;
+	}
+
+	foreach(l, elemlist)
+	{
+		char	   *standby_name = (char *) lfirst(l);
+
+		if (pg_strcasecmp(standby_name, application_name) == 0 ||
+			pg_strcasecmp(standby_name, "*") == 0)
+		{
+			found = true;
+			break;
+		}
+	}
+
+	pfree(rawstring);
+	list_free(elemlist);
+
+	return found;
+}
+
+/*
  * Update the LSNs on each queue based upon our latest state. This
  * implements a simple policy of first-valid-standby-releases-waiter.
  *
@@ -410,22 +643,27 @@ SyncRepGetSynchronousStandby(void)
  * perhaps also which information we store as well.
  */
 void
-SyncRepReleaseWaiters(void)
+SyncRepReleaseWaiters(bool walsender_cr_available_or_joining)
 {
 	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
+	int			numapply = 0;
+	int			numcausalreadsapply = 0;
+	bool		is_highest_priority_sync_standby;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
-	 * potential standbys then we have nothing to do. If we are still starting
-	 * up, still running base backup or the current flush position is still
-	 * invalid, then leave quickly also.
+	 * potential standbys and not in a state that causal_reads waits for, then
+	 * we have nothing to do. If we are still starting up, still running base
+	 * backup or the current flush position is still invalid, then leave
+	 * quickly also.
 	 */
-	if (MyWalSnd->sync_standby_priority == 0 ||
-		MyWalSnd->state < WALSNDSTATE_STREAMING ||
-		XLogRecPtrIsInvalid(MyWalSnd->flush))
+	if (!walsender_cr_available_or_joining &&
+		(MyWalSnd->sync_standby_priority == 0 ||
+		 MyWalSnd->state < WALSNDSTATE_STREAMING ||
+		 XLogRecPtrIsInvalid(MyWalSnd->flush)))
 		return;
 
 	/*
@@ -435,45 +673,77 @@ SyncRepReleaseWaiters(void)
 	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
 	syncWalSnd = SyncRepGetSynchronousStandby();
 
-	/* We should have found ourselves at least */
-	Assert(syncWalSnd != NULL);
+	/*
+	 * If we aren't managing the highest priority standby then make a note of
+	 * that so we can announce a takeover in the log if we ever get that job.
+	 */
+	is_highest_priority_sync_standby = syncWalSnd == MyWalSnd;
+	if (!is_highest_priority_sync_standby)
+		announce_next_takeover = true;
 
 	/*
-	 * If we aren't managing the highest priority standby then just leave.
+	 * If we aren't managing the highest priority standby or a standby in
+	 * causal reads 'joining' or 'available' state, then just leave.
 	 */
-	if (syncWalSnd != MyWalSnd)
+	if (!is_highest_priority_sync_standby && !walsender_cr_available_or_joining)
 	{
 		LWLockRelease(SyncRepLock);
-		announce_next_takeover = true;
 		return;
 	}
 
 	/*
 	 * Set the lsn first so that when we wake backends they will release up to
-	 * this location.
+	 * this location.  For the single-standby synchronous commit levels, we
+	 * only do this if we are the current synchronous standby and we are
+	 * advancing the LSN further than it has been advanced before, so that
+	 * SyncRepWaitForLSN can skip waiting in some cases.
 	 */
-	if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+	if (is_highest_priority_sync_standby)
 	{
-		walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
-		numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
-		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
+		if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
+			numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE,
+										MyWalSnd->write);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->write)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
+			numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH,
+										MyWalSnd->flush);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+			numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY,
+										MyWalSnd->apply);
+		}
 	}
+	/*
+	 * For causal_reads, all walsenders currently in available or joining
+	 * state must reach the LSN on their own, and standbys will reach LSNs in
+	 * any order.  It doesn't make sense to keep the highest seen LSN in a
+	 * single walsndctl->lsn element.  (CausalReadsWaitForLSN has handling for
+	 * LSNs that have already been reached).
+	 */
+	if (walsender_cr_available_or_joining)
+		numcausalreadsapply =
+			SyncRepWakeQueue(false, SYNC_REP_WAIT_CAUSAL_READS_APPLY,
+							 MyWalSnd->apply);
 
 	LWLockRelease(SyncRepLock);
 
-	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X",
+	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%X, %d procs to causal_reads apply",
 		 numwrite, (uint32) (MyWalSnd->write >> 32), (uint32) MyWalSnd->write,
-	   numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush);
+		 numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush,
+		 numapply, (uint32) (MyWalSnd->apply >> 32), (uint32) MyWalSnd->apply,
+		 numcausalreadsapply);
 
 	/*
 	 * If we are managing the highest priority standby, though we weren't
 	 * prior to this, then announce we are now the sync standby.
 	 */
-	if (announce_next_takeover)
+	if (is_highest_priority_sync_standby && announce_next_takeover)
 	{
 		announce_next_takeover = false;
 		ereport(LOG,
@@ -548,9 +818,8 @@ SyncRepGetStandbyPriority(void)
  * Must hold SyncRepLock.
  */
 static int
-SyncRepWakeQueue(bool all, int mode)
+SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn)
 {
-	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	PGPROC	   *proc = NULL;
 	PGPROC	   *thisproc = NULL;
 	int			numprocs = 0;
@@ -567,7 +836,7 @@ SyncRepWakeQueue(bool all, int mode)
 		/*
 		 * Assume the queue is ordered by LSN
 		 */
-		if (!all && walsndctl->lsn[mode] < proc->waitLSN)
+		if (!all && lsn < proc->waitLSN)
 			return numprocs;
 
 		/*
@@ -627,7 +896,7 @@ SyncRepUpdateSyncStandbysDefined(void)
 			int			i;
 
 			for (i = 0; i < NUM_SYNC_REP_WAIT_MODE; i++)
-				SyncRepWakeQueue(true, i);
+				SyncRepWakeQueue(true, i, InvalidXLogRecPtr);
 		}
 
 		/*
@@ -679,13 +948,31 @@ SyncRepQueueIsOrderedByLSN(int mode)
 #endif
 
 /*
+ * Make sure that CausalReadsWaitForLSN can't return until after the given
+ * lease expiry time has been reached.
+ *
+ * Wake up all backends waiting in CausalReadsWaitForLSN, because the set of
+ * available/joining peers has changed, and there is a new stall time they
+ * need to observe.
+ */
+void
+CausalReadsBeginStall(TimestampTz lease_expiry_time)
+{
+	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+	WalSndCtl->stall_causal_reads_until =
+		Max(WalSndCtl->stall_causal_reads_until, lease_expiry_time);
+	SyncRepWakeQueue(true, SYNC_REP_WAIT_CAUSAL_READS_APPLY, InvalidXLogRecPtr);
+	LWLockRelease(SyncRepLock);
+}
+
+/*
  * ===========================================================
  * Synchronous Replication functions executed by any process
  * ===========================================================
  */
 
 bool
-check_synchronous_standby_names(char **newval, void **extra, GucSource source)
+check_standby_names(char **newval, void **extra, GucSource source)
 {
 	char	   *rawstring;
 	List	   *elemlist;
@@ -728,6 +1015,9 @@ assign_synchronous_commit(int newval, void *extra)
 		case SYNCHRONOUS_COMMIT_REMOTE_FLUSH:
 			SyncRepWaitMode = SYNC_REP_WAIT_FLUSH;
 			break;
+		case SYNCHRONOUS_COMMIT_REMOTE_APPLY:
+			SyncRepWaitMode = SYNC_REP_WAIT_APPLY;
+			break;
 		default:
 			SyncRepWaitMode = SYNC_REP_NO_WAIT;
 			break;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 183a3a5..ab61b3f 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -52,6 +52,7 @@
 #include "libpq/pqformat.h"
 #include "libpq/pqsignal.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "replication/walsender.h"
 #include "storage/ipc.h"
@@ -96,6 +97,7 @@ static uint32 recvOff = 0;
  */
 static volatile sig_atomic_t got_SIGHUP = false;
 static volatile sig_atomic_t got_SIGTERM = false;
+static volatile sig_atomic_t got_SIGUSR2 = false;
 
 /*
  * LogstreamResult indicates the byte positions that we have already
@@ -140,14 +142,33 @@ static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
 static void XLogWalRcvSendReply(bool force, bool requestReply);
 static void XLogWalRcvSendHSFeedback(bool immed);
-static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
+static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+								  TimestampTz *causalReadsUntil);
 
 /* Signal handlers */
 static void WalRcvSigHupHandler(SIGNAL_ARGS);
 static void WalRcvSigUsr1Handler(SIGNAL_ARGS);
+static void WalRcvSigUsr2Handler(SIGNAL_ARGS);
 static void WalRcvShutdownHandler(SIGNAL_ARGS);
 static void WalRcvQuickDieHandler(SIGNAL_ARGS);
 
+static void WalRcvBlockSigUsr2(void)
+{
+	sigset_t mask;
+
+	sigemptyset(&mask);
+	sigaddset(&mask, SIGUSR2);
+	sigprocmask(SIG_BLOCK, &mask, NULL);
+}
+
+static void WalRcvUnblockSigUsr2(void)
+{
+	sigset_t mask;
+
+	sigemptyset(&mask);
+	sigaddset(&mask, SIGUSR2);
+	sigprocmask(SIG_UNBLOCK, &mask, NULL);
+}
 
 static void
 ProcessWalRcvInterrupts(void)
@@ -195,6 +216,7 @@ WalReceiverMain(void)
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz last_recv_timestamp;
 	bool		ping_sent;
+	bool		forceReply;
 
 	/*
 	 * WalRcv should be set up already (if we are a backend, we inherit this
@@ -246,6 +268,7 @@ WalReceiverMain(void)
 
 	/* Initialise to a sanish value */
 	walrcv->lastMsgSendTime = walrcv->lastMsgReceiptTime = walrcv->latestWalEndTime = GetCurrentTimestamp();
+	walrcv->causalReadsLease = 0;
 
 	SpinLockRelease(&walrcv->mutex);
 
@@ -263,7 +286,7 @@ WalReceiverMain(void)
 	pqsignal(SIGALRM, SIG_IGN);
 	pqsignal(SIGPIPE, SIG_IGN);
 	pqsignal(SIGUSR1, WalRcvSigUsr1Handler);
-	pqsignal(SIGUSR2, SIG_IGN);
+	pqsignal(SIGUSR2, WalRcvSigUsr2Handler);
 
 	/* Reset some signals that are accepted by postmaster but not here */
 	pqsignal(SIGCHLD, SIG_DFL);
@@ -294,6 +317,10 @@ WalReceiverMain(void)
 	/* Unblock signals (they were blocked when the postmaster forked us) */
 	PG_SETMASK(&UnBlockSig);
 
+	/* Block SIGUSR2 (we unblock it only during network waits). */
+	WalRcvBlockSigUsr2();
+	got_SIGUSR2 = false;
+
 	/* Establish the connection to the primary for XLOG streaming */
 	EnableWalRcvImmediateExit();
 	walrcv_connect(conninfo);
@@ -403,7 +430,9 @@ WalReceiverMain(void)
 				}
 
 				/* Wait a while for data to arrive */
+				WalRcvUnblockSigUsr2();
 				len = walrcv_receive(NAPTIME_PER_CYCLE, &buf);
+				WalRcvBlockSigUsr2();
 				if (len != 0)
 				{
 					/*
@@ -434,11 +463,21 @@ WalReceiverMain(void)
 							endofwal = true;
 							break;
 						}
+						WalRcvUnblockSigUsr2();
 						len = walrcv_receive(0, &buf);
+						WalRcvBlockSigUsr2();
+					}
+
+					if (got_SIGUSR2)
+					{
+						/* The recovery process asked us to force a reply. */
+						got_SIGUSR2 = false;
+						forceReply = true;
 					}
 
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(false, false);
+					XLogWalRcvSendReply(forceReply, false);
+					forceReply = false;
 
 					/*
 					 * If we've written some records, flush them to disk and
@@ -493,7 +532,15 @@ WalReceiverMain(void)
 						}
 					}
 
-					XLogWalRcvSendReply(requestReply, requestReply);
+					/* Check if the startup process has signaled us. */
+					if (got_SIGUSR2)
+					{
+						got_SIGUSR2 = false;
+						forceReply = true;
+					}
+
+					XLogWalRcvSendReply(requestReply || forceReply, requestReply);
+					forceReply = false;
 					XLogWalRcvSendHSFeedback(false);
 				}
 			}
@@ -735,6 +782,13 @@ WalRcvSigUsr1Handler(SIGNAL_ARGS)
 	errno = save_errno;
 }
 
+/* SIGUSR2: used to receive wakeups from recovery */
+static void
+WalRcvSigUsr2Handler(SIGNAL_ARGS)
+{
+	got_SIGUSR2 = true;
+}
+
 /* SIGTERM: set flag for main loop, or shutdown immediately if safe */
 static void
 WalRcvShutdownHandler(SIGNAL_ARGS)
@@ -795,6 +849,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 	XLogRecPtr	walEnd;
 	TimestampTz sendTime;
 	bool		replyRequested;
+	TimestampTz causalReadsLease;
 
 	resetStringInfo(&incoming_message);
 
@@ -815,7 +870,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				walEnd = pq_getmsgint64(&incoming_message);
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, NULL);
 
 				buf += hdrlen;
 				len -= hdrlen;
@@ -825,7 +880,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 		case 'k':				/* Keepalive */
 			{
 				/* copy message to StringInfo */
-				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
+				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char) + sizeof(int64);
 				if (len != hdrlen)
 					ereport(ERROR,
 							(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -837,8 +892,12 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
 				replyRequested = pq_getmsgbyte(&incoming_message);
+				causalReadsLease = IntegerTimestampToTimestampTz(
+					pq_getmsgint64(&incoming_message));
+				ProcessWalSndrMessage(walEnd, sendTime, &causalReadsLease);
 
-				ProcessWalSndrMessage(walEnd, sendTime);
+				/* Remember primary's timestamp at this WAL location. */
+				SetXLogReplayTimestampAtLsn(sendTime, walEnd);
 
 				/* If the primary requested a reply, send one immediately */
 				if (replyRequested)
@@ -1032,6 +1091,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	XLogRecPtr	applyPtr;
 	static TimestampTz sendTime = 0;
 	TimestampTz now;
+	TimestampTz applyTimestamp = 0;
 
 	/*
 	 * If the user doesn't want status to be reported to the master, be sure
@@ -1063,7 +1123,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	/* Construct a new message */
 	writePtr = LogstreamResult.Write;
 	flushPtr = LogstreamResult.Flush;
-	applyPtr = GetXLogReplayRecPtr(NULL);
+	applyTimestamp = GetXLogReplayTimestamp(&applyPtr);
 
 	resetStringInfo(&reply_message);
 	pq_sendbyte(&reply_message, 'r');
@@ -1071,6 +1131,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	pq_sendint64(&reply_message, flushPtr);
 	pq_sendint64(&reply_message, applyPtr);
 	pq_sendint64(&reply_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&reply_message, TimestampTzToIntegerTimestamp(applyTimestamp));
 	pq_sendbyte(&reply_message, requestReply ? 1 : 0);
 
 	/* Send it */
@@ -1169,15 +1230,52 @@ XLogWalRcvSendHSFeedback(bool immed)
  * Update shared memory status upon receiving a message from primary.
  *
  * 'walEnd' and 'sendTime' are the end-of-WAL and timestamp of the latest
- * message, reported by primary.
+ * message, reported by primary.  'causalReadsLease' is a pointer to
+ * the time the primary promises that this standby can safely claim to be
+ * causally consistent, to 0 if it cannot, or a NULL pointer for no change.
  */
 static void
-ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
+ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+					  TimestampTz *causalReadsLease)
 {
 	WalRcvData *walrcv = WalRcv;
 
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
 
+	/* Sanity check for the causalReadsLease time. */
+	if (causalReadsLease != NULL && *causalReadsLease != 0)
+	{
+		/* Deduce max_clock_skew from the causalReadsLease and sendTime. */
+#ifdef HAVE_INT64_TIMESTAMP
+		int64 diffMillis = (*causalReadsLease - sendTime) / 1000;
+#else
+		int64 diffMillis = (*causalReadsLease - sendTime) * 1000;
+#endif
+		int64 max_clock_skew = diffMillis / (CAUSAL_READS_CLOCK_SKEW_RATIO - 1);
+
+		if (sendTime > TimestampTzPlusMilliseconds(lastMsgReceiptTime, max_clock_skew))
+		{
+			/*
+			 * The primary's clock is more than max_clock_skew + network
+			 * latency ahead of the standby's clock.  (If the primary's clock
+			 * is more than max_clock_skew ahead of the standby's clock, but
+			 * by less than the network latency, then there isn't much we can
+			 * do to detect that; but it still seems useful to have this basic
+			 * sanity check for wildly misconfigured servers.)
+			 */
+			elog(LOG, "the primary server's clock time is too far ahead");
+			causalReadsLease = NULL;
+		}
+		/*
+		 * We could also try to detect cases where sendTime is more than
+		 * max_clock_skew in the past according to the standby's clock, but
+		 * that is indistinguishable from network latency/buffering, so we
+		 * could produce misleading error messages; if we do nothing, the
+		 * consequence is 'standby is not available for causal reads' errors
+		 * which should cause the user to investigate.
+		 */
+	}
+
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
 	if (walrcv->latestWalEnd < walEnd)
@@ -1185,6 +1283,8 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->latestWalEnd = walEnd;
 	walrcv->lastMsgSendTime = sendTime;
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
+	if (causalReadsLease != NULL)
+		walrcv->causalReadsLease = *causalReadsLease;
 	SpinLockRelease(&walrcv->mutex);
 
 	if (log_min_messages <= DEBUG2)
@@ -1215,3 +1315,23 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 		pfree(receipttime);
 	}
 }
+
+/*
+ * Wake up the walreceiver if it happens to be blocked in walrcv_receive,
+ * and tell it that a commit record has been applied.
+ *
+ * This is called by the startup process whenever interesting xlog records
+ * are applied, so that walreceiver can check if it needs to send an apply
+ * notification back to the master which may be waiting in a COMMIT with
+ * synchronous_commit = apply or causal_reads = on.
+ *
+ * TODO: This may change -- Simon Riggs suggested latches for this.  Maybe
+ * pipes would work too (and avoid interrupting systems calls and allow for
+ * multiplexed IO with the replication socket).
+ */
+void
+WalRcvWakeup(void)
+{
+	if (WalRcv->pid != 0)
+		kill(WalRcv->pid, SIGUSR2);
+}
diff --git a/src/backend/replication/walreceiverfuncs.c b/src/backend/replication/walreceiverfuncs.c
index 4452f25..a76d4da 100644
--- a/src/backend/replication/walreceiverfuncs.c
+++ b/src/backend/replication/walreceiverfuncs.c
@@ -25,9 +25,11 @@
 
 #include "access/xlog_internal.h"
 #include "postmaster/startup.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "storage/pmsignal.h"
 #include "storage/shmem.h"
+#include "utils/guc.h"
 #include "utils/timestamp.h"
 
 WalRcvData *WalRcv = NULL;
@@ -374,3 +376,21 @@ GetReplicationTransferLatency(void)
 
 	return ms;
 }
+
+/*
+ * Used by snapmgr to check if this standby has a valid lease, granting it the
+ * right to consider itself available for causal reads.
+ */
+bool
+WalRcvCausalReadsAvailable(void)
+{
+	WalRcvData *walrcv = WalRcv;
+	TimestampTz now = GetCurrentTimestamp();
+	bool result;
+
+	SpinLockAcquire(&walrcv->mutex);
+	result = walrcv->causalReadsLease != 0 && now <= walrcv->causalReadsLease;
+	SpinLockRelease(&walrcv->mutex);
+
+	return result;
+}
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 7b1b7f1..afd921b 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -153,9 +153,20 @@ static StringInfoData tmpbuf;
  */
 static TimestampTz last_reply_timestamp = 0;
 
+static TimestampTz last_keepalive_timestamp = 0;
+
 /* Have we sent a heartbeat message asking for reply, since last reply? */
 static bool waiting_for_ping_response = false;
 
+/* How long do need to stay in JOINING state? */
+static TimestampTz causal_reads_joining_until = 0;
+
+/* The last causal reads lease sent to the standby. */
+static TimestampTz causal_reads_last_lease = 0;
+
+/* Is this WALSender listed in causal_reads_standby_names? */
+static bool am_potential_causal_reads_standby = false;
+
 /*
  * While streaming WAL in Copy mode, streamingDoneSending is set to true
  * after we have sent CopyDone. We should not send any more CopyData messages
@@ -242,6 +253,57 @@ InitWalSender(void)
 }
 
 /*
+ * If we are exiting unexpectedly, we may need to communicate with concurrent
+ * causal_reads commits to maintain the causal consistency guarantee.
+ */
+static void
+PrepareUncleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * We've lost contact with the standby, but it may still be alive.  We
+		 * can't let any causal_reads transactions return until we've stalled
+		 * for long enough for a zombie standby to start raising errors
+		 * because its lease has expired.
+		 */
+		elog(LOG, "standby \"%s\" is lost (no longer available for causal reads)", application_name);
+		CausalReadsBeginStall(causal_reads_last_lease);
+
+		/*
+		 * We set the state to a lower level _after_ beginning the stall,
+		 * otherwise there would be a tiny window where commits could return
+		 * without observing the stall.
+		 */
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
+ * We are shutting down because we received a goodbye message from the
+ * walreceiver.
+ */
+static void
+PrepareCleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * The standby is shutting down, so it won't be running any more
+		 * transactions.  It is therefore safe to stop waiting for it, and no
+		 * stall is necessary.
+		 */
+		elog(LOG, "standby \"%s\" is leaving (no longer available for causal reads)", application_name);
+
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
  * Clean up after an error.
  *
  * WAL sender processes don't use transactions like regular backends do.
@@ -264,7 +326,10 @@ WalSndErrorCleanup(void)
 
 	replication_active = false;
 	if (walsender_ready_to_stop)
+	{
+		PrepareUncleanExit();
 		proc_exit(0);
+	}
 
 	/* Revert back to startup state */
 	WalSndSetState(WALSNDSTATE_STARTUP);
@@ -276,6 +341,8 @@ WalSndErrorCleanup(void)
 static void
 WalSndShutdown(void)
 {
+	PrepareUncleanExit();
+
 	/*
 	 * Reset whereToSendOutput to prevent ereport from attempting to send any
 	 * more messages to the standby.
@@ -1386,6 +1453,7 @@ ProcessRepliesIfAny(void)
 		if (r < 0)
 		{
 			/* unexpected error or EOF */
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1402,6 +1470,7 @@ ProcessRepliesIfAny(void)
 		resetStringInfo(&reply_message);
 		if (pq_getmessage(&reply_message, 0))
 		{
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1451,6 +1520,7 @@ ProcessRepliesIfAny(void)
 				 * 'X' means that the standby is closing down the socket.
 				 */
 			case 'X':
+				PrepareCleanExit();
 				proc_exit(0);
 
 			default:
@@ -1543,15 +1613,29 @@ ProcessStandbyReplyMessage(void)
 	XLogRecPtr	writePtr,
 				flushPtr,
 				applyPtr;
+	int			applyLagMs;
 	bool		replyRequested;
+	TimestampTz now = GetCurrentTimestamp();
+	TimestampTz applyTimestamp;
 
 	/* the caller already consumed the msgtype byte */
 	writePtr = pq_getmsgint64(&reply_message);
 	flushPtr = pq_getmsgint64(&reply_message);
 	applyPtr = pq_getmsgint64(&reply_message);
 	(void) pq_getmsgint64(&reply_message);		/* sendTime; not used ATM */
+	applyTimestamp = IntegerTimestampToTimestampTz(pq_getmsgint64(&reply_message));
 	replyRequested = pq_getmsgbyte(&reply_message);
 
+	/* Compute the apply lag in milliseconds. */
+	if (applyTimestamp == 0)
+		applyLagMs = -1;
+	else
+#ifdef HAVE_INT64_TIMESTAMP
+		applyLagMs = (now - applyTimestamp) / 1000;
+#else
+		applyLagMs = (now - applyTimestamp) * 1000.0;
+#endif
+
 	elog(DEBUG2, "write %X/%X flush %X/%X apply %X/%X%s",
 		 (uint32) (writePtr >> 32), (uint32) writePtr,
 		 (uint32) (flushPtr >> 32), (uint32) flushPtr,
@@ -1568,16 +1652,91 @@ ProcessStandbyReplyMessage(void)
 	 */
 	{
 		WalSnd *walsnd = MyWalSnd;
+		WalSndCausalReadsState causal_reads_state = walsnd->causal_reads_state;
+		bool causal_reads_state_changed = false;
+
+		/*
+		 * Handle causal reads state transitions, if a causal_reads_timeout is
+		 * configured, this standby is listed in causal_reads_standby_names,
+		 * and we are a primary database (not a cascading standby).
+		 */
+		if (causal_reads_timeout != 0 &&
+			am_potential_causal_reads_standby &&
+			!am_cascading_walsender)
+		{
+			if (applyLagMs >= 0 && applyLagMs < causal_reads_timeout)
+			{
+				if (causal_reads_state == WALSNDCRSTATE_UNAVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_JOINING;
+					causal_reads_joining_until =
+						TimestampTzPlusMilliseconds(now, causal_reads_timeout);
+					causal_reads_state_changed = true;
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING &&
+						 now >= causal_reads_joining_until)
+				{
+					causal_reads_state = WALSNDCRSTATE_AVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+			else
+			{
+				if (causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+					/*
+					 * We are dropping a causal reads available standby, so we
+					 * mustn't let any commit command that is waiting in
+					 * CausalReadsWaitForLSN return until we are sure that the
+					 * standby definitely knows that it's not available and
+					 * starts raising errors for causal_reads transactions.
+					 * TODO: We could just wait until the standby acks that
+					 * its lease has been cancelled, and start numbering
+					 * keepalives and sending the number back in replies, so
+					 * we know it's acking the right message; then lagging
+					 * standbys would be less disruptive, but for now we just
+					 * wait for the lease to expire, as we do when we lose
+					 * contact with a standby, for the sake of simplicity.
+					 */
+					CausalReadsBeginStall(causal_reads_last_lease);
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING)
+				{
+					/*
+					 * Dropping a joining standby doesn't require a stall,
+					 * because the standby doesn't think it's available, so
+					 * it's already raising the error for causal_reads
+					 * transactions.
+					 */
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+		}
 
 		SpinLockAcquire(&walsnd->mutex);
 		walsnd->write = writePtr;
 		walsnd->flush = flushPtr;
 		walsnd->apply = applyPtr;
+		walsnd->applyLagMs = applyLagMs;
+		walsnd->causal_reads_state = causal_reads_state;
 		SpinLockRelease(&walsnd->mutex);
+
+		if (causal_reads_state_changed)
+		{
+			WalSndKeepalive(true);
+			elog(LOG, "standby \"%s\" is %s", application_name,
+				 causal_reads_state == WALSNDCRSTATE_UNAVAILABLE ? "unavailable for causal reads" :
+				 causal_reads_state == WALSNDCRSTATE_JOINING ? "joining as a causal reads standby..." :
+				 causal_reads_state == WALSNDCRSTATE_AVAILABLE ? "available for causal reads" :
+				 "UNKNOWN");
+		}
 	}
 
 	if (!am_cascading_walsender)
-		SyncRepReleaseWaiters();
+		SyncRepReleaseWaiters(MyWalSnd->causal_reads_state >= WALSNDCRSTATE_JOINING);
 
 	/*
 	 * Advance our local xmin horizon when the client confirmed a flush.
@@ -1724,27 +1883,34 @@ WalSndComputeSleeptime(TimestampTz now)
 {
 	long		sleeptime = 10000;		/* 10 s */
 
-	if (wal_sender_timeout > 0 && last_reply_timestamp > 0)
+	if ((wal_sender_timeout > 0 || causal_reads_timeout > 0) && last_reply_timestamp > 0)
 	{
 		TimestampTz wakeup_time;
 		long		sec_to_timeout;
 		int			microsec_to_timeout;
 
-		/*
-		 * At the latest stop sleeping once wal_sender_timeout has been
-		 * reached.
-		 */
-		wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-												  wal_sender_timeout);
-
-		/*
-		 * If no ping has been sent yet, wakeup when it's time to do so.
-		 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
-		 * the timeout passed without a response.
-		 */
-		if (!waiting_for_ping_response)
+		if (causal_reads_timeout != 0)
+			wakeup_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+													  causal_reads_timeout /
+													  CAUSAL_READS_KEEPALIVE_RATIO);
+		else
+		{
+			/*
+			 * At the latest stop sleeping once wal_sender_timeout has been
+			 * reached.
+			 */
 			wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-													  wal_sender_timeout / 2);
+													  wal_sender_timeout);
+
+			/*
+			 * If no ping has been sent yet, wakeup when it's time to do so.
+			 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
+			 * the timeout passed without a response.
+			 */
+			if (!waiting_for_ping_response)
+				wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+														  wal_sender_timeout / 2);
+		}
 
 		/* Compute relative time until wakeup. */
 		TimestampDifference(now, wakeup_time,
@@ -1765,15 +1931,28 @@ static void
 WalSndCheckTimeOut(TimestampTz now)
 {
 	TimestampTz timeout;
+	int allowed_time;
 
 	/* don't bail out if we're doing something that doesn't require timeouts */
 	if (last_reply_timestamp <= 0)
 		return;
 
+	/*
+	 * If a causal_reads_timeout is configured, it is used instead of
+	 * wal_sender_timeout.  Ideally we'd use causal_reads_timeout / 2 +
+	 * allowance for network latency, but since walreceiver can become quite
+	 * bogged down fsyncing WAL we allow more tolerance.  (This could be
+	 * tightened up once standbys hand writing off to the WAL writer).
+	 */
+	if (causal_reads_timeout != 0)
+		allowed_time = causal_reads_timeout;
+	else
+		allowed_time = wal_sender_timeout;
+
 	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
-										  wal_sender_timeout);
+										  allowed_time);
 
-	if (wal_sender_timeout > 0 && now >= timeout)
+	if (allowed_time > 0 && now >= timeout)
 	{
 		/*
 		 * Since typically expiration of replication timeout means
@@ -1806,6 +1985,9 @@ WalSndLoop(WalSndSendDataCallback send_data)
 	last_reply_timestamp = GetCurrentTimestamp();
 	waiting_for_ping_response = false;
 
+	/* Check if we are managing potential causal_reads standby. */
+	am_potential_causal_reads_standby = CausalReadsPotentialStandby();
+
 	/*
 	 * Loop until we reach the end of this timeline or the client requests to
 	 * stop streaming.
@@ -1966,6 +2148,7 @@ InitWalSenderSlot(void)
 			walsnd->flush = InvalidXLogRecPtr;
 			walsnd->apply = InvalidXLogRecPtr;
 			walsnd->state = WALSNDSTATE_STARTUP;
+			walsnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
 			walsnd->latch = &MyProc->procLatch;
 			SpinLockRelease(&walsnd->mutex);
 			/* don't need the lock anymore */
@@ -2735,6 +2918,24 @@ WalSndGetStateString(WalSndState state)
 	return "UNKNOWN";
 }
 
+/*
+ * Return a string constant representing the causal reads state. This is used
+ * in system views, and should *not* be translated.
+ */
+static const char *
+WalSndGetCausalReadsStateString(WalSndCausalReadsState causal_reads_state)
+{
+	switch (causal_reads_state)
+	{
+		case WALSNDCRSTATE_UNAVAILABLE:
+			return "unavailable";
+		case WALSNDCRSTATE_JOINING:
+			return "joining";
+		case WALSNDCRSTATE_AVAILABLE:
+			return "available";
+	}
+	return "UNKNOWN";
+}
 
 /*
  * Returns activity of walsenders, including pids and xlog locations sent to
@@ -2743,7 +2944,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	8
+#define PG_STAT_GET_WAL_SENDERS_COLS	10
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2791,8 +2992,10 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		XLogRecPtr	write;
 		XLogRecPtr	flush;
 		XLogRecPtr	apply;
+		int			applyLagMs;
 		int			priority;
 		WalSndState state;
+		WalSndCausalReadsState causalReadsState;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
 		bool		nulls[PG_STAT_GET_WAL_SENDERS_COLS];
 
@@ -2802,9 +3005,11 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		SpinLockAcquire(&walsnd->mutex);
 		sentPtr = walsnd->sentPtr;
 		state = walsnd->state;
+		causalReadsState = walsnd->causal_reads_state;
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
+		applyLagMs = walsnd->applyLagMs;
 		priority = walsnd->sync_standby_priority;
 		SpinLockRelease(&walsnd->mutex);
 
@@ -2839,6 +3044,23 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				nulls[5] = true;
 			values[5] = LSNGetDatum(apply);
 
+			if (applyLagMs < 0)
+				nulls[6] = true;
+			else
+			{
+				Interval *applyLagInterval = palloc(sizeof(Interval));
+
+				applyLagInterval->month = 0;
+				applyLagInterval->day = 0;
+#ifdef HAVE_INT64_TIMESTAMP
+				applyLagInterval->time = applyLagMs * 1000;
+#else
+				applyLagInterval->time = applyLagMs / 1000.0;
+#endif
+				nulls[6] = false;
+				values[6] = IntervalPGetDatum(applyLagInterval);
+			}
+
 			/*
 			 * Treat a standby such as a pg_basebackup background process
 			 * which always returns an invalid flush location, as an
@@ -2846,18 +3068,21 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 			 */
 			priority = XLogRecPtrIsInvalid(walsnd->flush) ? 0 : priority;
 
-			values[6] = Int32GetDatum(priority);
+			values[7] = Int32GetDatum(priority);
 
 			/*
 			 * More easily understood version of standby state. This is purely
 			 * informational, not different from priority.
 			 */
 			if (priority == 0)
-				values[7] = CStringGetTextDatum("async");
+				values[8] = CStringGetTextDatum("async");
 			else if (walsnd == sync_standby)
-				values[7] = CStringGetTextDatum("sync");
+				values[8] = CStringGetTextDatum("sync");
 			else
-				values[7] = CStringGetTextDatum("potential");
+				values[8] = CStringGetTextDatum("potential");
+
+			values[9] =
+				CStringGetTextDatum(WalSndGetCausalReadsStateString(causalReadsState));
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
@@ -2877,14 +3102,52 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 static void
 WalSndKeepalive(bool requestReply)
 {
+	TimestampTz now;
+	TimestampTz causal_reads_lease;
+
 	elog(DEBUG2, "sending replication keepalive");
 
+	/*
+	 * If the walsender currently deems the standby to be available for causal
+	 * reads, then it grants a causal reads lease.  The lease authorizes the
+	 * standby to consider itself available for causal reads until a short
+	 * time in the future.  The primary promises to uphold the causal reads
+	 * guarantee until that time, by stalling commits until the the lease has
+	 * expired if necessary.
+	 */
+	now = GetCurrentTimestamp();
+	if (MyWalSnd->causal_reads_state < WALSNDCRSTATE_AVAILABLE)
+		causal_reads_lease = 0; /* Not available, no lease granted. */
+	else
+	{
+		/*
+		 * Since this timestamp is being sent to the standby where it will be
+		 * compared against a time generated by the standby's system clock, we
+		 * must consider clock skew.  First, we decide on a maximum tolerable
+		 * difference between system clocks.  If the primary's clock is ahead
+		 * of the standby's by more than this, then all bets are off (the
+		 * standby could falsely believe it has a valid lease).  If the
+		 * primary's clock is behind the standby's by more than this, then the
+		 * standby will err the other way and generate spurious errors in
+		 * causal_reads mode.  Rather than having a separate GUC for this, we
+		 * derive it from causal_reads_timeout.
+		 */
+		int max_clock_skew = causal_reads_timeout / CAUSAL_READS_CLOCK_SKEW_RATIO;
+
+		/* Compute and remember the expiry time of the lease we're granting. */
+		causal_reads_last_lease = TimestampTzPlusMilliseconds(now, causal_reads_timeout);
+		/* The version we'll send to the standby is adjusted to tolerate clock skew. */
+		causal_reads_lease =
+			TimestampTzPlusMilliseconds(causal_reads_last_lease, -max_clock_skew);
+	}
+
 	/* construct the message... */
 	resetStringInfo(&output_message);
 	pq_sendbyte(&output_message, 'k');
 	pq_sendint64(&output_message, sentPtr);
-	pq_sendint64(&output_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(now));
 	pq_sendbyte(&output_message, requestReply ? 1 : 0);
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(causal_reads_lease));
 
 	/* ... and send it wrapped in CopyData */
 	pq_putmessage_noblock('d', output_message.data, output_message.len);
@@ -2902,23 +3165,32 @@ WalSndKeepaliveIfNecessary(TimestampTz now)
 	 * Don't send keepalive messages if timeouts are globally disabled or
 	 * we're doing something not partaking in timeouts.
 	 */
-	if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
+	if ((wal_sender_timeout <= 0 && causal_reads_timeout == 0) || last_reply_timestamp <= 0)
 		return;
 
-	if (waiting_for_ping_response)
+	if (waiting_for_ping_response && causal_reads_timeout == 0)
 		return;
 
 	/*
 	 * If half of wal_sender_timeout has lapsed without receiving any reply
 	 * from the standby, send a keep-alive message to the standby requesting
 	 * an immediate reply.
+	 *
+	 * If causal_reads_timeout has been configured, use it to control
+	 * keepalive intervals rather than wal_sender_timeout.
 	 */
-	ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-											wal_sender_timeout / 2);
+	if (causal_reads_timeout != 0)
+		ping_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+												causal_reads_timeout /
+												CAUSAL_READS_KEEPALIVE_RATIO);
+	else
+		ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+												wal_sender_timeout / 2);
 	if (now >= ping_time)
 	{
 		WalSndKeepalive(true);
 		waiting_for_ping_response = true;
+		last_keepalive_timestamp = now;
 
 		/* Try to flush pending output to the client */
 		if (pq_flush_if_writable() != 0)
diff --git a/src/backend/utils/adt/timestamp.c b/src/backend/utils/adt/timestamp.c
index 8fbb310..12c8b88 100644
--- a/src/backend/utils/adt/timestamp.c
+++ b/src/backend/utils/adt/timestamp.c
@@ -1611,6 +1611,20 @@ IntegerTimestampToTimestampTz(int64 timestamp)
 #endif
 
 /*
+ * TimestampTzToIntegerTimestamp -- convert a native timestamp to int64 format
+ *
+ * When compiled with --enable-integer-datetimes, this is implemented as a
+ * no-op macro.
+ */
+#ifndef HAVE_INT64_TIMESTAMP
+int64
+TimestampTzToIntegerTimestamp(TimestampTz timestamp)
+{
+	return timestamp * 1000000;
+}
+#endif
+
+/*
  * TimestampDifference -- convert the difference between two timestamps
  *		into integer seconds and microseconds
  *
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index a185749..6e0b144 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -351,6 +351,7 @@ static const struct config_enum_entry constraint_exclusion_options[] = {
 static const struct config_enum_entry synchronous_commit_options[] = {
 	{"local", SYNCHRONOUS_COMMIT_LOCAL_FLUSH, false},
 	{"remote_write", SYNCHRONOUS_COMMIT_REMOTE_WRITE, false},
+	{"apply", SYNCHRONOUS_COMMIT_REMOTE_APPLY, false},
 	{"on", SYNCHRONOUS_COMMIT_ON, false},
 	{"off", SYNCHRONOUS_COMMIT_OFF, false},
 	{"true", SYNCHRONOUS_COMMIT_ON, true},
@@ -1618,6 +1619,16 @@ static struct config_bool ConfigureNamesBool[] =
 		NULL, NULL, NULL
 	},
 
+	{
+		{"causal_reads", PGC_USERSET, REPLICATION_STANDBY,
+		 gettext_noop("Enables causal reads."),
+		 NULL
+		},
+		&causal_reads,
+		false,
+		NULL, NULL, NULL
+	},
+
 	/* End-of-list marker */
 	{
 		{NULL, 0, 0, NULL, NULL}, NULL, false, NULL, NULL, NULL
@@ -1776,6 +1787,17 @@ static struct config_int ConfigureNamesInt[] =
 	},
 
 	{
+		{"causal_reads_timeout", PGC_SIGHUP, REPLICATION_STANDBY,
+			gettext_noop("Sets the maximum apply lag before causal reads standbys are no longer available."),
+			NULL,
+			GUC_UNIT_MS
+		},
+		&causal_reads_timeout,
+		0, 0, INT_MAX,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"max_connections", PGC_POSTMASTER, CONN_AUTH_SETTINGS,
 			gettext_noop("Sets the maximum number of concurrent connections."),
 			NULL
@@ -3361,7 +3383,18 @@ static struct config_string ConfigureNamesString[] =
 		},
 		&SyncRepStandbyNames,
 		"",
-		check_synchronous_standby_names, NULL, NULL
+		check_standby_names, NULL, NULL
+	},
+
+	{
+		{"causal_reads_standby_names", PGC_SIGHUP, REPLICATION_MASTER,
+			gettext_noop("List of names of potential causal reads standbys."),
+			NULL,
+			GUC_LIST_INPUT
+		},
+		&causal_reads_standby_names,
+		"*",
+		check_standby_names, NULL, NULL
 	},
 
 	{
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index 074935c..a466732 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -46,8 +46,11 @@
 
 #include "access/transam.h"
 #include "access/xact.h"
+#include "access/xlog.h"
 #include "lib/pairingheap.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
+#include "replication/walreceiver.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
 #include "storage/procarray.h"
@@ -209,6 +212,16 @@ GetTransactionSnapshot(void)
 				 "cannot take query snapshot during a parallel operation");
 
 		/*
+		 * In causal_reads mode on a standby, check if we have definitely
+		 * applied WAL for any COMMIT that returned successfully on the
+		 * primary.
+		 *
+		 * TODO: Machine readable error code?
+		 */
+		if (causal_reads && RecoveryInProgress() && !WalRcvCausalReadsAvailable())
+			elog(ERROR, "standby is not available for causal reads");
+
+		/*
 		 * In transaction-snapshot mode, the first snapshot must live until
 		 * end of xact regardless of what the caller does with it, so we must
 		 * make a copy of it rather than returning CurrentSnapshotData
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index cb1c2db..0f08ff5 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -60,7 +60,11 @@ typedef enum
 	SYNCHRONOUS_COMMIT_LOCAL_FLUSH,		/* wait for local flush only */
 	SYNCHRONOUS_COMMIT_REMOTE_WRITE,	/* wait for local flush and remote
 										 * write */
-	SYNCHRONOUS_COMMIT_REMOTE_FLUSH		/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_FLUSH,	/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_APPLY,	/* wait for local flush and remote
+										 * apply */
+	SYNCHRONOUS_COMMIT_CONSISTENT_APPLY /* wait for local flusha and remote
+										   apply with causal consistency */
 }	SyncCommitLevel;
 
 /* Define the default setting for synchonous_commit */
@@ -144,10 +148,13 @@ typedef void (*SubXactCallback) (SubXactEvent event, SubTransactionId mySubid,
  * EOXact... routines which run at the end of the original transaction
  * completion.
  */
+#define XACT_COMPLETION_SYNC_APPLY_FEEDBACK		(1U << 29)
 #define XACT_COMPLETION_UPDATE_RELCACHE_FILE	(1U << 30)
 #define XACT_COMPLETION_FORCE_SYNC_COMMIT		(1U << 31)
 
 /* Access macros for above flags */
+#define XactCompletionSyncApplyFeedback(xinfo) \
+	(!!(xinfo & XACT_COMPLETION_SYNC_APPLY_FEEDBACK))
 #define XactCompletionRelcacheInitFileInval(xinfo) \
 	(!!(xinfo & XACT_COMPLETION_UPDATE_RELCACHE_FILE))
 #define XactCompletionForceSyncCommit(xinfo) \
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 790ca66..8aeda11 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -235,6 +235,9 @@ extern void GetXLogReceiptTime(TimestampTz *rtime, bool *fromStream);
 extern XLogRecPtr GetXLogReplayRecPtr(TimeLineID *replayTLI);
 extern XLogRecPtr GetXLogInsertRecPtr(void);
 extern XLogRecPtr GetXLogWriteRecPtr(void);
+extern void SetXLogReplayTimestamp(TimestampTz timestamp);
+extern void SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn);
+extern TimestampTz GetXLogReplayTimestamp(XLogRecPtr *lsn);
 extern bool RecoveryIsPaused(void);
 extern void SetRecoveryPause(bool recoveryPause);
 extern TimestampTz GetLatestXTime(void);
@@ -267,6 +270,8 @@ extern bool CheckPromoteSignal(void);
 extern void WakeupRecovery(void);
 extern void SetWalWriterSleeping(bool sleeping);
 
+extern void XLogRequestWalReceiverReply(void);
+
 extern void assign_max_wal_size(int newval, void *extra);
 extern void assign_checkpoint_completion_target(double newval, void *extra);
 
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index d8640db..acb6796 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2783,7 +2783,7 @@ DATA(insert OID = 1936 (  pg_stat_get_backend_idset		PGNSP PGUID 12 1 100 0 0 f
 DESCR("statistics: currently active backend IDs");
 DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f f f t s r 1 0 2249 "23" "{23,26,23,26,25,25,25,16,1184,1184,1184,1184,869,25,23,28,28,16,25,25,23,16,25}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{pid,datid,pid,usesysid,application_name,state,query,waiting,xact_start,query_start,backend_start,state_change,client_addr,client_hostname,client_port,backend_xid,backend_xmin,ssl,sslversion,sslcipher,sslbits,sslcompression,sslclientdn}" _null_ _null_ pg_stat_get_activity _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active backends");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,23,25}" "{o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25,25}" "{o,o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state,causal_reads_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 2026 (  pg_backend_pid				PGNSP PGUID 12 1 0 0 0 f f f f t f s r 0 0 23 "" _null_ _null_ _null_ _null_ _null_ pg_backend_pid _null_ _null_ _null_ ));
 DESCR("statistics: current backend PID");
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index 71e2857..6a090b7 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -23,14 +23,34 @@
 #define SYNC_REP_NO_WAIT		-1
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
+#define SYNC_REP_WAIT_APPLY		2
+#define SYNC_REP_WAIT_CAUSAL_READS_APPLY 3
 
-#define NUM_SYNC_REP_WAIT_MODE	2
+#define NUM_SYNC_REP_WAIT_MODE	4
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
 #define SYNC_REP_WAITING			1
 #define SYNC_REP_WAIT_COMPLETE		2
 
+/*
+ * ratio of causal_read_timeout to max_clock_skew (4 means than the maximum
+ * tolerated clock difference between primary and standbys using causal_reads
+ * is 1/4 of causal_reads_timeout)
+ */
+#define CAUSAL_READS_CLOCK_SKEW_RATIO 4
+
+/*
+ * ratio of causal_reads_timeout to keepalive time (2 means that the effective
+ * keepalive time is 1/2 of the causal_reads_timeout GUC when it is non-zero)
+ */
+#define CAUSAL_READS_KEEPALIVE_RATIO 2
+
+/* GUC variables */
+extern int causal_reads_timeout;
+extern bool causal_reads;
+extern char *causal_reads_standby_names;
+
 /* user-settable parameters for synchronous replication */
 extern char *SyncRepStandbyNames;
 
@@ -42,16 +62,23 @@ extern void SyncRepCleanupAtProcExit(void);
 
 /* called by wal sender */
 extern void SyncRepInitConfig(void);
-extern void SyncRepReleaseWaiters(void);
+extern void SyncRepReleaseWaiters(bool walsender_cr_available_or_joining);
 
 /* called by checkpointer */
 extern void SyncRepUpdateSyncStandbysDefined(void);
 
+/* called by user backend (xact.c) */
+extern void CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN);
+
+/* called by wal sender */
+extern void CausalReadsBeginStall(TimestampTz lease_expiry_time);
+extern bool CausalReadsPotentialStandby(void);
+
 /* forward declaration to avoid pulling in walsender_private.h */
 struct WalSnd;
 extern struct WalSnd *SyncRepGetSynchronousStandby(void);
 
-extern bool check_synchronous_standby_names(char **newval, void **extra, GucSource source);
+extern bool check_standby_names(char **newval, void **extra, GucSource source);
 extern void assign_synchronous_commit(int newval, void *extra);
 
 #endif   /* _SYNCREP_H */
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 61255a9..40d99e6 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -79,6 +79,13 @@ typedef struct
 	TimeLineID	receivedTLI;
 
 	/*
+	 * causallyReadsLease is the time until which the primary has authorized
+	 * this standby to consider itself available for causal_reads mode, or 0
+	 * for not authorized.
+	 */
+	TimestampTz causalReadsLease;
+
+	/*
 	 * latestChunkStart is the starting byte position of the current "batch"
 	 * of received WAL.  It's actually the same as the previous value of
 	 * receivedUpto before the last flush to disk.  Startup process can use
@@ -160,5 +167,8 @@ extern void RequestXLogStreaming(TimeLineID tli, XLogRecPtr recptr,
 extern XLogRecPtr GetWalRcvWriteRecPtr(XLogRecPtr *latestChunkStart, TimeLineID *receiveTLI);
 extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
+extern void WalRcvWakeup(void);
+
+extern bool WalRcvCausalReadsAvailable(void);
 
 #endif   /* _WALRECEIVER_H */
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 6dae480..88b4fe9 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -27,6 +27,13 @@ typedef enum WalSndState
 	WALSNDSTATE_STREAMING
 } WalSndState;
 
+typedef enum WalSndCausalReadsState
+{
+	WALSNDCRSTATE_UNAVAILABLE = 0,
+	WALSNDCRSTATE_JOINING,
+	WALSNDCRSTATE_AVAILABLE
+} WalSndCausalReadsState;
+
 /*
  * Each walsender has a WalSnd struct in shared memory.
  */
@@ -34,6 +41,7 @@ typedef struct WalSnd
 {
 	pid_t		pid;			/* this walsender's process id, or 0 */
 	WalSndState state;			/* this walsender's state */
+	WalSndCausalReadsState causal_reads_state; /* the walsender's causal reads state */
 	XLogRecPtr	sentPtr;		/* WAL has been sent up to this point */
 	bool		needreload;		/* does currently-open file need to be
 								 * reloaded? */
@@ -46,6 +54,7 @@ typedef struct WalSnd
 	XLogRecPtr	write;
 	XLogRecPtr	flush;
 	XLogRecPtr	apply;
+	int			applyLagMs;
 
 	/* Protects shared variables shown above. */
 	slock_t		mutex;
@@ -88,6 +97,12 @@ typedef struct
 	 */
 	bool		sync_standbys_defined;
 
+	/*
+	 * Until when must commits in causal_reads stall?  This is used to wait
+	 * for causal reads leases to expire.
+	 */
+	TimestampTz	stall_causal_reads_until;
+
 	WalSnd		walsnds[FLEXIBLE_ARRAY_MEMBER];
 } WalSndCtlData;
 
diff --git a/src/include/utils/timestamp.h b/src/include/utils/timestamp.h
index 530fef1..0f4b166 100644
--- a/src/include/utils/timestamp.h
+++ b/src/include/utils/timestamp.h
@@ -227,9 +227,11 @@ extern bool TimestampDifferenceExceeds(TimestampTz start_time,
 #ifndef HAVE_INT64_TIMESTAMP
 extern int64 GetCurrentIntegerTimestamp(void);
 extern TimestampTz IntegerTimestampToTimestampTz(int64 timestamp);
+extern int64 TimestampTzToIntegerTimestamp(TimestampTz timestamp);
 #else
 #define GetCurrentIntegerTimestamp()	GetCurrentTimestamp()
 #define IntegerTimestampToTimestampTz(timestamp) (timestamp)
+#define TimestampTzToIntegerTimestamp(timestamp) (timestamp)
 #endif
 
 extern TimestampTz time_t_to_timestamptz(pg_time_t tm);
#25Joel Jacobson
joel@trustly.com
In reply to: Thomas Munro (#1)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

+1 to both the feature and the concept of how it's implemented.
Haven't looked at the code though.

This feature would be very useful for us at Trustly.
This would mean we got get rid of an entire system component in our
architecture (=memcached) which we only use to write data which must
be immediately readable at the sync slave after the master commits.
The only such data we currently have is the backoffice sessionid which
must be readable on the slave, otherwise the read-only calls which we
route to the slave might fail because it's missing.

On Wed, Nov 11, 2015 at 6:37 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

Hi hackers,

Many sites use hot standby servers to spread read-heavy workloads over more hardware, or at least would like to. This works well today if your application can tolerate some time lag on standbys. The problem is that there is no guarantee of when a particular commit will become visible for clients connected to standbys. The existing synchronous commit feature is no help here because it guarantees only that the WAL has been flushed on another server before commit returns. It says nothing about whether it has been applied or whether it has been applied on the standby that you happen to be talking to.

A while ago I posted a small patch[1] to allow synchronous_commit to wait for remote apply on the current synchronous standby, but (as Simon Riggs rightly pointed out in that thread) that part isn't the main problem. It seems to me that the main problem for a practical 'writer waits' system is how to support a dynamic set of servers, gracefully tolerating failures and timing out laggards, while also providing a strong guarantee during any such transitions. Now I would like to propose something to do that, and share a proof-of-concept patch.

=== PROPOSAL ===

The working name for the proposed feature is "causal reads", because it provides a form of "causal consistency"[2] (and "read-your-writes" consistency) no matter which server the client is connected to. There is a similar feature by the same name in another product (albeit implemented differently -- 'reader waits'; more about that later). I'm not wedded to the name.

The feature allows arbitrary read-only transactions to be run on any hot standby, with a specific guarantee about the visibility of preceding transactions. The guarantee is that if you set a new GUC "causal_reads = on" in any pair of consecutive transactions (tx1, tx2) where tx2 begins after tx1 successfully returns, then tx2 will either see tx1 or fail with a new error "standby is not available for causal reads", no matter which server it runs on. A discovery mechanism is also provided, giving an instantaneous snapshot of the set of standbys that are currently available for causal reads (ie won't raise the error), in the form of a new column in pg_stat_replication.

For example, a web server might run tx1 to insert a new row representing a message in a discussion forum on the primary server, and then send the user to another web page that runs tx2 to load all messages in the forum on an arbitrary hot standby server. If causal_reads = on in both tx1 and tx2 (for example, because it's on globally), then tx2 is guaranteed to see the new post, or get a (hopefully rare) error telling the client to retry on another server.

Very briefly, the approach is:
1. The primary tracks apply lag on each standby (including between commits).
2. The primary deems standbys 'available' for causal reads if they are applying WAL and replying to keepalives fast enough, and periodically sends the standby an authorization to consider itself available for causal reads until a time in the near future.
3. Commit on the primary with "causal_reads = on" waits for all 'available' standbys either to apply the commit record, or to cease to be 'available' and begin raising the error if they are still alive (because their authorizations have expired).
4. Standbys can start causal reads transactions only while they have an authorization with an expiry time in the future; otherwise they raise an error when an initial snapshot is taken.

In a follow-up email I can write about the design trade-offs considered (mainly 'writer waits' vs 'reader waits'), comparison with some other products, method of estimating replay lag, wait and timeout logic and how it maintains the guarantee in various failure scenarios, logic for standbys joining and leaving, implications of system clock skew between servers, or any other questions you may have, depending on feedback/interest (but see comments in the attached patch for some of those subjects). For now I didn't want to clog up the intertubes with too large a wall of text.

=== PROOF-OF-CONCEPT ===

Please see the POC patch attached. It adds two new GUCs. After setting up one or more hot standbys as per usual, simply add "causal_reads_timeout = 4s" to the primary's postgresql.conf and restart. Now, you can set "causal_reads = on" in some/all sessions to get guaranteed causal consistency. Expected behaviour: the causal reads guarantee is maintained at all times, even when you overwhelm, kill, crash, disconnect, restart, pause, add and remove standbys, and the primary drops them from the set it waits for in a timely fashion. You can monitor the system with the replay_lag and causal_reads_status in pg_stat_replication and some state transition LOG messages on the primary. (The patch also supports "synchronous_commit = apply", but it's not clear how useful that is in practice, as already discussed.)

Lastly, a few notes about how this feature related to some other work:

The current version of this patch has causal_reads as a feature separate from synchronous_commit, from a user's point of view. The thinking behind this is that load balancing and data loss avoidance are separate concerns: synchronous_commit deals with the latter, and causal_reads with the former. That said, existing SyncRep machinery is obviously used (specifically SyncRep queues, with a small modification, as a way to wait for apply messages to arrive from standbys). (An earlier prototype had causal reads as a new level for synchronous_commit and associated states as new walsender states above 'streaming'. When contemplating how to combine this proposal with the multiple-synchronous-standby patch, some colleagues and I came around to the view that the concerns are separate. The reason for wanting to configure complicated quorum definitions is to control data loss risks and has nothing to do with load balancing requirements, so we thought the features should probably be separate.)

The multiple-synchronous-servers patch[3] could be applied or not independently of this feature as a result of that separation, as it doesn't use synchronous_standby_names or indeed any kind of statically defined quorum.

The standby WAL writer patch[4] would significantly improve walreceiver performance and smoothness which would work very well with this proposal.

Please let me know what you think!

Thanks,

[1] /messages/by-id/CAEepm=1fqkivL4V-OTPHwSgw4aF9HcoGiMrCW-yBtjipX9gsag@mail.gmail.com

[2] From http://queue.acm.org/detail.cfm?id=1466448

"Causal consistency. If process A has communicated to process B that it has updated a data item, a subsequent access by process B will return the updated value, and a write is guaranteed to supersede the earlier write. Access by process C that has no causal relationship to process A is subject to the normal eventual consistency rules.

Read-your-writes consistency. This is an important model where process A, after it has updated a data item, always accesses the updated value and will never see an older value. This is a special case of the causal consistency model."

[3] /messages/by-id/CAOG9ApHYCPmTypAAwfD3_V7sVOkbnECFivmRc1AxhB40ZBSwNQ@mail.gmail.com

[4] /messages/by-id/CA+U5nMJifauXvVbx=v3UbYbHO3Jw2rdT4haL6CCooEDM5=4ASQ@mail.gmail.com

--
Thomas Munro
http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

--
Joel Jacobson

Mobile: +46703603801
Trustly.com | Newsroom | LinkedIn | Twitter

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#26Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Thomas Munro (#24)
1 attachment(s)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Wed, Dec 30, 2015 at 5:15 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Wed, Nov 18, 2015 at 11:50 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

Here is a new version of the patch with a few small improvements:
...
[causal-reads-v3.patch]

That didn't apply after 6e7b3359 (which fix a typo in a comment that I
moved). Here is a new version that does.

That one conflicts with b1a9bad9e744857291c7d5516080527da8219854, so
here is a new version.

--
Thomas Munro
http://www.enterprisedb.com

Attachments:

causal-reads-v5.patchapplication/octet-stream; name=causal-reads-v5.patchDownload
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index b0d5440..8bfc510 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,10 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
+	{
+		CausalReadsWaitForLSN(XactLastRecEnd);
 		SyncRepWaitForLSN(XactLastRecEnd);
+	}
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5117,6 +5120,13 @@ XactLogCommitRecord(TimestampTz commit_time,
 		xl_xinfo.xinfo |= XACT_COMPLETION_FORCE_SYNC_COMMIT;
 
 	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this commit is applied.
+	 */
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY || causal_reads)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
+	/*
 	 * Relcache invalidations requires information about the current database
 	 * and so does logical decoding.
 	 */
@@ -5452,6 +5462,19 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 	if (XactCompletionForceSyncCommit(parsed->xinfo))
 		XLogFlush(lsn);
 
+	/*
+	 * Record the primary's timestamp for the commit record, so it can be used
+	 * for tracking replay lag.
+	 */
+	SetXLogReplayTimestamp(parsed->xact_time);
+
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit or causal reads), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 /*
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 7d5d493..47da72f 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -80,6 +80,8 @@ extern uint32 bootstrap_data_checksum_version;
 #define PROMOTE_SIGNAL_FILE		"promote"
 #define FALLBACK_PROMOTE_SIGNAL_FILE "fallback_promote"
 
+/* Size of the circular buffer of timestamped LSNs. */
+#define MAX_TIMESTAMPED_LSNS 8192
 
 /* User-settable parameters */
 int			max_wal_size = 64;	/* 1 GB */
@@ -346,6 +348,12 @@ static XLogRecPtr RedoRecPtr;
 static bool doPageWrites;
 
 /*
+ * doRequestWalReceiverReply is used by recovery code to ask the main recovery
+ * loop to trigger a walreceiver reply.
+ */
+static bool doRequestWalReceiverReply;
+
+/*
  * RedoStartLSN points to the checkpoint's REDO location which is specified
  * in a backup label file, backup history file or control file. In standby
  * mode, XLOG streaming usually starts from the position where an invalid
@@ -357,6 +365,13 @@ static bool doPageWrites;
  */
 static XLogRecPtr RedoStartLSN = InvalidXLogRecPtr;
 
+/*
+ * LastReplayedTimestamp can be set by redo handlers when they apply a record
+ * that carries a timestamp, by calling SetXLogReplayedTimestamp.  The xlog
+ * apply loop can then update the value in shared memory.
+ */
+static TimestampTz LastReplayedTimestamp = 0;
+
 /*----------
  * Shared-memory data structures for XLOG control
  *
@@ -631,6 +646,21 @@ typedef struct XLogCtlData
 	/* current effective recovery target timeline */
 	TimeLineID	RecoveryTargetTLI;
 
+	/* timestamp from the most recently applied record that carried a timestamp. */
+	TimestampTz lastReplayedTimestamp;
+
+	/*
+	 * We maintain a circular buffer of LSNs and associated timestamps.
+	 * Walreceiver writes into it using information from timestamps, and the
+	 * startup recovery process reads from it and notifies walreceiver when
+	 * LSNs are replayed so that the timestamps can be fed back to the
+	 * upstream server, to track lag.
+	 */
+	Index		timestampedLsnRead;
+	Index		timestampedLsnWrite;
+	XLogRecPtr	timestampedLsn[MAX_TIMESTAMPED_LSNS];
+	TimestampTz	timestampedLsnTime[MAX_TIMESTAMPED_LSNS];
+
 	/*
 	 * timestamp of when we started replaying the current chunk of WAL data,
 	 * only relevant for replication or archive recovery
@@ -6850,14 +6880,58 @@ StartupXLOG(void)
 				error_context_stack = errcallback.previous;
 
 				/*
-				 * Update lastReplayedEndRecPtr after this record has been
-				 * successfully replayed.
+				 * Update lastReplayedEndRecPtr and lastReplayedTimestamp
+				 * after this record has been successfully replayed.
 				 */
 				SpinLockAcquire(&XLogCtl->info_lck);
 				XLogCtl->lastReplayedEndRecPtr = EndRecPtr;
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
+				if (LastReplayedTimestamp != 0)
+				{
+					/* If replaying a record produced a timestamp, use that. */
+					XLogCtl->lastReplayedTimestamp = LastReplayedTimestamp;
+					LastReplayedTimestamp = 0;
+				}
+				else
+				{
+					/*
+					 * If we have applied LSNs associated with timestamps
+					 * received by walreceiver, then use the recorded
+					 * timestamp.  We consume from the read end of the
+					 * circular buffer.
+					 */
+					while (XLogCtl->timestampedLsnRead !=
+						   XLogCtl->timestampedLsnWrite &&
+						   XLogCtl->timestampedLsn[XLogCtl->timestampedLsnRead]
+						   <= EndRecPtr)
+					{
+						if (XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead] >
+							XLogCtl->lastReplayedTimestamp)
+						{
+							XLogCtl->lastReplayedTimestamp =
+								XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead];
+							doRequestWalReceiverReply = true;
+						}
+						XLogCtl->timestampedLsnRead =
+							(XLogCtl->timestampedLsnRead + 1) % MAX_TIMESTAMPED_LSNS;
+					}
+				}
 				SpinLockRelease(&XLogCtl->info_lck);
 
+				/*
+				 * If rm_redo reported that it applied a commit record that
+				 * the master is waiting for by calling
+				 * XLogRequestWalReceiverReply, or we encountered a WAL
+				 * location that was associated with a timestamp above, then
+				 * we wake up the receiver so that it notices the updated
+				 * lastReplayedEndRecPtr and sends a reply to the master.
+				 */
+				if (doRequestWalReceiverReply)
+				{
+					doRequestWalReceiverReply = false;
+					WalRcvWakeup();
+				}
+
 				/* Remember this record as the last-applied one */
 				LastRec = ReadRecPtr;
 
@@ -11583,3 +11657,103 @@ SetWalWriterSleeping(bool sleeping)
 	XLogCtl->WalWriterSleeping = sleeping;
 	SpinLockRelease(&XLogCtl->info_lck);
 }
+
+/*
+ * Called by redo code to indicate that the xlog replay loop should wake up
+ * the walreceiver process so that a reply can be sent to the primary.
+ */
+void
+XLogRequestWalReceiverReply(void)
+{
+	doRequestWalReceiverReply = true;
+}
+
+/*
+ * Merge timestamps from keepalive messages with the timestamps from WAL
+ * records, so that we can track lag while idle or while replaying large
+ * amounts of WAL without commit records.  In the former case there is no lag,
+ * and in the latter case we will remember a timestamp that goes with an
+ * arbitrary LSN, and wait for that LSN to be replayed before using the
+ * timestamp.
+ *
+ * This is called by walreceiver on standby servers when keepalive messages
+ * arrive.
+ */
+void
+SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn)
+{
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn == XLogCtl->lastReplayedEndRecPtr)
+	{
+		/*
+		 * That is the last replayed LSN: we are fully replayed, so we can
+		 * update the replay timestamp immediately.
+		 */
+		XLogCtl->lastReplayedTimestamp = timestamp;
+	}
+	else
+	{
+		/*
+		 * There is WAL still to be applied.  We will associate the timestamp
+		 * with this WAL position and wait for it to be replayed.  We add it
+		 * at the 'write' end of the circular buffer of LSN/timestamp
+		 * mappings, which the replay loop will eventually read.
+		 */
+		Index w = XLogCtl->timestampedLsnWrite;
+		Index r = XLogCtl->timestampedLsnRead;
+
+		XLogCtl->timestampedLsn[w] = lsn;
+		XLogCtl->timestampedLsnTime[w] = timestamp;
+
+		/* Advance the write point. */
+		w = (w + 1) % MAX_TIMESTAMPED_LSNS;
+		XLogCtl->timestampedLsnWrite = w;
+		if (w == r)
+		{
+			/*
+			 * The buffer is full.  Advance the read point (throwing away
+			 * oldest values; we will begin to overestimate replay lag, until
+			 * lag decreases to a size our buffer can manage, or the next
+			 * commit record is replayed).
+			 */
+			r = (r + 1) % MAX_TIMESTAMPED_LSNS;
+			XLogCtl->timestampedLsnRead = r;
+		}
+	}
+	SpinLockRelease(&XLogCtl->info_lck);
+}
+
+/*
+ * Set the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary.  This can be called by redo handlers that have
+ * an appropriate timestamp (currently only commit records).  Updating the
+ * shared memory value is deferred until after the redo handler returns.
+ */
+void
+SetXLogReplayTimestamp(TimestampTz timestamp)
+{
+	LastReplayedTimestamp = timestamp;
+}
+
+/*
+ * Get the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the master, and also the most recently applied LSN.  (Note
+ * that the timestamp and the LSN don't necessarily relate to the same
+ * record.)
+ *
+ * This is similar to GetLatestXTime, except that it is not only advanced by
+ * commit records (see SetXLogReplayTimestampAtLsn).
+ */
+TimestampTz
+GetXLogReplayTimestamp(XLogRecPtr *lsn)
+{
+	TimestampTz result;
+
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn)
+		*lsn = XLogCtl->lastReplayedEndRecPtr;
+	result = XLogCtl->lastReplayedTimestamp;
+	SpinLockRelease(&XLogCtl->info_lck);
+
+	return result;
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 923fe58..5fce34f 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -655,8 +655,10 @@ CREATE VIEW pg_stat_replication AS
             W.write_location,
             W.flush_location,
             W.replay_location,
+            W.replay_lag,
             W.sync_priority,
-            W.sync_state
+            W.sync_state,
+            W.causal_reads_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
             pg_stat_get_wal_senders() AS W
     WHERE S.usesysid = U.oid AND
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 7f85b88..73e43e0 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -57,6 +57,11 @@
 #include "utils/builtins.h"
 #include "utils/ps_status.h"
 
+/* GUC variables */
+int causal_reads_timeout;
+bool causal_reads;
+char *causal_reads_standby_names;
+
 /* User-settable parameters for sync rep */
 char	   *SyncRepStandbyNames;
 
@@ -69,7 +74,7 @@ static int	SyncRepWaitMode = SYNC_REP_NO_WAIT;
 
 static void SyncRepQueueInsert(int mode);
 static void SyncRepCancelWait(void);
-static int	SyncRepWakeQueue(bool all, int mode);
+static int	SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn);
 
 static int	SyncRepGetStandbyPriority(void);
 
@@ -83,6 +88,239 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * ===========================================================
  */
 
+static bool
+SyncRepCheckEarlyExit(void)
+{
+	/*
+	 * If a wait for synchronous replication is pending, we can neither
+	 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
+	 * lead the client to believe that the transaction aborted, which
+	 * is not true: it's already committed locally. The former is no good
+	 * either: the client has requested synchronous replication, and is
+	 * entitled to assume that an acknowledged commit is also replicated,
+	 * which might not be true. So in this case we issue a WARNING (which
+	 * some clients may be able to interpret) and shut off further output.
+	 * We do NOT reset ProcDiePending, so that the process will die after
+	 * the commit is cleaned up.
+	 */
+	if (ProcDiePending)
+	{
+		ereport(WARNING,
+				(errcode(ERRCODE_ADMIN_SHUTDOWN),
+				 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * It's unclear what to do if a query cancel interrupt arrives.  We
+	 * can't actually abort at this point, but ignoring the interrupt
+	 * altogether is not helpful, so we just terminate the wait with a
+	 * suitable warning.
+	 */
+	if (QueryCancelPending)
+	{
+		QueryCancelPending = false;
+		ereport(WARNING,
+				(errmsg("canceling wait for synchronous replication due to user request"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * If the postmaster dies, we'll probably never get an
+	 * acknowledgement, because all the wal sender processes will exit. So
+	 * just bail out.
+	 */
+	if (!PostmasterIsAlive())
+	{
+		ProcDiePending = true;
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	return false;
+}
+
+/*
+ * Check if we can stop waiting for causal consistency.  We can stop waiting
+ * when the following conditions are met:
+ *
+ * 1.  All walsenders currently in 'joining' or 'available' state have
+ * applied the target LSN.
+ *
+ * 2.  Any stall periods caused by standbys dropping out of 'available' state
+ * have passed, so that we can be sure that their leases have expired and they
+ * have started rejecting causal reads transactions.
+ *
+ * The output parameter 'waitingFor' is set to the number of nodes we are
+ * currently waiting for.  The output parameters 'stallTimeMillis' is set to
+ * the number of milliseconds we need to wait for to observe any current
+ * commit stall.
+ *
+ * Returns true if commit can return control, because every standby has either
+ * applied the LSN or started rejecting causal_reads transactions.
+ */
+static bool
+CausalReadsCommitCanReturn(XLogRecPtr XactCommitLSN,
+						   int *waitingFor,
+						   long *stallTimeMillis)
+{
+	int i;
+	TimestampTz now;
+
+	/* Count how many joining/available nodes we are waiting for. */
+	*waitingFor = 0;
+	for (i = 0; i < max_wal_senders; ++i)
+	{
+		WalSnd *walsnd = &WalSndCtl->walsnds[i];
+
+		/*
+		 * Assuming atomic read of pid_t, we can check walsnd->pid without
+		 * acquiring the spinlock to avoid memory synchronization costs for
+		 * unused walsender slots.  We see a value that existed sometime at
+		 * least as recently as the last memory barrier.
+		 */
+		if (walsnd->pid != 0)
+		{
+			/*
+			 * We need to hold the spinlock to read LSNs, because we can't be
+			 * sure they can be read atomically.
+			 */
+			SpinLockAcquire(&walsnd->mutex);
+			if (walsnd->pid != 0 && walsnd->causal_reads_state >= WALSNDCRSTATE_JOINING)
+			{
+				if (walsnd->apply < XactCommitLSN)
+					++*waitingFor;
+			}
+			SpinLockRelease(&walsnd->mutex);
+		}
+	}
+
+	/* Check if there is a stall in progress that we need to observe. */
+	now = GetCurrentTimestamp();
+	LWLockAcquire(SyncRepLock, LW_SHARED);
+	if (WalSndCtl->stall_causal_reads_until > now)
+	{
+		long seconds;
+		int usecs;
+
+		/* Compute how long we have to wait, rounded up to nearest ms. */
+		TimestampDifference(now, WalSndCtl->stall_causal_reads_until,
+							&seconds, &usecs);
+		*stallTimeMillis = seconds * 1000 + (usecs + 999) / 1000;
+	}
+	else
+		*stallTimeMillis = 0;
+	LWLockRelease(SyncRepLock);
+
+	/* We are done if we are not waiting for any nodes or stalls. */
+	return *waitingFor == 0 && *stallTimeMillis == 0;
+}
+
+/*
+ * Wait for causal consistency in causal_reads mode, if requested by user.
+ */
+void
+CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN)
+{
+	long stallTimeMillis;
+	int waitingFor;
+
+	/* Leave if we aren't in causal_reads mode. */
+	if (!causal_reads)
+		return;
+
+	for (;;)
+	{
+		/* Reset latch before checking state. */
+		ResetLatch(MyLatch);
+
+		/*
+		 * Join the queue to be woken up if any causal reads joining/available
+		 * standby applies XactCommitLSN, if we aren't already in it.  We
+		 * don't actually know if we need to wait for any peers yet, but we
+		 * have to register just in case before checking the walsenders' state
+		 * to avoid a race condition that could occur if we did it after
+		 * calling CausalReadsCommitCanReturn.  (SyncRepWaitForLSN doesn't
+		 * have to do this because it can check the highest-seen LSN in
+		 * walsndctl->lsn[mode] which is protected by SyncRepLock, the same
+		 * lock as the queues.  We can't do that here, because there is no
+		 * single highest-seen LSN that is useful.  We must check
+		 * walsnd->apply for all relevant walsenders.  Therefore we must
+		 * register for notifications first, so that we can be notified via
+		 * our latch of any standby applying the LSN we're interested in after
+		 * we check but before we start waiting, or we could wait forever for
+		 * something that has already happened.)
+		 */
+		LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+		if (MyProc->syncRepState != SYNC_REP_WAITING)
+		{
+			MyProc->waitLSN = XactCommitLSN;
+			MyProc->syncRepState = SYNC_REP_WAITING;
+			SyncRepQueueInsert(SYNC_REP_WAIT_CAUSAL_READS_APPLY);
+			Assert(SyncRepQueueIsOrderedByLSN(SYNC_REP_WAIT_CAUSAL_READS_APPLY));
+		}
+		LWLockRelease(SyncRepLock);
+
+		/* Check if we're done. */
+		if (CausalReadsCommitCanReturn(XactCommitLSN, &waitingFor, &stallTimeMillis))
+		{
+			SyncRepCancelWait();
+			break;
+		}
+
+		Assert(waitingFor > 0 || stallTimeMillis > 0);
+
+		/* If we aren't actually waiting for any standbys, leave the queue. */
+		if (waitingFor == 0)
+			SyncRepCancelWait();
+
+		/* Update the ps title. */
+		if (update_process_title)
+		{
+			char buffer[80];
+
+			snprintf(buffer, sizeof(buffer),
+					 "waiting for %d peer(s) to apply %X/%X%s",
+					 waitingFor,
+					 (uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN,
+					 stallTimeMillis > 0 ? " (stalling)" : "");
+			set_ps_display(buffer, false);
+		}
+
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit()) /* Calls SyncRepCancelWait() if true. */
+			break;
+
+		/*
+		 * If are still waiting for peers, then we wait for any joining or
+		 * available peer to reach the LSN (or possibly stop being in one of
+		 * those states or go away).
+		 *
+		 * If not, there must be a non-zero stall time, so we wait for that to
+		 * elapse.
+		 */
+		if (waitingFor > 0)
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH, -1);
+		else
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH | WL_TIMEOUT,
+					  stallTimeMillis);
+	}
+
+	/* There is no way out of the loop that could leave us in the queue. */
+	Assert(SHMQueueIsDetached(&(MyProc->syncRepLinks)));
+	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
+	MyProc->waitLSN = 0;
+
+	if (update_process_title)
+		set_ps_display("", false); /* TODO: restore what was there */
+}
+
 /*
  * Wait for synchronous replication, if requested by user.
  *
@@ -180,57 +418,9 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 		if (syncRepState == SYNC_REP_WAIT_COMPLETE)
 			break;
 
-		/*
-		 * If a wait for synchronous replication is pending, we can neither
-		 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
-		 * lead the client to believe that the transaction aborted, which
-		 * is not true: it's already committed locally. The former is no good
-		 * either: the client has requested synchronous replication, and is
-		 * entitled to assume that an acknowledged commit is also replicated,
-		 * which might not be true. So in this case we issue a WARNING (which
-		 * some clients may be able to interpret) and shut off further output.
-		 * We do NOT reset ProcDiePending, so that the process will die after
-		 * the commit is cleaned up.
-		 */
-		if (ProcDiePending)
-		{
-			ereport(WARNING,
-					(errcode(ERRCODE_ADMIN_SHUTDOWN),
-					 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * It's unclear what to do if a query cancel interrupt arrives.  We
-		 * can't actually abort at this point, but ignoring the interrupt
-		 * altogether is not helpful, so we just terminate the wait with a
-		 * suitable warning.
-		 */
-		if (QueryCancelPending)
-		{
-			QueryCancelPending = false;
-			ereport(WARNING,
-					(errmsg("canceling wait for synchronous replication due to user request"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * If the postmaster dies, we'll probably never get an
-		 * acknowledgement, because all the wal sender processes will exit. So
-		 * just bail out.
-		 */
-		if (!PostmasterIsAlive())
-		{
-			ProcDiePending = true;
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit())
 			break;
-		}
 
 		/*
 		 * Wait on latch.  Any condition that should wake us up will set the
@@ -403,6 +593,49 @@ SyncRepGetSynchronousStandby(void)
 }
 
 /*
+ * Check if the current WALSender process's application_name matches a name in
+ * causal_reads_standby_names (including '*' for wildcard).
+ */
+bool
+CausalReadsPotentialStandby(void)
+{
+	char *rawstring;
+	List	   *elemlist;
+	ListCell   *l;
+	bool		found = false;
+
+	/* Need a modifiable copy of string */
+	rawstring = pstrdup(causal_reads_standby_names);
+
+	/* Parse string into list of identifiers */
+	if (!SplitIdentifierString(rawstring, ',', &elemlist))
+	{
+		/* syntax error in list */
+		pfree(rawstring);
+		list_free(elemlist);
+		/* GUC machinery will have already complained - no need to do again */
+		return 0;
+	}
+
+	foreach(l, elemlist)
+	{
+		char	   *standby_name = (char *) lfirst(l);
+
+		if (pg_strcasecmp(standby_name, application_name) == 0 ||
+			pg_strcasecmp(standby_name, "*") == 0)
+		{
+			found = true;
+			break;
+		}
+	}
+
+	pfree(rawstring);
+	list_free(elemlist);
+
+	return found;
+}
+
+/*
  * Update the LSNs on each queue based upon our latest state. This
  * implements a simple policy of first-valid-standby-releases-waiter.
  *
@@ -410,22 +643,27 @@ SyncRepGetSynchronousStandby(void)
  * perhaps also which information we store as well.
  */
 void
-SyncRepReleaseWaiters(void)
+SyncRepReleaseWaiters(bool walsender_cr_available_or_joining)
 {
 	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
+	int			numapply = 0;
+	int			numcausalreadsapply = 0;
+	bool		is_highest_priority_sync_standby;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
-	 * potential standbys then we have nothing to do. If we are still starting
-	 * up, still running base backup or the current flush position is still
-	 * invalid, then leave quickly also.
+	 * potential standbys and not in a state that causal_reads waits for, then
+	 * we have nothing to do. If we are still starting up, still running base
+	 * backup or the current flush position is still invalid, then leave
+	 * quickly also.
 	 */
-	if (MyWalSnd->sync_standby_priority == 0 ||
-		MyWalSnd->state < WALSNDSTATE_STREAMING ||
-		XLogRecPtrIsInvalid(MyWalSnd->flush))
+	if (!walsender_cr_available_or_joining &&
+		(MyWalSnd->sync_standby_priority == 0 ||
+		 MyWalSnd->state < WALSNDSTATE_STREAMING ||
+		 XLogRecPtrIsInvalid(MyWalSnd->flush)))
 		return;
 
 	/*
@@ -435,45 +673,77 @@ SyncRepReleaseWaiters(void)
 	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
 	syncWalSnd = SyncRepGetSynchronousStandby();
 
-	/* We should have found ourselves at least */
-	Assert(syncWalSnd != NULL);
+	/*
+	 * If we aren't managing the highest priority standby then make a note of
+	 * that so we can announce a takeover in the log if we ever get that job.
+	 */
+	is_highest_priority_sync_standby = syncWalSnd == MyWalSnd;
+	if (!is_highest_priority_sync_standby)
+		announce_next_takeover = true;
 
 	/*
-	 * If we aren't managing the highest priority standby then just leave.
+	 * If we aren't managing the highest priority standby or a standby in
+	 * causal reads 'joining' or 'available' state, then just leave.
 	 */
-	if (syncWalSnd != MyWalSnd)
+	if (!is_highest_priority_sync_standby && !walsender_cr_available_or_joining)
 	{
 		LWLockRelease(SyncRepLock);
-		announce_next_takeover = true;
 		return;
 	}
 
 	/*
 	 * Set the lsn first so that when we wake backends they will release up to
-	 * this location.
+	 * this location.  For the single-standby synchronous commit levels, we
+	 * only do this if we are the current synchronous standby and we are
+	 * advancing the LSN further than it has been advanced before, so that
+	 * SyncRepWaitForLSN can skip waiting in some cases.
 	 */
-	if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+	if (is_highest_priority_sync_standby)
 	{
-		walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
-		numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
-		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
+		if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
+			numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE,
+										MyWalSnd->write);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->write)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
+			numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH,
+										MyWalSnd->flush);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+			numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY,
+										MyWalSnd->apply);
+		}
 	}
+	/*
+	 * For causal_reads, all walsenders currently in available or joining
+	 * state must reach the LSN on their own, and standbys will reach LSNs in
+	 * any order.  It doesn't make sense to keep the highest seen LSN in a
+	 * single walsndctl->lsn element.  (CausalReadsWaitForLSN has handling for
+	 * LSNs that have already been reached).
+	 */
+	if (walsender_cr_available_or_joining)
+		numcausalreadsapply =
+			SyncRepWakeQueue(false, SYNC_REP_WAIT_CAUSAL_READS_APPLY,
+							 MyWalSnd->apply);
 
 	LWLockRelease(SyncRepLock);
 
-	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X",
+	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%X, %d procs to causal_reads apply",
 		 numwrite, (uint32) (MyWalSnd->write >> 32), (uint32) MyWalSnd->write,
-	   numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush);
+		 numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush,
+		 numapply, (uint32) (MyWalSnd->apply >> 32), (uint32) MyWalSnd->apply,
+		 numcausalreadsapply);
 
 	/*
 	 * If we are managing the highest priority standby, though we weren't
 	 * prior to this, then announce we are now the sync standby.
 	 */
-	if (announce_next_takeover)
+	if (is_highest_priority_sync_standby && announce_next_takeover)
 	{
 		announce_next_takeover = false;
 		ereport(LOG,
@@ -548,9 +818,8 @@ SyncRepGetStandbyPriority(void)
  * Must hold SyncRepLock.
  */
 static int
-SyncRepWakeQueue(bool all, int mode)
+SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn)
 {
-	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	PGPROC	   *proc = NULL;
 	PGPROC	   *thisproc = NULL;
 	int			numprocs = 0;
@@ -567,7 +836,7 @@ SyncRepWakeQueue(bool all, int mode)
 		/*
 		 * Assume the queue is ordered by LSN
 		 */
-		if (!all && walsndctl->lsn[mode] < proc->waitLSN)
+		if (!all && lsn < proc->waitLSN)
 			return numprocs;
 
 		/*
@@ -627,7 +896,7 @@ SyncRepUpdateSyncStandbysDefined(void)
 			int			i;
 
 			for (i = 0; i < NUM_SYNC_REP_WAIT_MODE; i++)
-				SyncRepWakeQueue(true, i);
+				SyncRepWakeQueue(true, i, InvalidXLogRecPtr);
 		}
 
 		/*
@@ -679,13 +948,31 @@ SyncRepQueueIsOrderedByLSN(int mode)
 #endif
 
 /*
+ * Make sure that CausalReadsWaitForLSN can't return until after the given
+ * lease expiry time has been reached.
+ *
+ * Wake up all backends waiting in CausalReadsWaitForLSN, because the set of
+ * available/joining peers has changed, and there is a new stall time they
+ * need to observe.
+ */
+void
+CausalReadsBeginStall(TimestampTz lease_expiry_time)
+{
+	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+	WalSndCtl->stall_causal_reads_until =
+		Max(WalSndCtl->stall_causal_reads_until, lease_expiry_time);
+	SyncRepWakeQueue(true, SYNC_REP_WAIT_CAUSAL_READS_APPLY, InvalidXLogRecPtr);
+	LWLockRelease(SyncRepLock);
+}
+
+/*
  * ===========================================================
  * Synchronous Replication functions executed by any process
  * ===========================================================
  */
 
 bool
-check_synchronous_standby_names(char **newval, void **extra, GucSource source)
+check_standby_names(char **newval, void **extra, GucSource source)
 {
 	char	   *rawstring;
 	List	   *elemlist;
@@ -728,6 +1015,9 @@ assign_synchronous_commit(int newval, void *extra)
 		case SYNCHRONOUS_COMMIT_REMOTE_FLUSH:
 			SyncRepWaitMode = SYNC_REP_WAIT_FLUSH;
 			break;
+		case SYNCHRONOUS_COMMIT_REMOTE_APPLY:
+			SyncRepWaitMode = SYNC_REP_WAIT_APPLY;
+			break;
 		default:
 			SyncRepWaitMode = SYNC_REP_NO_WAIT;
 			break;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 7b36e02..777f05c 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -55,6 +55,7 @@
 #include "libpq/pqformat.h"
 #include "libpq/pqsignal.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "replication/walsender.h"
 #include "storage/ipc.h"
@@ -101,6 +102,7 @@ static uint32 recvOff = 0;
  */
 static volatile sig_atomic_t got_SIGHUP = false;
 static volatile sig_atomic_t got_SIGTERM = false;
+static volatile sig_atomic_t got_SIGUSR2 = false;
 
 /*
  * LogstreamResult indicates the byte positions that we have already
@@ -145,14 +147,33 @@ static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
 static void XLogWalRcvSendReply(bool force, bool requestReply);
 static void XLogWalRcvSendHSFeedback(bool immed);
-static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
+static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+								  TimestampTz *causalReadsUntil);
 
 /* Signal handlers */
 static void WalRcvSigHupHandler(SIGNAL_ARGS);
 static void WalRcvSigUsr1Handler(SIGNAL_ARGS);
+static void WalRcvSigUsr2Handler(SIGNAL_ARGS);
 static void WalRcvShutdownHandler(SIGNAL_ARGS);
 static void WalRcvQuickDieHandler(SIGNAL_ARGS);
 
+static void WalRcvBlockSigUsr2(void)
+{
+	sigset_t mask;
+
+	sigemptyset(&mask);
+	sigaddset(&mask, SIGUSR2);
+	sigprocmask(SIG_BLOCK, &mask, NULL);
+}
+
+static void WalRcvUnblockSigUsr2(void)
+{
+	sigset_t mask;
+
+	sigemptyset(&mask);
+	sigaddset(&mask, SIGUSR2);
+	sigprocmask(SIG_UNBLOCK, &mask, NULL);
+}
 
 static void
 ProcessWalRcvInterrupts(void)
@@ -200,6 +221,7 @@ WalReceiverMain(void)
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz last_recv_timestamp;
 	bool		ping_sent;
+	bool		forceReply;
 
 	/*
 	 * WalRcv should be set up already (if we are a backend, we inherit this
@@ -251,6 +273,7 @@ WalReceiverMain(void)
 
 	/* Initialise to a sanish value */
 	walrcv->lastMsgSendTime = walrcv->lastMsgReceiptTime = walrcv->latestWalEndTime = GetCurrentTimestamp();
+	walrcv->causalReadsLease = 0;
 
 	SpinLockRelease(&walrcv->mutex);
 
@@ -268,7 +291,7 @@ WalReceiverMain(void)
 	pqsignal(SIGALRM, SIG_IGN);
 	pqsignal(SIGPIPE, SIG_IGN);
 	pqsignal(SIGUSR1, WalRcvSigUsr1Handler);
-	pqsignal(SIGUSR2, SIG_IGN);
+	pqsignal(SIGUSR2, WalRcvSigUsr2Handler);
 
 	/* Reset some signals that are accepted by postmaster but not here */
 	pqsignal(SIGCHLD, SIG_DFL);
@@ -299,6 +322,10 @@ WalReceiverMain(void)
 	/* Unblock signals (they were blocked when the postmaster forked us) */
 	PG_SETMASK(&UnBlockSig);
 
+	/* Block SIGUSR2 (we unblock it only during network waits). */
+	WalRcvBlockSigUsr2();
+	got_SIGUSR2 = false;
+
 	/* Establish the connection to the primary for XLOG streaming */
 	EnableWalRcvImmediateExit();
 	walrcv_connect(conninfo);
@@ -408,7 +435,9 @@ WalReceiverMain(void)
 				}
 
 				/* Wait a while for data to arrive */
+				WalRcvUnblockSigUsr2();
 				len = walrcv_receive(NAPTIME_PER_CYCLE, &buf);
+				WalRcvBlockSigUsr2();
 				if (len != 0)
 				{
 					/*
@@ -439,11 +468,21 @@ WalReceiverMain(void)
 							endofwal = true;
 							break;
 						}
+						WalRcvUnblockSigUsr2();
 						len = walrcv_receive(0, &buf);
+						WalRcvBlockSigUsr2();
+					}
+
+					if (got_SIGUSR2)
+					{
+						/* The recovery process asked us to force a reply. */
+						got_SIGUSR2 = false;
+						forceReply = true;
 					}
 
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(false, false);
+					XLogWalRcvSendReply(forceReply, false);
+					forceReply = false;
 
 					/*
 					 * If we've written some records, flush them to disk and
@@ -498,7 +537,15 @@ WalReceiverMain(void)
 						}
 					}
 
-					XLogWalRcvSendReply(requestReply, requestReply);
+					/* Check if the startup process has signaled us. */
+					if (got_SIGUSR2)
+					{
+						got_SIGUSR2 = false;
+						forceReply = true;
+					}
+
+					XLogWalRcvSendReply(requestReply || forceReply, requestReply);
+					forceReply = false;
 					XLogWalRcvSendHSFeedback(false);
 				}
 			}
@@ -740,6 +787,13 @@ WalRcvSigUsr1Handler(SIGNAL_ARGS)
 	errno = save_errno;
 }
 
+/* SIGUSR2: used to receive wakeups from recovery */
+static void
+WalRcvSigUsr2Handler(SIGNAL_ARGS)
+{
+	got_SIGUSR2 = true;
+}
+
 /* SIGTERM: set flag for main loop, or shutdown immediately if safe */
 static void
 WalRcvShutdownHandler(SIGNAL_ARGS)
@@ -800,6 +854,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 	XLogRecPtr	walEnd;
 	TimestampTz sendTime;
 	bool		replyRequested;
+	TimestampTz causalReadsLease;
 
 	resetStringInfo(&incoming_message);
 
@@ -820,7 +875,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				walEnd = pq_getmsgint64(&incoming_message);
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, NULL);
 
 				buf += hdrlen;
 				len -= hdrlen;
@@ -830,7 +885,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 		case 'k':				/* Keepalive */
 			{
 				/* copy message to StringInfo */
-				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
+				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char) + sizeof(int64);
 				if (len != hdrlen)
 					ereport(ERROR,
 							(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -842,8 +897,12 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
 				replyRequested = pq_getmsgbyte(&incoming_message);
+				causalReadsLease = IntegerTimestampToTimestampTz(
+					pq_getmsgint64(&incoming_message));
+				ProcessWalSndrMessage(walEnd, sendTime, &causalReadsLease);
 
-				ProcessWalSndrMessage(walEnd, sendTime);
+				/* Remember primary's timestamp at this WAL location. */
+				SetXLogReplayTimestampAtLsn(sendTime, walEnd);
 
 				/* If the primary requested a reply, send one immediately */
 				if (replyRequested)
@@ -1037,6 +1096,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	XLogRecPtr	applyPtr;
 	static TimestampTz sendTime = 0;
 	TimestampTz now;
+	TimestampTz applyTimestamp = 0;
 
 	/*
 	 * If the user doesn't want status to be reported to the master, be sure
@@ -1068,7 +1128,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	/* Construct a new message */
 	writePtr = LogstreamResult.Write;
 	flushPtr = LogstreamResult.Flush;
-	applyPtr = GetXLogReplayRecPtr(NULL);
+	applyTimestamp = GetXLogReplayTimestamp(&applyPtr);
 
 	resetStringInfo(&reply_message);
 	pq_sendbyte(&reply_message, 'r');
@@ -1076,6 +1136,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	pq_sendint64(&reply_message, flushPtr);
 	pq_sendint64(&reply_message, applyPtr);
 	pq_sendint64(&reply_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&reply_message, TimestampTzToIntegerTimestamp(applyTimestamp));
 	pq_sendbyte(&reply_message, requestReply ? 1 : 0);
 
 	/* Send it */
@@ -1174,15 +1235,52 @@ XLogWalRcvSendHSFeedback(bool immed)
  * Update shared memory status upon receiving a message from primary.
  *
  * 'walEnd' and 'sendTime' are the end-of-WAL and timestamp of the latest
- * message, reported by primary.
+ * message, reported by primary.  'causalReadsLease' is a pointer to
+ * the time the primary promises that this standby can safely claim to be
+ * causally consistent, to 0 if it cannot, or a NULL pointer for no change.
  */
 static void
-ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
+ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+					  TimestampTz *causalReadsLease)
 {
 	WalRcvData *walrcv = WalRcv;
 
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
 
+	/* Sanity check for the causalReadsLease time. */
+	if (causalReadsLease != NULL && *causalReadsLease != 0)
+	{
+		/* Deduce max_clock_skew from the causalReadsLease and sendTime. */
+#ifdef HAVE_INT64_TIMESTAMP
+		int64 diffMillis = (*causalReadsLease - sendTime) / 1000;
+#else
+		int64 diffMillis = (*causalReadsLease - sendTime) * 1000;
+#endif
+		int64 max_clock_skew = diffMillis / (CAUSAL_READS_CLOCK_SKEW_RATIO - 1);
+
+		if (sendTime > TimestampTzPlusMilliseconds(lastMsgReceiptTime, max_clock_skew))
+		{
+			/*
+			 * The primary's clock is more than max_clock_skew + network
+			 * latency ahead of the standby's clock.  (If the primary's clock
+			 * is more than max_clock_skew ahead of the standby's clock, but
+			 * by less than the network latency, then there isn't much we can
+			 * do to detect that; but it still seems useful to have this basic
+			 * sanity check for wildly misconfigured servers.)
+			 */
+			elog(LOG, "the primary server's clock time is too far ahead");
+			causalReadsLease = NULL;
+		}
+		/*
+		 * We could also try to detect cases where sendTime is more than
+		 * max_clock_skew in the past according to the standby's clock, but
+		 * that is indistinguishable from network latency/buffering, so we
+		 * could produce misleading error messages; if we do nothing, the
+		 * consequence is 'standby is not available for causal reads' errors
+		 * which should cause the user to investigate.
+		 */
+	}
+
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
 	if (walrcv->latestWalEnd < walEnd)
@@ -1190,6 +1288,8 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->latestWalEnd = walEnd;
 	walrcv->lastMsgSendTime = sendTime;
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
+	if (causalReadsLease != NULL)
+		walrcv->causalReadsLease = *causalReadsLease;
 	SpinLockRelease(&walrcv->mutex);
 
 	if (log_min_messages <= DEBUG2)
@@ -1222,6 +1322,26 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 }
 
 /*
+ * Wake up the walreceiver if it happens to be blocked in walrcv_receive,
+ * and tell it that a commit record has been applied.
+ *
+ * This is called by the startup process whenever interesting xlog records
+ * are applied, so that walreceiver can check if it needs to send an apply
+ * notification back to the master which may be waiting in a COMMIT with
+ * synchronous_commit = apply or causal_reads = on.
+ *
+ * TODO: This may change -- Simon Riggs suggested latches for this.  Maybe
+ * pipes would work too (and avoid interrupting systems calls and allow for
+ * multiplexed IO with the replication socket).
+ */
+void
+WalRcvWakeup(void)
+{
+	if (WalRcv->pid != 0)
+		kill(WalRcv->pid, SIGUSR2);
+}
+
+/*
  * Return a string constant representing the state. This is used
  * in system functions and views, and should *not* be translated.
  */
diff --git a/src/backend/replication/walreceiverfuncs.c b/src/backend/replication/walreceiverfuncs.c
index 5f6e423..f398a75 100644
--- a/src/backend/replication/walreceiverfuncs.c
+++ b/src/backend/replication/walreceiverfuncs.c
@@ -25,9 +25,11 @@
 
 #include "access/xlog_internal.h"
 #include "postmaster/startup.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "storage/pmsignal.h"
 #include "storage/shmem.h"
+#include "utils/guc.h"
 #include "utils/timestamp.h"
 
 WalRcvData *WalRcv = NULL;
@@ -374,3 +376,21 @@ GetReplicationTransferLatency(void)
 
 	return ms;
 }
+
+/*
+ * Used by snapmgr to check if this standby has a valid lease, granting it the
+ * right to consider itself available for causal reads.
+ */
+bool
+WalRcvCausalReadsAvailable(void)
+{
+	WalRcvData *walrcv = WalRcv;
+	TimestampTz now = GetCurrentTimestamp();
+	bool result;
+
+	SpinLockAcquire(&walrcv->mutex);
+	result = walrcv->causalReadsLease != 0 && now <= walrcv->causalReadsLease;
+	SpinLockRelease(&walrcv->mutex);
+
+	return result;
+}
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index c03e045..b1e5f95 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -153,9 +153,20 @@ static StringInfoData tmpbuf;
  */
 static TimestampTz last_reply_timestamp = 0;
 
+static TimestampTz last_keepalive_timestamp = 0;
+
 /* Have we sent a heartbeat message asking for reply, since last reply? */
 static bool waiting_for_ping_response = false;
 
+/* How long do need to stay in JOINING state? */
+static TimestampTz causal_reads_joining_until = 0;
+
+/* The last causal reads lease sent to the standby. */
+static TimestampTz causal_reads_last_lease = 0;
+
+/* Is this WALSender listed in causal_reads_standby_names? */
+static bool am_potential_causal_reads_standby = false;
+
 /*
  * While streaming WAL in Copy mode, streamingDoneSending is set to true
  * after we have sent CopyDone. We should not send any more CopyData messages
@@ -242,6 +253,57 @@ InitWalSender(void)
 }
 
 /*
+ * If we are exiting unexpectedly, we may need to communicate with concurrent
+ * causal_reads commits to maintain the causal consistency guarantee.
+ */
+static void
+PrepareUncleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * We've lost contact with the standby, but it may still be alive.  We
+		 * can't let any causal_reads transactions return until we've stalled
+		 * for long enough for a zombie standby to start raising errors
+		 * because its lease has expired.
+		 */
+		elog(LOG, "standby \"%s\" is lost (no longer available for causal reads)", application_name);
+		CausalReadsBeginStall(causal_reads_last_lease);
+
+		/*
+		 * We set the state to a lower level _after_ beginning the stall,
+		 * otherwise there would be a tiny window where commits could return
+		 * without observing the stall.
+		 */
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
+ * We are shutting down because we received a goodbye message from the
+ * walreceiver.
+ */
+static void
+PrepareCleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * The standby is shutting down, so it won't be running any more
+		 * transactions.  It is therefore safe to stop waiting for it, and no
+		 * stall is necessary.
+		 */
+		elog(LOG, "standby \"%s\" is leaving (no longer available for causal reads)", application_name);
+
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
  * Clean up after an error.
  *
  * WAL sender processes don't use transactions like regular backends do.
@@ -264,7 +326,10 @@ WalSndErrorCleanup(void)
 
 	replication_active = false;
 	if (walsender_ready_to_stop)
+	{
+		PrepareUncleanExit();
 		proc_exit(0);
+	}
 
 	/* Revert back to startup state */
 	WalSndSetState(WALSNDSTATE_STARTUP);
@@ -276,6 +341,8 @@ WalSndErrorCleanup(void)
 static void
 WalSndShutdown(void)
 {
+	PrepareUncleanExit();
+
 	/*
 	 * Reset whereToSendOutput to prevent ereport from attempting to send any
 	 * more messages to the standby.
@@ -1386,6 +1453,7 @@ ProcessRepliesIfAny(void)
 		if (r < 0)
 		{
 			/* unexpected error or EOF */
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1402,6 +1470,7 @@ ProcessRepliesIfAny(void)
 		resetStringInfo(&reply_message);
 		if (pq_getmessage(&reply_message, 0))
 		{
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1451,6 +1520,7 @@ ProcessRepliesIfAny(void)
 				 * 'X' means that the standby is closing down the socket.
 				 */
 			case 'X':
+				PrepareCleanExit();
 				proc_exit(0);
 
 			default:
@@ -1543,15 +1613,29 @@ ProcessStandbyReplyMessage(void)
 	XLogRecPtr	writePtr,
 				flushPtr,
 				applyPtr;
+	int			applyLagMs;
 	bool		replyRequested;
+	TimestampTz now = GetCurrentTimestamp();
+	TimestampTz applyTimestamp;
 
 	/* the caller already consumed the msgtype byte */
 	writePtr = pq_getmsgint64(&reply_message);
 	flushPtr = pq_getmsgint64(&reply_message);
 	applyPtr = pq_getmsgint64(&reply_message);
 	(void) pq_getmsgint64(&reply_message);		/* sendTime; not used ATM */
+	applyTimestamp = IntegerTimestampToTimestampTz(pq_getmsgint64(&reply_message));
 	replyRequested = pq_getmsgbyte(&reply_message);
 
+	/* Compute the apply lag in milliseconds. */
+	if (applyTimestamp == 0)
+		applyLagMs = -1;
+	else
+#ifdef HAVE_INT64_TIMESTAMP
+		applyLagMs = (now - applyTimestamp) / 1000;
+#else
+		applyLagMs = (now - applyTimestamp) * 1000.0;
+#endif
+
 	elog(DEBUG2, "write %X/%X flush %X/%X apply %X/%X%s",
 		 (uint32) (writePtr >> 32), (uint32) writePtr,
 		 (uint32) (flushPtr >> 32), (uint32) flushPtr,
@@ -1568,16 +1652,91 @@ ProcessStandbyReplyMessage(void)
 	 */
 	{
 		WalSnd *walsnd = MyWalSnd;
+		WalSndCausalReadsState causal_reads_state = walsnd->causal_reads_state;
+		bool causal_reads_state_changed = false;
+
+		/*
+		 * Handle causal reads state transitions, if a causal_reads_timeout is
+		 * configured, this standby is listed in causal_reads_standby_names,
+		 * and we are a primary database (not a cascading standby).
+		 */
+		if (causal_reads_timeout != 0 &&
+			am_potential_causal_reads_standby &&
+			!am_cascading_walsender)
+		{
+			if (applyLagMs >= 0 && applyLagMs < causal_reads_timeout)
+			{
+				if (causal_reads_state == WALSNDCRSTATE_UNAVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_JOINING;
+					causal_reads_joining_until =
+						TimestampTzPlusMilliseconds(now, causal_reads_timeout);
+					causal_reads_state_changed = true;
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING &&
+						 now >= causal_reads_joining_until)
+				{
+					causal_reads_state = WALSNDCRSTATE_AVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+			else
+			{
+				if (causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+					/*
+					 * We are dropping a causal reads available standby, so we
+					 * mustn't let any commit command that is waiting in
+					 * CausalReadsWaitForLSN return until we are sure that the
+					 * standby definitely knows that it's not available and
+					 * starts raising errors for causal_reads transactions.
+					 * TODO: We could just wait until the standby acks that
+					 * its lease has been cancelled, and start numbering
+					 * keepalives and sending the number back in replies, so
+					 * we know it's acking the right message; then lagging
+					 * standbys would be less disruptive, but for now we just
+					 * wait for the lease to expire, as we do when we lose
+					 * contact with a standby, for the sake of simplicity.
+					 */
+					CausalReadsBeginStall(causal_reads_last_lease);
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING)
+				{
+					/*
+					 * Dropping a joining standby doesn't require a stall,
+					 * because the standby doesn't think it's available, so
+					 * it's already raising the error for causal_reads
+					 * transactions.
+					 */
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+		}
 
 		SpinLockAcquire(&walsnd->mutex);
 		walsnd->write = writePtr;
 		walsnd->flush = flushPtr;
 		walsnd->apply = applyPtr;
+		walsnd->applyLagMs = applyLagMs;
+		walsnd->causal_reads_state = causal_reads_state;
 		SpinLockRelease(&walsnd->mutex);
+
+		if (causal_reads_state_changed)
+		{
+			WalSndKeepalive(true);
+			elog(LOG, "standby \"%s\" is %s", application_name,
+				 causal_reads_state == WALSNDCRSTATE_UNAVAILABLE ? "unavailable for causal reads" :
+				 causal_reads_state == WALSNDCRSTATE_JOINING ? "joining as a causal reads standby..." :
+				 causal_reads_state == WALSNDCRSTATE_AVAILABLE ? "available for causal reads" :
+				 "UNKNOWN");
+		}
 	}
 
 	if (!am_cascading_walsender)
-		SyncRepReleaseWaiters();
+		SyncRepReleaseWaiters(MyWalSnd->causal_reads_state >= WALSNDCRSTATE_JOINING);
 
 	/*
 	 * Advance our local xmin horizon when the client confirmed a flush.
@@ -1724,27 +1883,34 @@ WalSndComputeSleeptime(TimestampTz now)
 {
 	long		sleeptime = 10000;		/* 10 s */
 
-	if (wal_sender_timeout > 0 && last_reply_timestamp > 0)
+	if ((wal_sender_timeout > 0 || causal_reads_timeout > 0) && last_reply_timestamp > 0)
 	{
 		TimestampTz wakeup_time;
 		long		sec_to_timeout;
 		int			microsec_to_timeout;
 
-		/*
-		 * At the latest stop sleeping once wal_sender_timeout has been
-		 * reached.
-		 */
-		wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-												  wal_sender_timeout);
-
-		/*
-		 * If no ping has been sent yet, wakeup when it's time to do so.
-		 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
-		 * the timeout passed without a response.
-		 */
-		if (!waiting_for_ping_response)
+		if (causal_reads_timeout != 0)
+			wakeup_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+													  causal_reads_timeout /
+													  CAUSAL_READS_KEEPALIVE_RATIO);
+		else
+		{
+			/*
+			 * At the latest stop sleeping once wal_sender_timeout has been
+			 * reached.
+			 */
 			wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-													  wal_sender_timeout / 2);
+													  wal_sender_timeout);
+
+			/*
+			 * If no ping has been sent yet, wakeup when it's time to do so.
+			 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
+			 * the timeout passed without a response.
+			 */
+			if (!waiting_for_ping_response)
+				wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+														  wal_sender_timeout / 2);
+		}
 
 		/* Compute relative time until wakeup. */
 		TimestampDifference(now, wakeup_time,
@@ -1765,15 +1931,28 @@ static void
 WalSndCheckTimeOut(TimestampTz now)
 {
 	TimestampTz timeout;
+	int allowed_time;
 
 	/* don't bail out if we're doing something that doesn't require timeouts */
 	if (last_reply_timestamp <= 0)
 		return;
 
+	/*
+	 * If a causal_reads_timeout is configured, it is used instead of
+	 * wal_sender_timeout.  Ideally we'd use causal_reads_timeout / 2 +
+	 * allowance for network latency, but since walreceiver can become quite
+	 * bogged down fsyncing WAL we allow more tolerance.  (This could be
+	 * tightened up once standbys hand writing off to the WAL writer).
+	 */
+	if (causal_reads_timeout != 0)
+		allowed_time = causal_reads_timeout;
+	else
+		allowed_time = wal_sender_timeout;
+
 	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
-										  wal_sender_timeout);
+										  allowed_time);
 
-	if (wal_sender_timeout > 0 && now >= timeout)
+	if (allowed_time > 0 && now >= timeout)
 	{
 		/*
 		 * Since typically expiration of replication timeout means
@@ -1806,6 +1985,9 @@ WalSndLoop(WalSndSendDataCallback send_data)
 	last_reply_timestamp = GetCurrentTimestamp();
 	waiting_for_ping_response = false;
 
+	/* Check if we are managing potential causal_reads standby. */
+	am_potential_causal_reads_standby = CausalReadsPotentialStandby();
+
 	/*
 	 * Loop until we reach the end of this timeline or the client requests to
 	 * stop streaming.
@@ -1966,6 +2148,7 @@ InitWalSenderSlot(void)
 			walsnd->flush = InvalidXLogRecPtr;
 			walsnd->apply = InvalidXLogRecPtr;
 			walsnd->state = WALSNDSTATE_STARTUP;
+			walsnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
 			walsnd->latch = &MyProc->procLatch;
 			SpinLockRelease(&walsnd->mutex);
 			/* don't need the lock anymore */
@@ -2735,6 +2918,24 @@ WalSndGetStateString(WalSndState state)
 	return "UNKNOWN";
 }
 
+/*
+ * Return a string constant representing the causal reads state. This is used
+ * in system views, and should *not* be translated.
+ */
+static const char *
+WalSndGetCausalReadsStateString(WalSndCausalReadsState causal_reads_state)
+{
+	switch (causal_reads_state)
+	{
+		case WALSNDCRSTATE_UNAVAILABLE:
+			return "unavailable";
+		case WALSNDCRSTATE_JOINING:
+			return "joining";
+		case WALSNDCRSTATE_AVAILABLE:
+			return "available";
+	}
+	return "UNKNOWN";
+}
 
 /*
  * Returns activity of walsenders, including pids and xlog locations sent to
@@ -2743,7 +2944,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	8
+#define PG_STAT_GET_WAL_SENDERS_COLS	10
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2791,8 +2992,10 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		XLogRecPtr	write;
 		XLogRecPtr	flush;
 		XLogRecPtr	apply;
+		int			applyLagMs;
 		int			priority;
 		WalSndState state;
+		WalSndCausalReadsState causalReadsState;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
 		bool		nulls[PG_STAT_GET_WAL_SENDERS_COLS];
 
@@ -2802,9 +3005,11 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		SpinLockAcquire(&walsnd->mutex);
 		sentPtr = walsnd->sentPtr;
 		state = walsnd->state;
+		causalReadsState = walsnd->causal_reads_state;
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
+		applyLagMs = walsnd->applyLagMs;
 		priority = walsnd->sync_standby_priority;
 		SpinLockRelease(&walsnd->mutex);
 
@@ -2839,6 +3044,23 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				nulls[5] = true;
 			values[5] = LSNGetDatum(apply);
 
+			if (applyLagMs < 0)
+				nulls[6] = true;
+			else
+			{
+				Interval *applyLagInterval = palloc(sizeof(Interval));
+
+				applyLagInterval->month = 0;
+				applyLagInterval->day = 0;
+#ifdef HAVE_INT64_TIMESTAMP
+				applyLagInterval->time = applyLagMs * 1000;
+#else
+				applyLagInterval->time = applyLagMs / 1000.0;
+#endif
+				nulls[6] = false;
+				values[6] = IntervalPGetDatum(applyLagInterval);
+			}
+
 			/*
 			 * Treat a standby such as a pg_basebackup background process
 			 * which always returns an invalid flush location, as an
@@ -2846,18 +3068,21 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 			 */
 			priority = XLogRecPtrIsInvalid(walsnd->flush) ? 0 : priority;
 
-			values[6] = Int32GetDatum(priority);
+			values[7] = Int32GetDatum(priority);
 
 			/*
 			 * More easily understood version of standby state. This is purely
 			 * informational, not different from priority.
 			 */
 			if (priority == 0)
-				values[7] = CStringGetTextDatum("async");
+				values[8] = CStringGetTextDatum("async");
 			else if (walsnd == sync_standby)
-				values[7] = CStringGetTextDatum("sync");
+				values[8] = CStringGetTextDatum("sync");
 			else
-				values[7] = CStringGetTextDatum("potential");
+				values[8] = CStringGetTextDatum("potential");
+
+			values[9] =
+				CStringGetTextDatum(WalSndGetCausalReadsStateString(causalReadsState));
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
@@ -2877,14 +3102,52 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 static void
 WalSndKeepalive(bool requestReply)
 {
+	TimestampTz now;
+	TimestampTz causal_reads_lease;
+
 	elog(DEBUG2, "sending replication keepalive");
 
+	/*
+	 * If the walsender currently deems the standby to be available for causal
+	 * reads, then it grants a causal reads lease.  The lease authorizes the
+	 * standby to consider itself available for causal reads until a short
+	 * time in the future.  The primary promises to uphold the causal reads
+	 * guarantee until that time, by stalling commits until the the lease has
+	 * expired if necessary.
+	 */
+	now = GetCurrentTimestamp();
+	if (MyWalSnd->causal_reads_state < WALSNDCRSTATE_AVAILABLE)
+		causal_reads_lease = 0; /* Not available, no lease granted. */
+	else
+	{
+		/*
+		 * Since this timestamp is being sent to the standby where it will be
+		 * compared against a time generated by the standby's system clock, we
+		 * must consider clock skew.  First, we decide on a maximum tolerable
+		 * difference between system clocks.  If the primary's clock is ahead
+		 * of the standby's by more than this, then all bets are off (the
+		 * standby could falsely believe it has a valid lease).  If the
+		 * primary's clock is behind the standby's by more than this, then the
+		 * standby will err the other way and generate spurious errors in
+		 * causal_reads mode.  Rather than having a separate GUC for this, we
+		 * derive it from causal_reads_timeout.
+		 */
+		int max_clock_skew = causal_reads_timeout / CAUSAL_READS_CLOCK_SKEW_RATIO;
+
+		/* Compute and remember the expiry time of the lease we're granting. */
+		causal_reads_last_lease = TimestampTzPlusMilliseconds(now, causal_reads_timeout);
+		/* The version we'll send to the standby is adjusted to tolerate clock skew. */
+		causal_reads_lease =
+			TimestampTzPlusMilliseconds(causal_reads_last_lease, -max_clock_skew);
+	}
+
 	/* construct the message... */
 	resetStringInfo(&output_message);
 	pq_sendbyte(&output_message, 'k');
 	pq_sendint64(&output_message, sentPtr);
-	pq_sendint64(&output_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(now));
 	pq_sendbyte(&output_message, requestReply ? 1 : 0);
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(causal_reads_lease));
 
 	/* ... and send it wrapped in CopyData */
 	pq_putmessage_noblock('d', output_message.data, output_message.len);
@@ -2902,23 +3165,32 @@ WalSndKeepaliveIfNecessary(TimestampTz now)
 	 * Don't send keepalive messages if timeouts are globally disabled or
 	 * we're doing something not partaking in timeouts.
 	 */
-	if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
+	if ((wal_sender_timeout <= 0 && causal_reads_timeout == 0) || last_reply_timestamp <= 0)
 		return;
 
-	if (waiting_for_ping_response)
+	if (waiting_for_ping_response && causal_reads_timeout == 0)
 		return;
 
 	/*
 	 * If half of wal_sender_timeout has lapsed without receiving any reply
 	 * from the standby, send a keep-alive message to the standby requesting
 	 * an immediate reply.
+	 *
+	 * If causal_reads_timeout has been configured, use it to control
+	 * keepalive intervals rather than wal_sender_timeout.
 	 */
-	ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-											wal_sender_timeout / 2);
+	if (causal_reads_timeout != 0)
+		ping_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+												causal_reads_timeout /
+												CAUSAL_READS_KEEPALIVE_RATIO);
+	else
+		ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+												wal_sender_timeout / 2);
 	if (now >= ping_time)
 	{
 		WalSndKeepalive(true);
 		waiting_for_ping_response = true;
+		last_keepalive_timestamp = now;
 
 		/* Try to flush pending output to the client */
 		if (pq_flush_if_writable() != 0)
diff --git a/src/backend/utils/adt/timestamp.c b/src/backend/utils/adt/timestamp.c
index 6871092..826df5b 100644
--- a/src/backend/utils/adt/timestamp.c
+++ b/src/backend/utils/adt/timestamp.c
@@ -1611,6 +1611,20 @@ IntegerTimestampToTimestampTz(int64 timestamp)
 #endif
 
 /*
+ * TimestampTzToIntegerTimestamp -- convert a native timestamp to int64 format
+ *
+ * When compiled with --enable-integer-datetimes, this is implemented as a
+ * no-op macro.
+ */
+#ifndef HAVE_INT64_TIMESTAMP
+int64
+TimestampTzToIntegerTimestamp(TimestampTz timestamp)
+{
+	return timestamp * 1000000;
+}
+#endif
+
+/*
  * TimestampDifference -- convert the difference between two timestamps
  *		into integer seconds and microseconds
  *
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 38ba82f..891a9b0 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -351,6 +351,7 @@ static const struct config_enum_entry constraint_exclusion_options[] = {
 static const struct config_enum_entry synchronous_commit_options[] = {
 	{"local", SYNCHRONOUS_COMMIT_LOCAL_FLUSH, false},
 	{"remote_write", SYNCHRONOUS_COMMIT_REMOTE_WRITE, false},
+	{"apply", SYNCHRONOUS_COMMIT_REMOTE_APPLY, false},
 	{"on", SYNCHRONOUS_COMMIT_ON, false},
 	{"off", SYNCHRONOUS_COMMIT_OFF, false},
 	{"true", SYNCHRONOUS_COMMIT_ON, true},
@@ -1618,6 +1619,16 @@ static struct config_bool ConfigureNamesBool[] =
 		NULL, NULL, NULL
 	},
 
+	{
+		{"causal_reads", PGC_USERSET, REPLICATION_STANDBY,
+		 gettext_noop("Enables causal reads."),
+		 NULL
+		},
+		&causal_reads,
+		false,
+		NULL, NULL, NULL
+	},
+
 	/* End-of-list marker */
 	{
 		{NULL, 0, 0, NULL, NULL}, NULL, false, NULL, NULL, NULL
@@ -1776,6 +1787,17 @@ static struct config_int ConfigureNamesInt[] =
 	},
 
 	{
+		{"causal_reads_timeout", PGC_SIGHUP, REPLICATION_STANDBY,
+			gettext_noop("Sets the maximum apply lag before causal reads standbys are no longer available."),
+			NULL,
+			GUC_UNIT_MS
+		},
+		&causal_reads_timeout,
+		0, 0, INT_MAX,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"max_connections", PGC_POSTMASTER, CONN_AUTH_SETTINGS,
 			gettext_noop("Sets the maximum number of concurrent connections."),
 			NULL
@@ -3361,7 +3383,18 @@ static struct config_string ConfigureNamesString[] =
 		},
 		&SyncRepStandbyNames,
 		"",
-		check_synchronous_standby_names, NULL, NULL
+		check_standby_names, NULL, NULL
+	},
+
+	{
+		{"causal_reads_standby_names", PGC_SIGHUP, REPLICATION_MASTER,
+			gettext_noop("List of names of potential causal reads standbys."),
+			NULL,
+			GUC_LIST_INPUT
+		},
+		&causal_reads_standby_names,
+		"*",
+		check_standby_names, NULL, NULL
 	},
 
 	{
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index 63e908d..966b7fb 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -46,8 +46,11 @@
 
 #include "access/transam.h"
 #include "access/xact.h"
+#include "access/xlog.h"
 #include "lib/pairingheap.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
+#include "replication/walreceiver.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
 #include "storage/procarray.h"
@@ -209,6 +212,16 @@ GetTransactionSnapshot(void)
 				 "cannot take query snapshot during a parallel operation");
 
 		/*
+		 * In causal_reads mode on a standby, check if we have definitely
+		 * applied WAL for any COMMIT that returned successfully on the
+		 * primary.
+		 *
+		 * TODO: Machine readable error code?
+		 */
+		if (causal_reads && RecoveryInProgress() && !WalRcvCausalReadsAvailable())
+			elog(ERROR, "standby is not available for causal reads");
+
+		/*
 		 * In transaction-snapshot mode, the first snapshot must live until
 		 * end of xact regardless of what the caller does with it, so we must
 		 * make a copy of it rather than returning CurrentSnapshotData
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index ebeb582..4037dc6 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -60,7 +60,11 @@ typedef enum
 	SYNCHRONOUS_COMMIT_LOCAL_FLUSH,		/* wait for local flush only */
 	SYNCHRONOUS_COMMIT_REMOTE_WRITE,	/* wait for local flush and remote
 										 * write */
-	SYNCHRONOUS_COMMIT_REMOTE_FLUSH		/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_FLUSH,	/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_APPLY,	/* wait for local flush and remote
+										 * apply */
+	SYNCHRONOUS_COMMIT_CONSISTENT_APPLY /* wait for local flusha and remote
+										   apply with causal consistency */
 }	SyncCommitLevel;
 
 /* Define the default setting for synchonous_commit */
@@ -144,10 +148,13 @@ typedef void (*SubXactCallback) (SubXactEvent event, SubTransactionId mySubid,
  * EOXact... routines which run at the end of the original transaction
  * completion.
  */
+#define XACT_COMPLETION_SYNC_APPLY_FEEDBACK		(1U << 29)
 #define XACT_COMPLETION_UPDATE_RELCACHE_FILE	(1U << 30)
 #define XACT_COMPLETION_FORCE_SYNC_COMMIT		(1U << 31)
 
 /* Access macros for above flags */
+#define XactCompletionSyncApplyFeedback(xinfo) \
+	(!!(xinfo & XACT_COMPLETION_SYNC_APPLY_FEEDBACK))
 #define XactCompletionRelcacheInitFileInval(xinfo) \
 	(!!(xinfo & XACT_COMPLETION_UPDATE_RELCACHE_FILE))
 #define XactCompletionForceSyncCommit(xinfo) \
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 3de337a..2b11392 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -235,6 +235,9 @@ extern void GetXLogReceiptTime(TimestampTz *rtime, bool *fromStream);
 extern XLogRecPtr GetXLogReplayRecPtr(TimeLineID *replayTLI);
 extern XLogRecPtr GetXLogInsertRecPtr(void);
 extern XLogRecPtr GetXLogWriteRecPtr(void);
+extern void SetXLogReplayTimestamp(TimestampTz timestamp);
+extern void SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn);
+extern TimestampTz GetXLogReplayTimestamp(XLogRecPtr *lsn);
 extern bool RecoveryIsPaused(void);
 extern void SetRecoveryPause(bool recoveryPause);
 extern TimestampTz GetLatestXTime(void);
@@ -267,6 +270,8 @@ extern bool CheckPromoteSignal(void);
 extern void WakeupRecovery(void);
 extern void SetWalWriterSleeping(bool sleeping);
 
+extern void XLogRequestWalReceiverReply(void);
+
 extern void assign_max_wal_size(int newval, void *extra);
 extern void assign_checkpoint_completion_target(double newval, void *extra);
 
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 6beefa2..81763d7 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2684,7 +2684,7 @@ DATA(insert OID = 1936 (  pg_stat_get_backend_idset		PGNSP PGUID 12 1 100 0 0 f
 DESCR("statistics: currently active backend IDs");
 DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f f f t s r 1 0 2249 "23" "{23,26,23,26,25,25,25,16,1184,1184,1184,1184,869,25,23,28,28,16,25,25,23,16,25}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{pid,datid,pid,usesysid,application_name,state,query,waiting,xact_start,query_start,backend_start,state_change,client_addr,client_hostname,client_port,backend_xid,backend_xmin,ssl,sslversion,sslcipher,sslbits,sslcompression,sslclientdn}" _null_ _null_ pg_stat_get_activity _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active backends");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,23,25}" "{o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25,25}" "{o,o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state,causal_reads_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index 96e059b..76a4ee9 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -23,14 +23,34 @@
 #define SYNC_REP_NO_WAIT		-1
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
+#define SYNC_REP_WAIT_APPLY		2
+#define SYNC_REP_WAIT_CAUSAL_READS_APPLY 3
 
-#define NUM_SYNC_REP_WAIT_MODE	2
+#define NUM_SYNC_REP_WAIT_MODE	4
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
 #define SYNC_REP_WAITING			1
 #define SYNC_REP_WAIT_COMPLETE		2
 
+/*
+ * ratio of causal_read_timeout to max_clock_skew (4 means than the maximum
+ * tolerated clock difference between primary and standbys using causal_reads
+ * is 1/4 of causal_reads_timeout)
+ */
+#define CAUSAL_READS_CLOCK_SKEW_RATIO 4
+
+/*
+ * ratio of causal_reads_timeout to keepalive time (2 means that the effective
+ * keepalive time is 1/2 of the causal_reads_timeout GUC when it is non-zero)
+ */
+#define CAUSAL_READS_KEEPALIVE_RATIO 2
+
+/* GUC variables */
+extern int causal_reads_timeout;
+extern bool causal_reads;
+extern char *causal_reads_standby_names;
+
 /* user-settable parameters for synchronous replication */
 extern char *SyncRepStandbyNames;
 
@@ -42,16 +62,23 @@ extern void SyncRepCleanupAtProcExit(void);
 
 /* called by wal sender */
 extern void SyncRepInitConfig(void);
-extern void SyncRepReleaseWaiters(void);
+extern void SyncRepReleaseWaiters(bool walsender_cr_available_or_joining);
 
 /* called by checkpointer */
 extern void SyncRepUpdateSyncStandbysDefined(void);
 
+/* called by user backend (xact.c) */
+extern void CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN);
+
+/* called by wal sender */
+extern void CausalReadsBeginStall(TimestampTz lease_expiry_time);
+extern bool CausalReadsPotentialStandby(void);
+
 /* forward declaration to avoid pulling in walsender_private.h */
 struct WalSnd;
 extern struct WalSnd *SyncRepGetSynchronousStandby(void);
 
-extern bool check_synchronous_standby_names(char **newval, void **extra, GucSource source);
+extern bool check_standby_names(char **newval, void **extra, GucSource source);
 extern void assign_synchronous_commit(int newval, void *extra);
 
 #endif   /* _SYNCREP_H */
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 6eacb09..f9eca28 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -80,6 +80,13 @@ typedef struct
 	TimeLineID	receivedTLI;
 
 	/*
+	 * causallyReadsLease is the time until which the primary has authorized
+	 * this standby to consider itself available for causal_reads mode, or 0
+	 * for not authorized.
+	 */
+	TimestampTz causalReadsLease;
+
+	/*
 	 * latestChunkStart is the starting byte position of the current "batch"
 	 * of received WAL.  It's actually the same as the previous value of
 	 * receivedUpto before the last flush to disk.  Startup process can use
@@ -162,5 +169,8 @@ extern void RequestXLogStreaming(TimeLineID tli, XLogRecPtr recptr,
 extern XLogRecPtr GetWalRcvWriteRecPtr(XLogRecPtr *latestChunkStart, TimeLineID *receiveTLI);
 extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
+extern void WalRcvWakeup(void);
+
+extern bool WalRcvCausalReadsAvailable(void);
 
 #endif   /* _WALRECEIVER_H */
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 7794aa5..81a2776 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -27,6 +27,13 @@ typedef enum WalSndState
 	WALSNDSTATE_STREAMING
 } WalSndState;
 
+typedef enum WalSndCausalReadsState
+{
+	WALSNDCRSTATE_UNAVAILABLE = 0,
+	WALSNDCRSTATE_JOINING,
+	WALSNDCRSTATE_AVAILABLE
+} WalSndCausalReadsState;
+
 /*
  * Each walsender has a WalSnd struct in shared memory.
  */
@@ -34,6 +41,7 @@ typedef struct WalSnd
 {
 	pid_t		pid;			/* this walsender's process id, or 0 */
 	WalSndState state;			/* this walsender's state */
+	WalSndCausalReadsState causal_reads_state; /* the walsender's causal reads state */
 	XLogRecPtr	sentPtr;		/* WAL has been sent up to this point */
 	bool		needreload;		/* does currently-open file need to be
 								 * reloaded? */
@@ -46,6 +54,7 @@ typedef struct WalSnd
 	XLogRecPtr	write;
 	XLogRecPtr	flush;
 	XLogRecPtr	apply;
+	int			applyLagMs;
 
 	/* Protects shared variables shown above. */
 	slock_t		mutex;
@@ -88,6 +97,12 @@ typedef struct
 	 */
 	bool		sync_standbys_defined;
 
+	/*
+	 * Until when must commits in causal_reads stall?  This is used to wait
+	 * for causal reads leases to expire.
+	 */
+	TimestampTz	stall_causal_reads_until;
+
 	WalSnd		walsnds[FLEXIBLE_ARRAY_MEMBER];
 } WalSndCtlData;
 
diff --git a/src/include/utils/timestamp.h b/src/include/utils/timestamp.h
index fbead3a..297e151 100644
--- a/src/include/utils/timestamp.h
+++ b/src/include/utils/timestamp.h
@@ -227,9 +227,11 @@ extern bool TimestampDifferenceExceeds(TimestampTz start_time,
 #ifndef HAVE_INT64_TIMESTAMP
 extern int64 GetCurrentIntegerTimestamp(void);
 extern TimestampTz IntegerTimestampToTimestampTz(int64 timestamp);
+extern int64 TimestampTzToIntegerTimestamp(TimestampTz timestamp);
 #else
 #define GetCurrentIntegerTimestamp()	GetCurrentTimestamp()
 #define IntegerTimestampToTimestampTz(timestamp) (timestamp)
+#define TimestampTzToIntegerTimestamp(timestamp) (timestamp)
 #endif
 
 extern TimestampTz time_t_to_timestamptz(pg_time_t tm);
#27Michael Paquier
michael.paquier@gmail.com
In reply to: Thomas Munro (#26)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Wed, Jan 20, 2016 at 1:12 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Wed, Dec 30, 2015 at 5:15 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Wed, Nov 18, 2015 at 11:50 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

Here is a new version of the patch with a few small improvements:
...
[causal-reads-v3.patch]

That didn't apply after 6e7b3359 (which fix a typo in a comment that I
moved). Here is a new version that does.

That one conflicts with b1a9bad9e744857291c7d5516080527da8219854, so
here is a new version.

You should try to register it to a CF, though it may be too late for
9.6 if that's rather intrusive.
--
Michael

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#28Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Thomas Munro (#26)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

Hi Thomas,

On 2016/01/20 13:12, Thomas Munro wrote:

That one conflicts with b1a9bad9e744857291c7d5516080527da8219854, so
here is a new version.

-    if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+    if (is_highest_priority_sync_standby)

[ ... ]

- if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
- {
- walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
- numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);

[ ... ]

+        if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->write)
+        {
+            walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
+            numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH,
+                                        MyWalSnd->flush);

There seems to be a copy-pasto there - shouldn't that be:

+ if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)

Thanks,
Amit

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#29Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Amit Langote (#28)
1 attachment(s)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Wed, Feb 3, 2016 at 10:59 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

There seems to be a copy-pasto there - shouldn't that be:

+ if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)

Indeed, thanks! New patch attached.

--
Thomas Munro
http://www.enterprisedb.com

Attachments:

causal-reads-v6.patchapplication/octet-stream; name=causal-reads-v6.patchDownload
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index b0d5440..8bfc510 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,10 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
+	{
+		CausalReadsWaitForLSN(XactLastRecEnd);
 		SyncRepWaitForLSN(XactLastRecEnd);
+	}
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5117,6 +5120,13 @@ XactLogCommitRecord(TimestampTz commit_time,
 		xl_xinfo.xinfo |= XACT_COMPLETION_FORCE_SYNC_COMMIT;
 
 	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this commit is applied.
+	 */
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY || causal_reads)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
+	/*
 	 * Relcache invalidations requires information about the current database
 	 * and so does logical decoding.
 	 */
@@ -5452,6 +5462,19 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 	if (XactCompletionForceSyncCommit(parsed->xinfo))
 		XLogFlush(lsn);
 
+	/*
+	 * Record the primary's timestamp for the commit record, so it can be used
+	 * for tracking replay lag.
+	 */
+	SetXLogReplayTimestamp(parsed->xact_time);
+
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit or causal reads), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 /*
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index a2846c4..e64f897 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -80,6 +80,8 @@ extern uint32 bootstrap_data_checksum_version;
 #define PROMOTE_SIGNAL_FILE		"promote"
 #define FALLBACK_PROMOTE_SIGNAL_FILE "fallback_promote"
 
+/* Size of the circular buffer of timestamped LSNs. */
+#define MAX_TIMESTAMPED_LSNS 8192
 
 /* User-settable parameters */
 int			max_wal_size = 64;	/* 1 GB */
@@ -345,6 +347,12 @@ static XLogRecPtr RedoRecPtr;
 static bool doPageWrites;
 
 /*
+ * doRequestWalReceiverReply is used by recovery code to ask the main recovery
+ * loop to trigger a walreceiver reply.
+ */
+static bool doRequestWalReceiverReply;
+
+/*
  * RedoStartLSN points to the checkpoint's REDO location which is specified
  * in a backup label file, backup history file or control file. In standby
  * mode, XLOG streaming usually starts from the position where an invalid
@@ -356,6 +364,13 @@ static bool doPageWrites;
  */
 static XLogRecPtr RedoStartLSN = InvalidXLogRecPtr;
 
+/*
+ * LastReplayedTimestamp can be set by redo handlers when they apply a record
+ * that carries a timestamp, by calling SetXLogReplayedTimestamp.  The xlog
+ * apply loop can then update the value in shared memory.
+ */
+static TimestampTz LastReplayedTimestamp = 0;
+
 /*----------
  * Shared-memory data structures for XLOG control
  *
@@ -630,6 +645,21 @@ typedef struct XLogCtlData
 	/* current effective recovery target timeline */
 	TimeLineID	RecoveryTargetTLI;
 
+	/* timestamp from the most recently applied record that carried a timestamp. */
+	TimestampTz lastReplayedTimestamp;
+
+	/*
+	 * We maintain a circular buffer of LSNs and associated timestamps.
+	 * Walreceiver writes into it using information from timestamps, and the
+	 * startup recovery process reads from it and notifies walreceiver when
+	 * LSNs are replayed so that the timestamps can be fed back to the
+	 * upstream server, to track lag.
+	 */
+	Index		timestampedLsnRead;
+	Index		timestampedLsnWrite;
+	XLogRecPtr	timestampedLsn[MAX_TIMESTAMPED_LSNS];
+	TimestampTz	timestampedLsnTime[MAX_TIMESTAMPED_LSNS];
+
 	/*
 	 * timestamp of when we started replaying the current chunk of WAL data,
 	 * only relevant for replication or archive recovery
@@ -6849,14 +6879,58 @@ StartupXLOG(void)
 				error_context_stack = errcallback.previous;
 
 				/*
-				 * Update lastReplayedEndRecPtr after this record has been
-				 * successfully replayed.
+				 * Update lastReplayedEndRecPtr and lastReplayedTimestamp
+				 * after this record has been successfully replayed.
 				 */
 				SpinLockAcquire(&XLogCtl->info_lck);
 				XLogCtl->lastReplayedEndRecPtr = EndRecPtr;
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
+				if (LastReplayedTimestamp != 0)
+				{
+					/* If replaying a record produced a timestamp, use that. */
+					XLogCtl->lastReplayedTimestamp = LastReplayedTimestamp;
+					LastReplayedTimestamp = 0;
+				}
+				else
+				{
+					/*
+					 * If we have applied LSNs associated with timestamps
+					 * received by walreceiver, then use the recorded
+					 * timestamp.  We consume from the read end of the
+					 * circular buffer.
+					 */
+					while (XLogCtl->timestampedLsnRead !=
+						   XLogCtl->timestampedLsnWrite &&
+						   XLogCtl->timestampedLsn[XLogCtl->timestampedLsnRead]
+						   <= EndRecPtr)
+					{
+						if (XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead] >
+							XLogCtl->lastReplayedTimestamp)
+						{
+							XLogCtl->lastReplayedTimestamp =
+								XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead];
+							doRequestWalReceiverReply = true;
+						}
+						XLogCtl->timestampedLsnRead =
+							(XLogCtl->timestampedLsnRead + 1) % MAX_TIMESTAMPED_LSNS;
+					}
+				}
 				SpinLockRelease(&XLogCtl->info_lck);
 
+				/*
+				 * If rm_redo reported that it applied a commit record that
+				 * the master is waiting for by calling
+				 * XLogRequestWalReceiverReply, or we encountered a WAL
+				 * location that was associated with a timestamp above, then
+				 * we wake up the receiver so that it notices the updated
+				 * lastReplayedEndRecPtr and sends a reply to the master.
+				 */
+				if (doRequestWalReceiverReply)
+				{
+					doRequestWalReceiverReply = false;
+					WalRcvWakeup();
+				}
+
 				/* Remember this record as the last-applied one */
 				LastRec = ReadRecPtr;
 
@@ -11582,3 +11656,103 @@ SetWalWriterSleeping(bool sleeping)
 	XLogCtl->WalWriterSleeping = sleeping;
 	SpinLockRelease(&XLogCtl->info_lck);
 }
+
+/*
+ * Called by redo code to indicate that the xlog replay loop should wake up
+ * the walreceiver process so that a reply can be sent to the primary.
+ */
+void
+XLogRequestWalReceiverReply(void)
+{
+	doRequestWalReceiverReply = true;
+}
+
+/*
+ * Merge timestamps from keepalive messages with the timestamps from WAL
+ * records, so that we can track lag while idle or while replaying large
+ * amounts of WAL without commit records.  In the former case there is no lag,
+ * and in the latter case we will remember a timestamp that goes with an
+ * arbitrary LSN, and wait for that LSN to be replayed before using the
+ * timestamp.
+ *
+ * This is called by walreceiver on standby servers when keepalive messages
+ * arrive.
+ */
+void
+SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn)
+{
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn == XLogCtl->lastReplayedEndRecPtr)
+	{
+		/*
+		 * That is the last replayed LSN: we are fully replayed, so we can
+		 * update the replay timestamp immediately.
+		 */
+		XLogCtl->lastReplayedTimestamp = timestamp;
+	}
+	else
+	{
+		/*
+		 * There is WAL still to be applied.  We will associate the timestamp
+		 * with this WAL position and wait for it to be replayed.  We add it
+		 * at the 'write' end of the circular buffer of LSN/timestamp
+		 * mappings, which the replay loop will eventually read.
+		 */
+		Index w = XLogCtl->timestampedLsnWrite;
+		Index r = XLogCtl->timestampedLsnRead;
+
+		XLogCtl->timestampedLsn[w] = lsn;
+		XLogCtl->timestampedLsnTime[w] = timestamp;
+
+		/* Advance the write point. */
+		w = (w + 1) % MAX_TIMESTAMPED_LSNS;
+		XLogCtl->timestampedLsnWrite = w;
+		if (w == r)
+		{
+			/*
+			 * The buffer is full.  Advance the read point (throwing away
+			 * oldest values; we will begin to overestimate replay lag, until
+			 * lag decreases to a size our buffer can manage, or the next
+			 * commit record is replayed).
+			 */
+			r = (r + 1) % MAX_TIMESTAMPED_LSNS;
+			XLogCtl->timestampedLsnRead = r;
+		}
+	}
+	SpinLockRelease(&XLogCtl->info_lck);
+}
+
+/*
+ * Set the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary.  This can be called by redo handlers that have
+ * an appropriate timestamp (currently only commit records).  Updating the
+ * shared memory value is deferred until after the redo handler returns.
+ */
+void
+SetXLogReplayTimestamp(TimestampTz timestamp)
+{
+	LastReplayedTimestamp = timestamp;
+}
+
+/*
+ * Get the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the master, and also the most recently applied LSN.  (Note
+ * that the timestamp and the LSN don't necessarily relate to the same
+ * record.)
+ *
+ * This is similar to GetLatestXTime, except that it is not only advanced by
+ * commit records (see SetXLogReplayTimestampAtLsn).
+ */
+TimestampTz
+GetXLogReplayTimestamp(XLogRecPtr *lsn)
+{
+	TimestampTz result;
+
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn)
+		*lsn = XLogCtl->lastReplayedEndRecPtr;
+	result = XLogCtl->lastReplayedTimestamp;
+	SpinLockRelease(&XLogCtl->info_lck);
+
+	return result;
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 923fe58..5fce34f 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -655,8 +655,10 @@ CREATE VIEW pg_stat_replication AS
             W.write_location,
             W.flush_location,
             W.replay_location,
+            W.replay_lag,
             W.sync_priority,
-            W.sync_state
+            W.sync_state,
+            W.causal_reads_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
             pg_stat_get_wal_senders() AS W
     WHERE S.usesysid = U.oid AND
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 7f85b88..3ecc7d7 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -57,6 +57,11 @@
 #include "utils/builtins.h"
 #include "utils/ps_status.h"
 
+/* GUC variables */
+int causal_reads_timeout;
+bool causal_reads;
+char *causal_reads_standby_names;
+
 /* User-settable parameters for sync rep */
 char	   *SyncRepStandbyNames;
 
@@ -69,7 +74,7 @@ static int	SyncRepWaitMode = SYNC_REP_NO_WAIT;
 
 static void SyncRepQueueInsert(int mode);
 static void SyncRepCancelWait(void);
-static int	SyncRepWakeQueue(bool all, int mode);
+static int	SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn);
 
 static int	SyncRepGetStandbyPriority(void);
 
@@ -83,6 +88,239 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * ===========================================================
  */
 
+static bool
+SyncRepCheckEarlyExit(void)
+{
+	/*
+	 * If a wait for synchronous replication is pending, we can neither
+	 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
+	 * lead the client to believe that the transaction aborted, which
+	 * is not true: it's already committed locally. The former is no good
+	 * either: the client has requested synchronous replication, and is
+	 * entitled to assume that an acknowledged commit is also replicated,
+	 * which might not be true. So in this case we issue a WARNING (which
+	 * some clients may be able to interpret) and shut off further output.
+	 * We do NOT reset ProcDiePending, so that the process will die after
+	 * the commit is cleaned up.
+	 */
+	if (ProcDiePending)
+	{
+		ereport(WARNING,
+				(errcode(ERRCODE_ADMIN_SHUTDOWN),
+				 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * It's unclear what to do if a query cancel interrupt arrives.  We
+	 * can't actually abort at this point, but ignoring the interrupt
+	 * altogether is not helpful, so we just terminate the wait with a
+	 * suitable warning.
+	 */
+	if (QueryCancelPending)
+	{
+		QueryCancelPending = false;
+		ereport(WARNING,
+				(errmsg("canceling wait for synchronous replication due to user request"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * If the postmaster dies, we'll probably never get an
+	 * acknowledgement, because all the wal sender processes will exit. So
+	 * just bail out.
+	 */
+	if (!PostmasterIsAlive())
+	{
+		ProcDiePending = true;
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	return false;
+}
+
+/*
+ * Check if we can stop waiting for causal consistency.  We can stop waiting
+ * when the following conditions are met:
+ *
+ * 1.  All walsenders currently in 'joining' or 'available' state have
+ * applied the target LSN.
+ *
+ * 2.  Any stall periods caused by standbys dropping out of 'available' state
+ * have passed, so that we can be sure that their leases have expired and they
+ * have started rejecting causal reads transactions.
+ *
+ * The output parameter 'waitingFor' is set to the number of nodes we are
+ * currently waiting for.  The output parameters 'stallTimeMillis' is set to
+ * the number of milliseconds we need to wait for to observe any current
+ * commit stall.
+ *
+ * Returns true if commit can return control, because every standby has either
+ * applied the LSN or started rejecting causal_reads transactions.
+ */
+static bool
+CausalReadsCommitCanReturn(XLogRecPtr XactCommitLSN,
+						   int *waitingFor,
+						   long *stallTimeMillis)
+{
+	int i;
+	TimestampTz now;
+
+	/* Count how many joining/available nodes we are waiting for. */
+	*waitingFor = 0;
+	for (i = 0; i < max_wal_senders; ++i)
+	{
+		WalSnd *walsnd = &WalSndCtl->walsnds[i];
+
+		/*
+		 * Assuming atomic read of pid_t, we can check walsnd->pid without
+		 * acquiring the spinlock to avoid memory synchronization costs for
+		 * unused walsender slots.  We see a value that existed sometime at
+		 * least as recently as the last memory barrier.
+		 */
+		if (walsnd->pid != 0)
+		{
+			/*
+			 * We need to hold the spinlock to read LSNs, because we can't be
+			 * sure they can be read atomically.
+			 */
+			SpinLockAcquire(&walsnd->mutex);
+			if (walsnd->pid != 0 && walsnd->causal_reads_state >= WALSNDCRSTATE_JOINING)
+			{
+				if (walsnd->apply < XactCommitLSN)
+					++*waitingFor;
+			}
+			SpinLockRelease(&walsnd->mutex);
+		}
+	}
+
+	/* Check if there is a stall in progress that we need to observe. */
+	now = GetCurrentTimestamp();
+	LWLockAcquire(SyncRepLock, LW_SHARED);
+	if (WalSndCtl->stall_causal_reads_until > now)
+	{
+		long seconds;
+		int usecs;
+
+		/* Compute how long we have to wait, rounded up to nearest ms. */
+		TimestampDifference(now, WalSndCtl->stall_causal_reads_until,
+							&seconds, &usecs);
+		*stallTimeMillis = seconds * 1000 + (usecs + 999) / 1000;
+	}
+	else
+		*stallTimeMillis = 0;
+	LWLockRelease(SyncRepLock);
+
+	/* We are done if we are not waiting for any nodes or stalls. */
+	return *waitingFor == 0 && *stallTimeMillis == 0;
+}
+
+/*
+ * Wait for causal consistency in causal_reads mode, if requested by user.
+ */
+void
+CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN)
+{
+	long stallTimeMillis;
+	int waitingFor;
+
+	/* Leave if we aren't in causal_reads mode. */
+	if (!causal_reads)
+		return;
+
+	for (;;)
+	{
+		/* Reset latch before checking state. */
+		ResetLatch(MyLatch);
+
+		/*
+		 * Join the queue to be woken up if any causal reads joining/available
+		 * standby applies XactCommitLSN, if we aren't already in it.  We
+		 * don't actually know if we need to wait for any peers yet, but we
+		 * have to register just in case before checking the walsenders' state
+		 * to avoid a race condition that could occur if we did it after
+		 * calling CausalReadsCommitCanReturn.  (SyncRepWaitForLSN doesn't
+		 * have to do this because it can check the highest-seen LSN in
+		 * walsndctl->lsn[mode] which is protected by SyncRepLock, the same
+		 * lock as the queues.  We can't do that here, because there is no
+		 * single highest-seen LSN that is useful.  We must check
+		 * walsnd->apply for all relevant walsenders.  Therefore we must
+		 * register for notifications first, so that we can be notified via
+		 * our latch of any standby applying the LSN we're interested in after
+		 * we check but before we start waiting, or we could wait forever for
+		 * something that has already happened.)
+		 */
+		LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+		if (MyProc->syncRepState != SYNC_REP_WAITING)
+		{
+			MyProc->waitLSN = XactCommitLSN;
+			MyProc->syncRepState = SYNC_REP_WAITING;
+			SyncRepQueueInsert(SYNC_REP_WAIT_CAUSAL_READS_APPLY);
+			Assert(SyncRepQueueIsOrderedByLSN(SYNC_REP_WAIT_CAUSAL_READS_APPLY));
+		}
+		LWLockRelease(SyncRepLock);
+
+		/* Check if we're done. */
+		if (CausalReadsCommitCanReturn(XactCommitLSN, &waitingFor, &stallTimeMillis))
+		{
+			SyncRepCancelWait();
+			break;
+		}
+
+		Assert(waitingFor > 0 || stallTimeMillis > 0);
+
+		/* If we aren't actually waiting for any standbys, leave the queue. */
+		if (waitingFor == 0)
+			SyncRepCancelWait();
+
+		/* Update the ps title. */
+		if (update_process_title)
+		{
+			char buffer[80];
+
+			snprintf(buffer, sizeof(buffer),
+					 "waiting for %d peer(s) to apply %X/%X%s",
+					 waitingFor,
+					 (uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN,
+					 stallTimeMillis > 0 ? " (stalling)" : "");
+			set_ps_display(buffer, false);
+		}
+
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit()) /* Calls SyncRepCancelWait() if true. */
+			break;
+
+		/*
+		 * If are still waiting for peers, then we wait for any joining or
+		 * available peer to reach the LSN (or possibly stop being in one of
+		 * those states or go away).
+		 *
+		 * If not, there must be a non-zero stall time, so we wait for that to
+		 * elapse.
+		 */
+		if (waitingFor > 0)
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH, -1);
+		else
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH | WL_TIMEOUT,
+					  stallTimeMillis);
+	}
+
+	/* There is no way out of the loop that could leave us in the queue. */
+	Assert(SHMQueueIsDetached(&(MyProc->syncRepLinks)));
+	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
+	MyProc->waitLSN = 0;
+
+	if (update_process_title)
+		set_ps_display("", false); /* TODO: restore what was there */
+}
+
 /*
  * Wait for synchronous replication, if requested by user.
  *
@@ -180,57 +418,9 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 		if (syncRepState == SYNC_REP_WAIT_COMPLETE)
 			break;
 
-		/*
-		 * If a wait for synchronous replication is pending, we can neither
-		 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
-		 * lead the client to believe that the transaction aborted, which
-		 * is not true: it's already committed locally. The former is no good
-		 * either: the client has requested synchronous replication, and is
-		 * entitled to assume that an acknowledged commit is also replicated,
-		 * which might not be true. So in this case we issue a WARNING (which
-		 * some clients may be able to interpret) and shut off further output.
-		 * We do NOT reset ProcDiePending, so that the process will die after
-		 * the commit is cleaned up.
-		 */
-		if (ProcDiePending)
-		{
-			ereport(WARNING,
-					(errcode(ERRCODE_ADMIN_SHUTDOWN),
-					 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * It's unclear what to do if a query cancel interrupt arrives.  We
-		 * can't actually abort at this point, but ignoring the interrupt
-		 * altogether is not helpful, so we just terminate the wait with a
-		 * suitable warning.
-		 */
-		if (QueryCancelPending)
-		{
-			QueryCancelPending = false;
-			ereport(WARNING,
-					(errmsg("canceling wait for synchronous replication due to user request"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * If the postmaster dies, we'll probably never get an
-		 * acknowledgement, because all the wal sender processes will exit. So
-		 * just bail out.
-		 */
-		if (!PostmasterIsAlive())
-		{
-			ProcDiePending = true;
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit())
 			break;
-		}
 
 		/*
 		 * Wait on latch.  Any condition that should wake us up will set the
@@ -403,6 +593,49 @@ SyncRepGetSynchronousStandby(void)
 }
 
 /*
+ * Check if the current WALSender process's application_name matches a name in
+ * causal_reads_standby_names (including '*' for wildcard).
+ */
+bool
+CausalReadsPotentialStandby(void)
+{
+	char *rawstring;
+	List	   *elemlist;
+	ListCell   *l;
+	bool		found = false;
+
+	/* Need a modifiable copy of string */
+	rawstring = pstrdup(causal_reads_standby_names);
+
+	/* Parse string into list of identifiers */
+	if (!SplitIdentifierString(rawstring, ',', &elemlist))
+	{
+		/* syntax error in list */
+		pfree(rawstring);
+		list_free(elemlist);
+		/* GUC machinery will have already complained - no need to do again */
+		return 0;
+	}
+
+	foreach(l, elemlist)
+	{
+		char	   *standby_name = (char *) lfirst(l);
+
+		if (pg_strcasecmp(standby_name, application_name) == 0 ||
+			pg_strcasecmp(standby_name, "*") == 0)
+		{
+			found = true;
+			break;
+		}
+	}
+
+	pfree(rawstring);
+	list_free(elemlist);
+
+	return found;
+}
+
+/*
  * Update the LSNs on each queue based upon our latest state. This
  * implements a simple policy of first-valid-standby-releases-waiter.
  *
@@ -410,22 +643,27 @@ SyncRepGetSynchronousStandby(void)
  * perhaps also which information we store as well.
  */
 void
-SyncRepReleaseWaiters(void)
+SyncRepReleaseWaiters(bool walsender_cr_available_or_joining)
 {
 	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
+	int			numapply = 0;
+	int			numcausalreadsapply = 0;
+	bool		is_highest_priority_sync_standby;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
-	 * potential standbys then we have nothing to do. If we are still starting
-	 * up, still running base backup or the current flush position is still
-	 * invalid, then leave quickly also.
+	 * potential standbys and not in a state that causal_reads waits for, then
+	 * we have nothing to do. If we are still starting up, still running base
+	 * backup or the current flush position is still invalid, then leave
+	 * quickly also.
 	 */
-	if (MyWalSnd->sync_standby_priority == 0 ||
-		MyWalSnd->state < WALSNDSTATE_STREAMING ||
-		XLogRecPtrIsInvalid(MyWalSnd->flush))
+	if (!walsender_cr_available_or_joining &&
+		(MyWalSnd->sync_standby_priority == 0 ||
+		 MyWalSnd->state < WALSNDSTATE_STREAMING ||
+		 XLogRecPtrIsInvalid(MyWalSnd->flush)))
 		return;
 
 	/*
@@ -435,45 +673,77 @@ SyncRepReleaseWaiters(void)
 	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
 	syncWalSnd = SyncRepGetSynchronousStandby();
 
-	/* We should have found ourselves at least */
-	Assert(syncWalSnd != NULL);
+	/*
+	 * If we aren't managing the highest priority standby then make a note of
+	 * that so we can announce a takeover in the log if we ever get that job.
+	 */
+	is_highest_priority_sync_standby = syncWalSnd == MyWalSnd;
+	if (!is_highest_priority_sync_standby)
+		announce_next_takeover = true;
 
 	/*
-	 * If we aren't managing the highest priority standby then just leave.
+	 * If we aren't managing the highest priority standby or a standby in
+	 * causal reads 'joining' or 'available' state, then just leave.
 	 */
-	if (syncWalSnd != MyWalSnd)
+	if (!is_highest_priority_sync_standby && !walsender_cr_available_or_joining)
 	{
 		LWLockRelease(SyncRepLock);
-		announce_next_takeover = true;
 		return;
 	}
 
 	/*
 	 * Set the lsn first so that when we wake backends they will release up to
-	 * this location.
+	 * this location.  For the single-standby synchronous commit levels, we
+	 * only do this if we are the current synchronous standby and we are
+	 * advancing the LSN further than it has been advanced before, so that
+	 * SyncRepWaitForLSN can skip waiting in some cases.
 	 */
-	if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+	if (is_highest_priority_sync_standby)
 	{
-		walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
-		numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
-		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
+		if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
+			numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE,
+										MyWalSnd->write);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
+			numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH,
+										MyWalSnd->flush);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+			numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY,
+										MyWalSnd->apply);
+		}
 	}
+	/*
+	 * For causal_reads, all walsenders currently in available or joining
+	 * state must reach the LSN on their own, and standbys will reach LSNs in
+	 * any order.  It doesn't make sense to keep the highest seen LSN in a
+	 * single walsndctl->lsn element.  (CausalReadsWaitForLSN has handling for
+	 * LSNs that have already been reached).
+	 */
+	if (walsender_cr_available_or_joining)
+		numcausalreadsapply =
+			SyncRepWakeQueue(false, SYNC_REP_WAIT_CAUSAL_READS_APPLY,
+							 MyWalSnd->apply);
 
 	LWLockRelease(SyncRepLock);
 
-	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X",
+	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%X, %d procs to causal_reads apply",
 		 numwrite, (uint32) (MyWalSnd->write >> 32), (uint32) MyWalSnd->write,
-	   numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush);
+		 numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush,
+		 numapply, (uint32) (MyWalSnd->apply >> 32), (uint32) MyWalSnd->apply,
+		 numcausalreadsapply);
 
 	/*
 	 * If we are managing the highest priority standby, though we weren't
 	 * prior to this, then announce we are now the sync standby.
 	 */
-	if (announce_next_takeover)
+	if (is_highest_priority_sync_standby && announce_next_takeover)
 	{
 		announce_next_takeover = false;
 		ereport(LOG,
@@ -548,9 +818,8 @@ SyncRepGetStandbyPriority(void)
  * Must hold SyncRepLock.
  */
 static int
-SyncRepWakeQueue(bool all, int mode)
+SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn)
 {
-	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	PGPROC	   *proc = NULL;
 	PGPROC	   *thisproc = NULL;
 	int			numprocs = 0;
@@ -567,7 +836,7 @@ SyncRepWakeQueue(bool all, int mode)
 		/*
 		 * Assume the queue is ordered by LSN
 		 */
-		if (!all && walsndctl->lsn[mode] < proc->waitLSN)
+		if (!all && lsn < proc->waitLSN)
 			return numprocs;
 
 		/*
@@ -627,7 +896,7 @@ SyncRepUpdateSyncStandbysDefined(void)
 			int			i;
 
 			for (i = 0; i < NUM_SYNC_REP_WAIT_MODE; i++)
-				SyncRepWakeQueue(true, i);
+				SyncRepWakeQueue(true, i, InvalidXLogRecPtr);
 		}
 
 		/*
@@ -679,13 +948,31 @@ SyncRepQueueIsOrderedByLSN(int mode)
 #endif
 
 /*
+ * Make sure that CausalReadsWaitForLSN can't return until after the given
+ * lease expiry time has been reached.
+ *
+ * Wake up all backends waiting in CausalReadsWaitForLSN, because the set of
+ * available/joining peers has changed, and there is a new stall time they
+ * need to observe.
+ */
+void
+CausalReadsBeginStall(TimestampTz lease_expiry_time)
+{
+	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+	WalSndCtl->stall_causal_reads_until =
+		Max(WalSndCtl->stall_causal_reads_until, lease_expiry_time);
+	SyncRepWakeQueue(true, SYNC_REP_WAIT_CAUSAL_READS_APPLY, InvalidXLogRecPtr);
+	LWLockRelease(SyncRepLock);
+}
+
+/*
  * ===========================================================
  * Synchronous Replication functions executed by any process
  * ===========================================================
  */
 
 bool
-check_synchronous_standby_names(char **newval, void **extra, GucSource source)
+check_standby_names(char **newval, void **extra, GucSource source)
 {
 	char	   *rawstring;
 	List	   *elemlist;
@@ -728,6 +1015,9 @@ assign_synchronous_commit(int newval, void *extra)
 		case SYNCHRONOUS_COMMIT_REMOTE_FLUSH:
 			SyncRepWaitMode = SYNC_REP_WAIT_FLUSH;
 			break;
+		case SYNCHRONOUS_COMMIT_REMOTE_APPLY:
+			SyncRepWaitMode = SYNC_REP_WAIT_APPLY;
+			break;
 		default:
 			SyncRepWaitMode = SYNC_REP_NO_WAIT;
 			break;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 7b36e02..777f05c 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -55,6 +55,7 @@
 #include "libpq/pqformat.h"
 #include "libpq/pqsignal.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "replication/walsender.h"
 #include "storage/ipc.h"
@@ -101,6 +102,7 @@ static uint32 recvOff = 0;
  */
 static volatile sig_atomic_t got_SIGHUP = false;
 static volatile sig_atomic_t got_SIGTERM = false;
+static volatile sig_atomic_t got_SIGUSR2 = false;
 
 /*
  * LogstreamResult indicates the byte positions that we have already
@@ -145,14 +147,33 @@ static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
 static void XLogWalRcvSendReply(bool force, bool requestReply);
 static void XLogWalRcvSendHSFeedback(bool immed);
-static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
+static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+								  TimestampTz *causalReadsUntil);
 
 /* Signal handlers */
 static void WalRcvSigHupHandler(SIGNAL_ARGS);
 static void WalRcvSigUsr1Handler(SIGNAL_ARGS);
+static void WalRcvSigUsr2Handler(SIGNAL_ARGS);
 static void WalRcvShutdownHandler(SIGNAL_ARGS);
 static void WalRcvQuickDieHandler(SIGNAL_ARGS);
 
+static void WalRcvBlockSigUsr2(void)
+{
+	sigset_t mask;
+
+	sigemptyset(&mask);
+	sigaddset(&mask, SIGUSR2);
+	sigprocmask(SIG_BLOCK, &mask, NULL);
+}
+
+static void WalRcvUnblockSigUsr2(void)
+{
+	sigset_t mask;
+
+	sigemptyset(&mask);
+	sigaddset(&mask, SIGUSR2);
+	sigprocmask(SIG_UNBLOCK, &mask, NULL);
+}
 
 static void
 ProcessWalRcvInterrupts(void)
@@ -200,6 +221,7 @@ WalReceiverMain(void)
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz last_recv_timestamp;
 	bool		ping_sent;
+	bool		forceReply;
 
 	/*
 	 * WalRcv should be set up already (if we are a backend, we inherit this
@@ -251,6 +273,7 @@ WalReceiverMain(void)
 
 	/* Initialise to a sanish value */
 	walrcv->lastMsgSendTime = walrcv->lastMsgReceiptTime = walrcv->latestWalEndTime = GetCurrentTimestamp();
+	walrcv->causalReadsLease = 0;
 
 	SpinLockRelease(&walrcv->mutex);
 
@@ -268,7 +291,7 @@ WalReceiverMain(void)
 	pqsignal(SIGALRM, SIG_IGN);
 	pqsignal(SIGPIPE, SIG_IGN);
 	pqsignal(SIGUSR1, WalRcvSigUsr1Handler);
-	pqsignal(SIGUSR2, SIG_IGN);
+	pqsignal(SIGUSR2, WalRcvSigUsr2Handler);
 
 	/* Reset some signals that are accepted by postmaster but not here */
 	pqsignal(SIGCHLD, SIG_DFL);
@@ -299,6 +322,10 @@ WalReceiverMain(void)
 	/* Unblock signals (they were blocked when the postmaster forked us) */
 	PG_SETMASK(&UnBlockSig);
 
+	/* Block SIGUSR2 (we unblock it only during network waits). */
+	WalRcvBlockSigUsr2();
+	got_SIGUSR2 = false;
+
 	/* Establish the connection to the primary for XLOG streaming */
 	EnableWalRcvImmediateExit();
 	walrcv_connect(conninfo);
@@ -408,7 +435,9 @@ WalReceiverMain(void)
 				}
 
 				/* Wait a while for data to arrive */
+				WalRcvUnblockSigUsr2();
 				len = walrcv_receive(NAPTIME_PER_CYCLE, &buf);
+				WalRcvBlockSigUsr2();
 				if (len != 0)
 				{
 					/*
@@ -439,11 +468,21 @@ WalReceiverMain(void)
 							endofwal = true;
 							break;
 						}
+						WalRcvUnblockSigUsr2();
 						len = walrcv_receive(0, &buf);
+						WalRcvBlockSigUsr2();
+					}
+
+					if (got_SIGUSR2)
+					{
+						/* The recovery process asked us to force a reply. */
+						got_SIGUSR2 = false;
+						forceReply = true;
 					}
 
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(false, false);
+					XLogWalRcvSendReply(forceReply, false);
+					forceReply = false;
 
 					/*
 					 * If we've written some records, flush them to disk and
@@ -498,7 +537,15 @@ WalReceiverMain(void)
 						}
 					}
 
-					XLogWalRcvSendReply(requestReply, requestReply);
+					/* Check if the startup process has signaled us. */
+					if (got_SIGUSR2)
+					{
+						got_SIGUSR2 = false;
+						forceReply = true;
+					}
+
+					XLogWalRcvSendReply(requestReply || forceReply, requestReply);
+					forceReply = false;
 					XLogWalRcvSendHSFeedback(false);
 				}
 			}
@@ -740,6 +787,13 @@ WalRcvSigUsr1Handler(SIGNAL_ARGS)
 	errno = save_errno;
 }
 
+/* SIGUSR2: used to receive wakeups from recovery */
+static void
+WalRcvSigUsr2Handler(SIGNAL_ARGS)
+{
+	got_SIGUSR2 = true;
+}
+
 /* SIGTERM: set flag for main loop, or shutdown immediately if safe */
 static void
 WalRcvShutdownHandler(SIGNAL_ARGS)
@@ -800,6 +854,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 	XLogRecPtr	walEnd;
 	TimestampTz sendTime;
 	bool		replyRequested;
+	TimestampTz causalReadsLease;
 
 	resetStringInfo(&incoming_message);
 
@@ -820,7 +875,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				walEnd = pq_getmsgint64(&incoming_message);
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, NULL);
 
 				buf += hdrlen;
 				len -= hdrlen;
@@ -830,7 +885,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 		case 'k':				/* Keepalive */
 			{
 				/* copy message to StringInfo */
-				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
+				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char) + sizeof(int64);
 				if (len != hdrlen)
 					ereport(ERROR,
 							(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -842,8 +897,12 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
 				replyRequested = pq_getmsgbyte(&incoming_message);
+				causalReadsLease = IntegerTimestampToTimestampTz(
+					pq_getmsgint64(&incoming_message));
+				ProcessWalSndrMessage(walEnd, sendTime, &causalReadsLease);
 
-				ProcessWalSndrMessage(walEnd, sendTime);
+				/* Remember primary's timestamp at this WAL location. */
+				SetXLogReplayTimestampAtLsn(sendTime, walEnd);
 
 				/* If the primary requested a reply, send one immediately */
 				if (replyRequested)
@@ -1037,6 +1096,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	XLogRecPtr	applyPtr;
 	static TimestampTz sendTime = 0;
 	TimestampTz now;
+	TimestampTz applyTimestamp = 0;
 
 	/*
 	 * If the user doesn't want status to be reported to the master, be sure
@@ -1068,7 +1128,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	/* Construct a new message */
 	writePtr = LogstreamResult.Write;
 	flushPtr = LogstreamResult.Flush;
-	applyPtr = GetXLogReplayRecPtr(NULL);
+	applyTimestamp = GetXLogReplayTimestamp(&applyPtr);
 
 	resetStringInfo(&reply_message);
 	pq_sendbyte(&reply_message, 'r');
@@ -1076,6 +1136,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	pq_sendint64(&reply_message, flushPtr);
 	pq_sendint64(&reply_message, applyPtr);
 	pq_sendint64(&reply_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&reply_message, TimestampTzToIntegerTimestamp(applyTimestamp));
 	pq_sendbyte(&reply_message, requestReply ? 1 : 0);
 
 	/* Send it */
@@ -1174,15 +1235,52 @@ XLogWalRcvSendHSFeedback(bool immed)
  * Update shared memory status upon receiving a message from primary.
  *
  * 'walEnd' and 'sendTime' are the end-of-WAL and timestamp of the latest
- * message, reported by primary.
+ * message, reported by primary.  'causalReadsLease' is a pointer to
+ * the time the primary promises that this standby can safely claim to be
+ * causally consistent, to 0 if it cannot, or a NULL pointer for no change.
  */
 static void
-ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
+ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+					  TimestampTz *causalReadsLease)
 {
 	WalRcvData *walrcv = WalRcv;
 
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
 
+	/* Sanity check for the causalReadsLease time. */
+	if (causalReadsLease != NULL && *causalReadsLease != 0)
+	{
+		/* Deduce max_clock_skew from the causalReadsLease and sendTime. */
+#ifdef HAVE_INT64_TIMESTAMP
+		int64 diffMillis = (*causalReadsLease - sendTime) / 1000;
+#else
+		int64 diffMillis = (*causalReadsLease - sendTime) * 1000;
+#endif
+		int64 max_clock_skew = diffMillis / (CAUSAL_READS_CLOCK_SKEW_RATIO - 1);
+
+		if (sendTime > TimestampTzPlusMilliseconds(lastMsgReceiptTime, max_clock_skew))
+		{
+			/*
+			 * The primary's clock is more than max_clock_skew + network
+			 * latency ahead of the standby's clock.  (If the primary's clock
+			 * is more than max_clock_skew ahead of the standby's clock, but
+			 * by less than the network latency, then there isn't much we can
+			 * do to detect that; but it still seems useful to have this basic
+			 * sanity check for wildly misconfigured servers.)
+			 */
+			elog(LOG, "the primary server's clock time is too far ahead");
+			causalReadsLease = NULL;
+		}
+		/*
+		 * We could also try to detect cases where sendTime is more than
+		 * max_clock_skew in the past according to the standby's clock, but
+		 * that is indistinguishable from network latency/buffering, so we
+		 * could produce misleading error messages; if we do nothing, the
+		 * consequence is 'standby is not available for causal reads' errors
+		 * which should cause the user to investigate.
+		 */
+	}
+
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
 	if (walrcv->latestWalEnd < walEnd)
@@ -1190,6 +1288,8 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->latestWalEnd = walEnd;
 	walrcv->lastMsgSendTime = sendTime;
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
+	if (causalReadsLease != NULL)
+		walrcv->causalReadsLease = *causalReadsLease;
 	SpinLockRelease(&walrcv->mutex);
 
 	if (log_min_messages <= DEBUG2)
@@ -1222,6 +1322,26 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 }
 
 /*
+ * Wake up the walreceiver if it happens to be blocked in walrcv_receive,
+ * and tell it that a commit record has been applied.
+ *
+ * This is called by the startup process whenever interesting xlog records
+ * are applied, so that walreceiver can check if it needs to send an apply
+ * notification back to the master which may be waiting in a COMMIT with
+ * synchronous_commit = apply or causal_reads = on.
+ *
+ * TODO: This may change -- Simon Riggs suggested latches for this.  Maybe
+ * pipes would work too (and avoid interrupting systems calls and allow for
+ * multiplexed IO with the replication socket).
+ */
+void
+WalRcvWakeup(void)
+{
+	if (WalRcv->pid != 0)
+		kill(WalRcv->pid, SIGUSR2);
+}
+
+/*
  * Return a string constant representing the state. This is used
  * in system functions and views, and should *not* be translated.
  */
diff --git a/src/backend/replication/walreceiverfuncs.c b/src/backend/replication/walreceiverfuncs.c
index 5f6e423..f398a75 100644
--- a/src/backend/replication/walreceiverfuncs.c
+++ b/src/backend/replication/walreceiverfuncs.c
@@ -25,9 +25,11 @@
 
 #include "access/xlog_internal.h"
 #include "postmaster/startup.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "storage/pmsignal.h"
 #include "storage/shmem.h"
+#include "utils/guc.h"
 #include "utils/timestamp.h"
 
 WalRcvData *WalRcv = NULL;
@@ -374,3 +376,21 @@ GetReplicationTransferLatency(void)
 
 	return ms;
 }
+
+/*
+ * Used by snapmgr to check if this standby has a valid lease, granting it the
+ * right to consider itself available for causal reads.
+ */
+bool
+WalRcvCausalReadsAvailable(void)
+{
+	WalRcvData *walrcv = WalRcv;
+	TimestampTz now = GetCurrentTimestamp();
+	bool result;
+
+	SpinLockAcquire(&walrcv->mutex);
+	result = walrcv->causalReadsLease != 0 && now <= walrcv->causalReadsLease;
+	SpinLockRelease(&walrcv->mutex);
+
+	return result;
+}
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index c03e045..b1e5f95 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -153,9 +153,20 @@ static StringInfoData tmpbuf;
  */
 static TimestampTz last_reply_timestamp = 0;
 
+static TimestampTz last_keepalive_timestamp = 0;
+
 /* Have we sent a heartbeat message asking for reply, since last reply? */
 static bool waiting_for_ping_response = false;
 
+/* How long do need to stay in JOINING state? */
+static TimestampTz causal_reads_joining_until = 0;
+
+/* The last causal reads lease sent to the standby. */
+static TimestampTz causal_reads_last_lease = 0;
+
+/* Is this WALSender listed in causal_reads_standby_names? */
+static bool am_potential_causal_reads_standby = false;
+
 /*
  * While streaming WAL in Copy mode, streamingDoneSending is set to true
  * after we have sent CopyDone. We should not send any more CopyData messages
@@ -242,6 +253,57 @@ InitWalSender(void)
 }
 
 /*
+ * If we are exiting unexpectedly, we may need to communicate with concurrent
+ * causal_reads commits to maintain the causal consistency guarantee.
+ */
+static void
+PrepareUncleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * We've lost contact with the standby, but it may still be alive.  We
+		 * can't let any causal_reads transactions return until we've stalled
+		 * for long enough for a zombie standby to start raising errors
+		 * because its lease has expired.
+		 */
+		elog(LOG, "standby \"%s\" is lost (no longer available for causal reads)", application_name);
+		CausalReadsBeginStall(causal_reads_last_lease);
+
+		/*
+		 * We set the state to a lower level _after_ beginning the stall,
+		 * otherwise there would be a tiny window where commits could return
+		 * without observing the stall.
+		 */
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
+ * We are shutting down because we received a goodbye message from the
+ * walreceiver.
+ */
+static void
+PrepareCleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * The standby is shutting down, so it won't be running any more
+		 * transactions.  It is therefore safe to stop waiting for it, and no
+		 * stall is necessary.
+		 */
+		elog(LOG, "standby \"%s\" is leaving (no longer available for causal reads)", application_name);
+
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
  * Clean up after an error.
  *
  * WAL sender processes don't use transactions like regular backends do.
@@ -264,7 +326,10 @@ WalSndErrorCleanup(void)
 
 	replication_active = false;
 	if (walsender_ready_to_stop)
+	{
+		PrepareUncleanExit();
 		proc_exit(0);
+	}
 
 	/* Revert back to startup state */
 	WalSndSetState(WALSNDSTATE_STARTUP);
@@ -276,6 +341,8 @@ WalSndErrorCleanup(void)
 static void
 WalSndShutdown(void)
 {
+	PrepareUncleanExit();
+
 	/*
 	 * Reset whereToSendOutput to prevent ereport from attempting to send any
 	 * more messages to the standby.
@@ -1386,6 +1453,7 @@ ProcessRepliesIfAny(void)
 		if (r < 0)
 		{
 			/* unexpected error or EOF */
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1402,6 +1470,7 @@ ProcessRepliesIfAny(void)
 		resetStringInfo(&reply_message);
 		if (pq_getmessage(&reply_message, 0))
 		{
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1451,6 +1520,7 @@ ProcessRepliesIfAny(void)
 				 * 'X' means that the standby is closing down the socket.
 				 */
 			case 'X':
+				PrepareCleanExit();
 				proc_exit(0);
 
 			default:
@@ -1543,15 +1613,29 @@ ProcessStandbyReplyMessage(void)
 	XLogRecPtr	writePtr,
 				flushPtr,
 				applyPtr;
+	int			applyLagMs;
 	bool		replyRequested;
+	TimestampTz now = GetCurrentTimestamp();
+	TimestampTz applyTimestamp;
 
 	/* the caller already consumed the msgtype byte */
 	writePtr = pq_getmsgint64(&reply_message);
 	flushPtr = pq_getmsgint64(&reply_message);
 	applyPtr = pq_getmsgint64(&reply_message);
 	(void) pq_getmsgint64(&reply_message);		/* sendTime; not used ATM */
+	applyTimestamp = IntegerTimestampToTimestampTz(pq_getmsgint64(&reply_message));
 	replyRequested = pq_getmsgbyte(&reply_message);
 
+	/* Compute the apply lag in milliseconds. */
+	if (applyTimestamp == 0)
+		applyLagMs = -1;
+	else
+#ifdef HAVE_INT64_TIMESTAMP
+		applyLagMs = (now - applyTimestamp) / 1000;
+#else
+		applyLagMs = (now - applyTimestamp) * 1000.0;
+#endif
+
 	elog(DEBUG2, "write %X/%X flush %X/%X apply %X/%X%s",
 		 (uint32) (writePtr >> 32), (uint32) writePtr,
 		 (uint32) (flushPtr >> 32), (uint32) flushPtr,
@@ -1568,16 +1652,91 @@ ProcessStandbyReplyMessage(void)
 	 */
 	{
 		WalSnd *walsnd = MyWalSnd;
+		WalSndCausalReadsState causal_reads_state = walsnd->causal_reads_state;
+		bool causal_reads_state_changed = false;
+
+		/*
+		 * Handle causal reads state transitions, if a causal_reads_timeout is
+		 * configured, this standby is listed in causal_reads_standby_names,
+		 * and we are a primary database (not a cascading standby).
+		 */
+		if (causal_reads_timeout != 0 &&
+			am_potential_causal_reads_standby &&
+			!am_cascading_walsender)
+		{
+			if (applyLagMs >= 0 && applyLagMs < causal_reads_timeout)
+			{
+				if (causal_reads_state == WALSNDCRSTATE_UNAVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_JOINING;
+					causal_reads_joining_until =
+						TimestampTzPlusMilliseconds(now, causal_reads_timeout);
+					causal_reads_state_changed = true;
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING &&
+						 now >= causal_reads_joining_until)
+				{
+					causal_reads_state = WALSNDCRSTATE_AVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+			else
+			{
+				if (causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+					/*
+					 * We are dropping a causal reads available standby, so we
+					 * mustn't let any commit command that is waiting in
+					 * CausalReadsWaitForLSN return until we are sure that the
+					 * standby definitely knows that it's not available and
+					 * starts raising errors for causal_reads transactions.
+					 * TODO: We could just wait until the standby acks that
+					 * its lease has been cancelled, and start numbering
+					 * keepalives and sending the number back in replies, so
+					 * we know it's acking the right message; then lagging
+					 * standbys would be less disruptive, but for now we just
+					 * wait for the lease to expire, as we do when we lose
+					 * contact with a standby, for the sake of simplicity.
+					 */
+					CausalReadsBeginStall(causal_reads_last_lease);
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING)
+				{
+					/*
+					 * Dropping a joining standby doesn't require a stall,
+					 * because the standby doesn't think it's available, so
+					 * it's already raising the error for causal_reads
+					 * transactions.
+					 */
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+		}
 
 		SpinLockAcquire(&walsnd->mutex);
 		walsnd->write = writePtr;
 		walsnd->flush = flushPtr;
 		walsnd->apply = applyPtr;
+		walsnd->applyLagMs = applyLagMs;
+		walsnd->causal_reads_state = causal_reads_state;
 		SpinLockRelease(&walsnd->mutex);
+
+		if (causal_reads_state_changed)
+		{
+			WalSndKeepalive(true);
+			elog(LOG, "standby \"%s\" is %s", application_name,
+				 causal_reads_state == WALSNDCRSTATE_UNAVAILABLE ? "unavailable for causal reads" :
+				 causal_reads_state == WALSNDCRSTATE_JOINING ? "joining as a causal reads standby..." :
+				 causal_reads_state == WALSNDCRSTATE_AVAILABLE ? "available for causal reads" :
+				 "UNKNOWN");
+		}
 	}
 
 	if (!am_cascading_walsender)
-		SyncRepReleaseWaiters();
+		SyncRepReleaseWaiters(MyWalSnd->causal_reads_state >= WALSNDCRSTATE_JOINING);
 
 	/*
 	 * Advance our local xmin horizon when the client confirmed a flush.
@@ -1724,27 +1883,34 @@ WalSndComputeSleeptime(TimestampTz now)
 {
 	long		sleeptime = 10000;		/* 10 s */
 
-	if (wal_sender_timeout > 0 && last_reply_timestamp > 0)
+	if ((wal_sender_timeout > 0 || causal_reads_timeout > 0) && last_reply_timestamp > 0)
 	{
 		TimestampTz wakeup_time;
 		long		sec_to_timeout;
 		int			microsec_to_timeout;
 
-		/*
-		 * At the latest stop sleeping once wal_sender_timeout has been
-		 * reached.
-		 */
-		wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-												  wal_sender_timeout);
-
-		/*
-		 * If no ping has been sent yet, wakeup when it's time to do so.
-		 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
-		 * the timeout passed without a response.
-		 */
-		if (!waiting_for_ping_response)
+		if (causal_reads_timeout != 0)
+			wakeup_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+													  causal_reads_timeout /
+													  CAUSAL_READS_KEEPALIVE_RATIO);
+		else
+		{
+			/*
+			 * At the latest stop sleeping once wal_sender_timeout has been
+			 * reached.
+			 */
 			wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-													  wal_sender_timeout / 2);
+													  wal_sender_timeout);
+
+			/*
+			 * If no ping has been sent yet, wakeup when it's time to do so.
+			 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
+			 * the timeout passed without a response.
+			 */
+			if (!waiting_for_ping_response)
+				wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+														  wal_sender_timeout / 2);
+		}
 
 		/* Compute relative time until wakeup. */
 		TimestampDifference(now, wakeup_time,
@@ -1765,15 +1931,28 @@ static void
 WalSndCheckTimeOut(TimestampTz now)
 {
 	TimestampTz timeout;
+	int allowed_time;
 
 	/* don't bail out if we're doing something that doesn't require timeouts */
 	if (last_reply_timestamp <= 0)
 		return;
 
+	/*
+	 * If a causal_reads_timeout is configured, it is used instead of
+	 * wal_sender_timeout.  Ideally we'd use causal_reads_timeout / 2 +
+	 * allowance for network latency, but since walreceiver can become quite
+	 * bogged down fsyncing WAL we allow more tolerance.  (This could be
+	 * tightened up once standbys hand writing off to the WAL writer).
+	 */
+	if (causal_reads_timeout != 0)
+		allowed_time = causal_reads_timeout;
+	else
+		allowed_time = wal_sender_timeout;
+
 	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
-										  wal_sender_timeout);
+										  allowed_time);
 
-	if (wal_sender_timeout > 0 && now >= timeout)
+	if (allowed_time > 0 && now >= timeout)
 	{
 		/*
 		 * Since typically expiration of replication timeout means
@@ -1806,6 +1985,9 @@ WalSndLoop(WalSndSendDataCallback send_data)
 	last_reply_timestamp = GetCurrentTimestamp();
 	waiting_for_ping_response = false;
 
+	/* Check if we are managing potential causal_reads standby. */
+	am_potential_causal_reads_standby = CausalReadsPotentialStandby();
+
 	/*
 	 * Loop until we reach the end of this timeline or the client requests to
 	 * stop streaming.
@@ -1966,6 +2148,7 @@ InitWalSenderSlot(void)
 			walsnd->flush = InvalidXLogRecPtr;
 			walsnd->apply = InvalidXLogRecPtr;
 			walsnd->state = WALSNDSTATE_STARTUP;
+			walsnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
 			walsnd->latch = &MyProc->procLatch;
 			SpinLockRelease(&walsnd->mutex);
 			/* don't need the lock anymore */
@@ -2735,6 +2918,24 @@ WalSndGetStateString(WalSndState state)
 	return "UNKNOWN";
 }
 
+/*
+ * Return a string constant representing the causal reads state. This is used
+ * in system views, and should *not* be translated.
+ */
+static const char *
+WalSndGetCausalReadsStateString(WalSndCausalReadsState causal_reads_state)
+{
+	switch (causal_reads_state)
+	{
+		case WALSNDCRSTATE_UNAVAILABLE:
+			return "unavailable";
+		case WALSNDCRSTATE_JOINING:
+			return "joining";
+		case WALSNDCRSTATE_AVAILABLE:
+			return "available";
+	}
+	return "UNKNOWN";
+}
 
 /*
  * Returns activity of walsenders, including pids and xlog locations sent to
@@ -2743,7 +2944,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	8
+#define PG_STAT_GET_WAL_SENDERS_COLS	10
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2791,8 +2992,10 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		XLogRecPtr	write;
 		XLogRecPtr	flush;
 		XLogRecPtr	apply;
+		int			applyLagMs;
 		int			priority;
 		WalSndState state;
+		WalSndCausalReadsState causalReadsState;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
 		bool		nulls[PG_STAT_GET_WAL_SENDERS_COLS];
 
@@ -2802,9 +3005,11 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		SpinLockAcquire(&walsnd->mutex);
 		sentPtr = walsnd->sentPtr;
 		state = walsnd->state;
+		causalReadsState = walsnd->causal_reads_state;
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
+		applyLagMs = walsnd->applyLagMs;
 		priority = walsnd->sync_standby_priority;
 		SpinLockRelease(&walsnd->mutex);
 
@@ -2839,6 +3044,23 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				nulls[5] = true;
 			values[5] = LSNGetDatum(apply);
 
+			if (applyLagMs < 0)
+				nulls[6] = true;
+			else
+			{
+				Interval *applyLagInterval = palloc(sizeof(Interval));
+
+				applyLagInterval->month = 0;
+				applyLagInterval->day = 0;
+#ifdef HAVE_INT64_TIMESTAMP
+				applyLagInterval->time = applyLagMs * 1000;
+#else
+				applyLagInterval->time = applyLagMs / 1000.0;
+#endif
+				nulls[6] = false;
+				values[6] = IntervalPGetDatum(applyLagInterval);
+			}
+
 			/*
 			 * Treat a standby such as a pg_basebackup background process
 			 * which always returns an invalid flush location, as an
@@ -2846,18 +3068,21 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 			 */
 			priority = XLogRecPtrIsInvalid(walsnd->flush) ? 0 : priority;
 
-			values[6] = Int32GetDatum(priority);
+			values[7] = Int32GetDatum(priority);
 
 			/*
 			 * More easily understood version of standby state. This is purely
 			 * informational, not different from priority.
 			 */
 			if (priority == 0)
-				values[7] = CStringGetTextDatum("async");
+				values[8] = CStringGetTextDatum("async");
 			else if (walsnd == sync_standby)
-				values[7] = CStringGetTextDatum("sync");
+				values[8] = CStringGetTextDatum("sync");
 			else
-				values[7] = CStringGetTextDatum("potential");
+				values[8] = CStringGetTextDatum("potential");
+
+			values[9] =
+				CStringGetTextDatum(WalSndGetCausalReadsStateString(causalReadsState));
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
@@ -2877,14 +3102,52 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 static void
 WalSndKeepalive(bool requestReply)
 {
+	TimestampTz now;
+	TimestampTz causal_reads_lease;
+
 	elog(DEBUG2, "sending replication keepalive");
 
+	/*
+	 * If the walsender currently deems the standby to be available for causal
+	 * reads, then it grants a causal reads lease.  The lease authorizes the
+	 * standby to consider itself available for causal reads until a short
+	 * time in the future.  The primary promises to uphold the causal reads
+	 * guarantee until that time, by stalling commits until the the lease has
+	 * expired if necessary.
+	 */
+	now = GetCurrentTimestamp();
+	if (MyWalSnd->causal_reads_state < WALSNDCRSTATE_AVAILABLE)
+		causal_reads_lease = 0; /* Not available, no lease granted. */
+	else
+	{
+		/*
+		 * Since this timestamp is being sent to the standby where it will be
+		 * compared against a time generated by the standby's system clock, we
+		 * must consider clock skew.  First, we decide on a maximum tolerable
+		 * difference between system clocks.  If the primary's clock is ahead
+		 * of the standby's by more than this, then all bets are off (the
+		 * standby could falsely believe it has a valid lease).  If the
+		 * primary's clock is behind the standby's by more than this, then the
+		 * standby will err the other way and generate spurious errors in
+		 * causal_reads mode.  Rather than having a separate GUC for this, we
+		 * derive it from causal_reads_timeout.
+		 */
+		int max_clock_skew = causal_reads_timeout / CAUSAL_READS_CLOCK_SKEW_RATIO;
+
+		/* Compute and remember the expiry time of the lease we're granting. */
+		causal_reads_last_lease = TimestampTzPlusMilliseconds(now, causal_reads_timeout);
+		/* The version we'll send to the standby is adjusted to tolerate clock skew. */
+		causal_reads_lease =
+			TimestampTzPlusMilliseconds(causal_reads_last_lease, -max_clock_skew);
+	}
+
 	/* construct the message... */
 	resetStringInfo(&output_message);
 	pq_sendbyte(&output_message, 'k');
 	pq_sendint64(&output_message, sentPtr);
-	pq_sendint64(&output_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(now));
 	pq_sendbyte(&output_message, requestReply ? 1 : 0);
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(causal_reads_lease));
 
 	/* ... and send it wrapped in CopyData */
 	pq_putmessage_noblock('d', output_message.data, output_message.len);
@@ -2902,23 +3165,32 @@ WalSndKeepaliveIfNecessary(TimestampTz now)
 	 * Don't send keepalive messages if timeouts are globally disabled or
 	 * we're doing something not partaking in timeouts.
 	 */
-	if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
+	if ((wal_sender_timeout <= 0 && causal_reads_timeout == 0) || last_reply_timestamp <= 0)
 		return;
 
-	if (waiting_for_ping_response)
+	if (waiting_for_ping_response && causal_reads_timeout == 0)
 		return;
 
 	/*
 	 * If half of wal_sender_timeout has lapsed without receiving any reply
 	 * from the standby, send a keep-alive message to the standby requesting
 	 * an immediate reply.
+	 *
+	 * If causal_reads_timeout has been configured, use it to control
+	 * keepalive intervals rather than wal_sender_timeout.
 	 */
-	ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-											wal_sender_timeout / 2);
+	if (causal_reads_timeout != 0)
+		ping_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+												causal_reads_timeout /
+												CAUSAL_READS_KEEPALIVE_RATIO);
+	else
+		ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+												wal_sender_timeout / 2);
 	if (now >= ping_time)
 	{
 		WalSndKeepalive(true);
 		waiting_for_ping_response = true;
+		last_keepalive_timestamp = now;
 
 		/* Try to flush pending output to the client */
 		if (pq_flush_if_writable() != 0)
diff --git a/src/backend/utils/adt/timestamp.c b/src/backend/utils/adt/timestamp.c
index 1525d2a..6ff111f 100644
--- a/src/backend/utils/adt/timestamp.c
+++ b/src/backend/utils/adt/timestamp.c
@@ -1611,6 +1611,20 @@ IntegerTimestampToTimestampTz(int64 timestamp)
 #endif
 
 /*
+ * TimestampTzToIntegerTimestamp -- convert a native timestamp to int64 format
+ *
+ * When compiled with --enable-integer-datetimes, this is implemented as a
+ * no-op macro.
+ */
+#ifndef HAVE_INT64_TIMESTAMP
+int64
+TimestampTzToIntegerTimestamp(TimestampTz timestamp)
+{
+	return timestamp * 1000000;
+}
+#endif
+
+/*
  * TimestampDifference -- convert the difference between two timestamps
  *		into integer seconds and microseconds
  *
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index b8d34b5..b136807 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -351,6 +351,7 @@ static const struct config_enum_entry constraint_exclusion_options[] = {
 static const struct config_enum_entry synchronous_commit_options[] = {
 	{"local", SYNCHRONOUS_COMMIT_LOCAL_FLUSH, false},
 	{"remote_write", SYNCHRONOUS_COMMIT_REMOTE_WRITE, false},
+	{"apply", SYNCHRONOUS_COMMIT_REMOTE_APPLY, false},
 	{"on", SYNCHRONOUS_COMMIT_ON, false},
 	{"off", SYNCHRONOUS_COMMIT_OFF, false},
 	{"true", SYNCHRONOUS_COMMIT_ON, true},
@@ -1618,6 +1619,16 @@ static struct config_bool ConfigureNamesBool[] =
 		NULL, NULL, NULL
 	},
 
+	{
+		{"causal_reads", PGC_USERSET, REPLICATION_STANDBY,
+		 gettext_noop("Enables causal reads."),
+		 NULL
+		},
+		&causal_reads,
+		false,
+		NULL, NULL, NULL
+	},
+
 	/* End-of-list marker */
 	{
 		{NULL, 0, 0, NULL, NULL}, NULL, false, NULL, NULL, NULL
@@ -1776,6 +1787,17 @@ static struct config_int ConfigureNamesInt[] =
 	},
 
 	{
+		{"causal_reads_timeout", PGC_SIGHUP, REPLICATION_STANDBY,
+			gettext_noop("Sets the maximum apply lag before causal reads standbys are no longer available."),
+			NULL,
+			GUC_UNIT_MS
+		},
+		&causal_reads_timeout,
+		0, 0, INT_MAX,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"max_connections", PGC_POSTMASTER, CONN_AUTH_SETTINGS,
 			gettext_noop("Sets the maximum number of concurrent connections."),
 			NULL
@@ -3361,7 +3383,18 @@ static struct config_string ConfigureNamesString[] =
 		},
 		&SyncRepStandbyNames,
 		"",
-		check_synchronous_standby_names, NULL, NULL
+		check_standby_names, NULL, NULL
+	},
+
+	{
+		{"causal_reads_standby_names", PGC_SIGHUP, REPLICATION_MASTER,
+			gettext_noop("List of names of potential causal reads standbys."),
+			NULL,
+			GUC_LIST_INPUT
+		},
+		&causal_reads_standby_names,
+		"*",
+		check_standby_names, NULL, NULL
 	},
 
 	{
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index 63e908d..966b7fb 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -46,8 +46,11 @@
 
 #include "access/transam.h"
 #include "access/xact.h"
+#include "access/xlog.h"
 #include "lib/pairingheap.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
+#include "replication/walreceiver.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
 #include "storage/procarray.h"
@@ -209,6 +212,16 @@ GetTransactionSnapshot(void)
 				 "cannot take query snapshot during a parallel operation");
 
 		/*
+		 * In causal_reads mode on a standby, check if we have definitely
+		 * applied WAL for any COMMIT that returned successfully on the
+		 * primary.
+		 *
+		 * TODO: Machine readable error code?
+		 */
+		if (causal_reads && RecoveryInProgress() && !WalRcvCausalReadsAvailable())
+			elog(ERROR, "standby is not available for causal reads");
+
+		/*
 		 * In transaction-snapshot mode, the first snapshot must live until
 		 * end of xact regardless of what the caller does with it, so we must
 		 * make a copy of it rather than returning CurrentSnapshotData
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index ebeb582..4037dc6 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -60,7 +60,11 @@ typedef enum
 	SYNCHRONOUS_COMMIT_LOCAL_FLUSH,		/* wait for local flush only */
 	SYNCHRONOUS_COMMIT_REMOTE_WRITE,	/* wait for local flush and remote
 										 * write */
-	SYNCHRONOUS_COMMIT_REMOTE_FLUSH		/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_FLUSH,	/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_APPLY,	/* wait for local flush and remote
+										 * apply */
+	SYNCHRONOUS_COMMIT_CONSISTENT_APPLY /* wait for local flusha and remote
+										   apply with causal consistency */
 }	SyncCommitLevel;
 
 /* Define the default setting for synchonous_commit */
@@ -144,10 +148,13 @@ typedef void (*SubXactCallback) (SubXactEvent event, SubTransactionId mySubid,
  * EOXact... routines which run at the end of the original transaction
  * completion.
  */
+#define XACT_COMPLETION_SYNC_APPLY_FEEDBACK		(1U << 29)
 #define XACT_COMPLETION_UPDATE_RELCACHE_FILE	(1U << 30)
 #define XACT_COMPLETION_FORCE_SYNC_COMMIT		(1U << 31)
 
 /* Access macros for above flags */
+#define XactCompletionSyncApplyFeedback(xinfo) \
+	(!!(xinfo & XACT_COMPLETION_SYNC_APPLY_FEEDBACK))
 #define XactCompletionRelcacheInitFileInval(xinfo) \
 	(!!(xinfo & XACT_COMPLETION_UPDATE_RELCACHE_FILE))
 #define XactCompletionForceSyncCommit(xinfo) \
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index ecd30ce..efb9719 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -236,6 +236,9 @@ extern void GetXLogReceiptTime(TimestampTz *rtime, bool *fromStream);
 extern XLogRecPtr GetXLogReplayRecPtr(TimeLineID *replayTLI);
 extern XLogRecPtr GetXLogInsertRecPtr(void);
 extern XLogRecPtr GetXLogWriteRecPtr(void);
+extern void SetXLogReplayTimestamp(TimestampTz timestamp);
+extern void SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn);
+extern TimestampTz GetXLogReplayTimestamp(XLogRecPtr *lsn);
 extern bool RecoveryIsPaused(void);
 extern void SetRecoveryPause(bool recoveryPause);
 extern TimestampTz GetLatestXTime(void);
@@ -268,6 +271,8 @@ extern bool CheckPromoteSignal(void);
 extern void WakeupRecovery(void);
 extern void SetWalWriterSleeping(bool sleeping);
 
+extern void XLogRequestWalReceiverReply(void);
+
 extern void assign_max_wal_size(int newval, void *extra);
 extern void assign_checkpoint_completion_target(double newval, void *extra);
 
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index a2248b4..bce8229 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2702,7 +2702,7 @@ DATA(insert OID = 1936 (  pg_stat_get_backend_idset		PGNSP PGUID 12 1 100 0 0 f
 DESCR("statistics: currently active backend IDs");
 DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f f f t s r 1 0 2249 "23" "{23,26,23,26,25,25,25,16,1184,1184,1184,1184,869,25,23,28,28,16,25,25,23,16,25}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{pid,datid,pid,usesysid,application_name,state,query,waiting,xact_start,query_start,backend_start,state_change,client_addr,client_hostname,client_port,backend_xid,backend_xmin,ssl,sslversion,sslcipher,sslbits,sslcompression,sslclientdn}" _null_ _null_ pg_stat_get_activity _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active backends");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,23,25}" "{o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25,25}" "{o,o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state,causal_reads_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index 96e059b..76a4ee9 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -23,14 +23,34 @@
 #define SYNC_REP_NO_WAIT		-1
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
+#define SYNC_REP_WAIT_APPLY		2
+#define SYNC_REP_WAIT_CAUSAL_READS_APPLY 3
 
-#define NUM_SYNC_REP_WAIT_MODE	2
+#define NUM_SYNC_REP_WAIT_MODE	4
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
 #define SYNC_REP_WAITING			1
 #define SYNC_REP_WAIT_COMPLETE		2
 
+/*
+ * ratio of causal_read_timeout to max_clock_skew (4 means than the maximum
+ * tolerated clock difference between primary and standbys using causal_reads
+ * is 1/4 of causal_reads_timeout)
+ */
+#define CAUSAL_READS_CLOCK_SKEW_RATIO 4
+
+/*
+ * ratio of causal_reads_timeout to keepalive time (2 means that the effective
+ * keepalive time is 1/2 of the causal_reads_timeout GUC when it is non-zero)
+ */
+#define CAUSAL_READS_KEEPALIVE_RATIO 2
+
+/* GUC variables */
+extern int causal_reads_timeout;
+extern bool causal_reads;
+extern char *causal_reads_standby_names;
+
 /* user-settable parameters for synchronous replication */
 extern char *SyncRepStandbyNames;
 
@@ -42,16 +62,23 @@ extern void SyncRepCleanupAtProcExit(void);
 
 /* called by wal sender */
 extern void SyncRepInitConfig(void);
-extern void SyncRepReleaseWaiters(void);
+extern void SyncRepReleaseWaiters(bool walsender_cr_available_or_joining);
 
 /* called by checkpointer */
 extern void SyncRepUpdateSyncStandbysDefined(void);
 
+/* called by user backend (xact.c) */
+extern void CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN);
+
+/* called by wal sender */
+extern void CausalReadsBeginStall(TimestampTz lease_expiry_time);
+extern bool CausalReadsPotentialStandby(void);
+
 /* forward declaration to avoid pulling in walsender_private.h */
 struct WalSnd;
 extern struct WalSnd *SyncRepGetSynchronousStandby(void);
 
-extern bool check_synchronous_standby_names(char **newval, void **extra, GucSource source);
+extern bool check_standby_names(char **newval, void **extra, GucSource source);
 extern void assign_synchronous_commit(int newval, void *extra);
 
 #endif   /* _SYNCREP_H */
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 6eacb09..7f83934 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -80,6 +80,13 @@ typedef struct
 	TimeLineID	receivedTLI;
 
 	/*
+	 * causalReadsLease is the time until which the primary has authorized
+	 * this standby to consider itself available for causal_reads mode, or 0
+	 * for not authorized.
+	 */
+	TimestampTz causalReadsLease;
+
+	/*
 	 * latestChunkStart is the starting byte position of the current "batch"
 	 * of received WAL.  It's actually the same as the previous value of
 	 * receivedUpto before the last flush to disk.  Startup process can use
@@ -162,5 +169,8 @@ extern void RequestXLogStreaming(TimeLineID tli, XLogRecPtr recptr,
 extern XLogRecPtr GetWalRcvWriteRecPtr(XLogRecPtr *latestChunkStart, TimeLineID *receiveTLI);
 extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
+extern void WalRcvWakeup(void);
+
+extern bool WalRcvCausalReadsAvailable(void);
 
 #endif   /* _WALRECEIVER_H */
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 7794aa5..81a2776 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -27,6 +27,13 @@ typedef enum WalSndState
 	WALSNDSTATE_STREAMING
 } WalSndState;
 
+typedef enum WalSndCausalReadsState
+{
+	WALSNDCRSTATE_UNAVAILABLE = 0,
+	WALSNDCRSTATE_JOINING,
+	WALSNDCRSTATE_AVAILABLE
+} WalSndCausalReadsState;
+
 /*
  * Each walsender has a WalSnd struct in shared memory.
  */
@@ -34,6 +41,7 @@ typedef struct WalSnd
 {
 	pid_t		pid;			/* this walsender's process id, or 0 */
 	WalSndState state;			/* this walsender's state */
+	WalSndCausalReadsState causal_reads_state; /* the walsender's causal reads state */
 	XLogRecPtr	sentPtr;		/* WAL has been sent up to this point */
 	bool		needreload;		/* does currently-open file need to be
 								 * reloaded? */
@@ -46,6 +54,7 @@ typedef struct WalSnd
 	XLogRecPtr	write;
 	XLogRecPtr	flush;
 	XLogRecPtr	apply;
+	int			applyLagMs;
 
 	/* Protects shared variables shown above. */
 	slock_t		mutex;
@@ -88,6 +97,12 @@ typedef struct
 	 */
 	bool		sync_standbys_defined;
 
+	/*
+	 * Until when must commits in causal_reads stall?  This is used to wait
+	 * for causal reads leases to expire.
+	 */
+	TimestampTz	stall_causal_reads_until;
+
 	WalSnd		walsnds[FLEXIBLE_ARRAY_MEMBER];
 } WalSndCtlData;
 
diff --git a/src/include/utils/timestamp.h b/src/include/utils/timestamp.h
index fbead3a..297e151 100644
--- a/src/include/utils/timestamp.h
+++ b/src/include/utils/timestamp.h
@@ -227,9 +227,11 @@ extern bool TimestampDifferenceExceeds(TimestampTz start_time,
 #ifndef HAVE_INT64_TIMESTAMP
 extern int64 GetCurrentIntegerTimestamp(void);
 extern TimestampTz IntegerTimestampToTimestampTz(int64 timestamp);
+extern int64 TimestampTzToIntegerTimestamp(TimestampTz timestamp);
 #else
 #define GetCurrentIntegerTimestamp()	GetCurrentTimestamp()
 #define IntegerTimestampToTimestampTz(timestamp) (timestamp)
+#define TimestampTzToIntegerTimestamp(timestamp) (timestamp)
 #endif
 
 extern TimestampTz time_t_to_timestamptz(pg_time_t tm);
#30Thom Brown
thom@linux.com
In reply to: Thomas Munro (#29)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On 3 February 2016 at 10:46, Thomas Munro <thomas.munro@enterprisedb.com> wrote:

On Wed, Feb 3, 2016 at 10:59 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

There seems to be a copy-pasto there - shouldn't that be:

+ if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)

Indeed, thanks! New patch attached.

I've given this a test drive, and it works exactly as described.

But one thing which confuses me is when a standby, with causal_reads
enabled, has just finished starting up. I can't connect to it
because:

FATAL: standby is not available for causal reads

However, this is the same message when I'm successfully connected, but
it's lagging, and the primary is still waiting for the standby to
catch up:

ERROR: standby is not available for causal reads

What is the difference here? The problem being reported appears to be
identical, but in the first case I can't connect, but in the second
case I can (although I still can't issue queries).

Thom

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#31Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Thom Brown (#30)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Mon, Feb 22, 2016 at 2:10 AM, Thom Brown <thom@linux.com> wrote:

On 3 February 2016 at 10:46, Thomas Munro <thomas.munro@enterprisedb.com> wrote:

On Wed, Feb 3, 2016 at 10:59 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

There seems to be a copy-pasto there - shouldn't that be:

+ if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)

Indeed, thanks! New patch attached.

I've given this a test drive, and it works exactly as described.

Thanks for trying it out!

But one thing which confuses me is when a standby, with causal_reads
enabled, has just finished starting up. I can't connect to it
because:

FATAL: standby is not available for causal reads

However, this is the same message when I'm successfully connected, but
it's lagging, and the primary is still waiting for the standby to
catch up:

ERROR: standby is not available for causal reads

What is the difference here? The problem being reported appears to be
identical, but in the first case I can't connect, but in the second
case I can (although I still can't issue queries).

Right, you get the error at login before it has managed to connect to
the primary, and for a short time after while it's in 'joining' state,
or potentially longer if there is a backlog of WAL to apply.

The reason is that when causal_reads = on in postgresql.conf (as
opposed to being set for an individual session or role), causal reads
logic is used for snapshots taken during authentication (in fact the
error is generated when trying to take a snapshot slightly before
authentication proper begins, in InitPostgres). I think that's a
desirable feature: if you have causal reads on and you create/alter a
database/role (for example setting a new password) and commit, and
then you immediately try to connect to that database/role on a standby
where you have causal reads enabled system-wide, then you get the
causal reads guarantee during authentication: you either see the
effects of your earlier transaction or you see the error. As you have
discovered, there is a small window after a standby comes up where it
will show the error because it hasn't got a lease yet so it can't let
you log in yet because it could be seeing a stale catalog (your user
may not exist on the standby yet, or have been altered in some way, or
your database may not exist yet, etc).

Does that make sense?

--
Thomas Munro
http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#32Thom Brown
thom@linux.com
In reply to: Thomas Munro (#31)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On 21 February 2016 at 23:18, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Mon, Feb 22, 2016 at 2:10 AM, Thom Brown <thom@linux.com> wrote:

On 3 February 2016 at 10:46, Thomas Munro <thomas.munro@enterprisedb.com> wrote:

On Wed, Feb 3, 2016 at 10:59 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

There seems to be a copy-pasto there - shouldn't that be:

+ if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)

Indeed, thanks! New patch attached.

I've given this a test drive, and it works exactly as described.

Thanks for trying it out!

But one thing which confuses me is when a standby, with causal_reads
enabled, has just finished starting up. I can't connect to it
because:

FATAL: standby is not available for causal reads

However, this is the same message when I'm successfully connected, but
it's lagging, and the primary is still waiting for the standby to
catch up:

ERROR: standby is not available for causal reads

What is the difference here? The problem being reported appears to be
identical, but in the first case I can't connect, but in the second
case I can (although I still can't issue queries).

Right, you get the error at login before it has managed to connect to
the primary, and for a short time after while it's in 'joining' state,
or potentially longer if there is a backlog of WAL to apply.

The reason is that when causal_reads = on in postgresql.conf (as
opposed to being set for an individual session or role), causal reads
logic is used for snapshots taken during authentication (in fact the
error is generated when trying to take a snapshot slightly before
authentication proper begins, in InitPostgres). I think that's a
desirable feature: if you have causal reads on and you create/alter a
database/role (for example setting a new password) and commit, and
then you immediately try to connect to that database/role on a standby
where you have causal reads enabled system-wide, then you get the
causal reads guarantee during authentication: you either see the
effects of your earlier transaction or you see the error. As you have
discovered, there is a small window after a standby comes up where it
will show the error because it hasn't got a lease yet so it can't let
you log in yet because it could be seeing a stale catalog (your user
may not exist on the standby yet, or have been altered in some way, or
your database may not exist yet, etc).

Does that make sense?

Ah, alles klar. Yes, that makes sense now. I've been trying to break
it the past few days, and this was the only thing which I wasn't clear
on. The parameters all work as described

The replay_lag is particularly cool. Didn't think it was possible to
glean this information on the primary, but the timings are correct in
my tests.

+1 for this patch. Looks like this solves the problem that
semi-synchronous replication tries to solve, although arguably in a
more sensible way.

Thom

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#33Michael Paquier
michael.paquier@gmail.com
In reply to: Thom Brown (#32)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Mon, Feb 22, 2016 at 9:39 AM, Thom Brown <thom@linux.com> wrote:

On 21 February 2016 at 23:18, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:
The replay_lag is particularly cool. Didn't think it was possible to
glean this information on the primary, but the timings are correct in
my tests.

+1 for this patch. Looks like this solves the problem that
semi-synchronous replication tries to solve, although arguably in a
more sensible way.

Yeah, having extra logic at application layer to check if a certain
LSN position has been applied or not is doable, but if we can avoid it
that's a clear plus.

This patch has no documentation. I will try to figure out by myself
how the new parameters interact with the rest of the syncrep code
while looking at it but if we want to move on to get something
committable for 9.6 it would be good to get some documentation soon.
--
Michael

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#34Thom Brown
thom@linux.com
In reply to: Michael Paquier (#33)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On 27 February 2016 at 13:20, Michael Paquier <michael.paquier@gmail.com> wrote:

On Mon, Feb 22, 2016 at 9:39 AM, Thom Brown <thom@linux.com> wrote:

On 21 February 2016 at 23:18, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:
The replay_lag is particularly cool. Didn't think it was possible to
glean this information on the primary, but the timings are correct in
my tests.

+1 for this patch. Looks like this solves the problem that
semi-synchronous replication tries to solve, although arguably in a
more sensible way.

Yeah, having extra logic at application layer to check if a certain
LSN position has been applied or not is doable, but if we can avoid it
that's a clear plus.

This patch has no documentation. I will try to figure out by myself
how the new parameters interact with the rest of the syncrep code
while looking at it but if we want to move on to get something
committable for 9.6 it would be good to get some documentation soon.

Could we rename "apply" to "remote_apply"? It seems more consistent
with "remote_write", and matches its own enum entry too.

Thom

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#35Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Michael Paquier (#33)
1 attachment(s)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Sun, Feb 28, 2016 at 2:20 AM, Michael Paquier
<michael.paquier@gmail.com> wrote:

On Mon, Feb 22, 2016 at 9:39 AM, Thom Brown <thom@linux.com> wrote:

On 21 February 2016 at 23:18, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:
The replay_lag is particularly cool. Didn't think it was possible to
glean this information on the primary, but the timings are correct in
my tests.

+1 for this patch. Looks like this solves the problem that
semi-synchronous replication tries to solve, although arguably in a
more sensible way.

Yeah, having extra logic at application layer to check if a certain
LSN position has been applied or not is doable, but if we can avoid it
that's a clear plus.

This patch has no documentation. I will try to figure out by myself
how the new parameters interact with the rest of the syncrep code
while looking at it but if we want to move on to get something
committable for 9.6 it would be good to get some documentation soon.

Thanks for looking at the patch! Here is a new version with the
following changes:

1. Some draft user documentation has been added, as requested.

2. The new synchronous commit level (separate from the causal reads
feature, but implemented for completeness) is now called
"remote_apply" instead of "apply", as suggested by Thom Brown.

3. There is a draft README.causal_reads file which provides some
developer notes about state transitions, leases and clock skew.

4. The 'joining' state management has been improved (it's now based
on xlog positions rather than time; see the README and comments for
details).

5. The ps title is now restored after it is modified during causal
reads-related waiting.

6. I assigned an errcode (40P02) for causal reads failures (useful
for clients/middleware/libraries that might want to handle this error
automatically), as suggested by a couple of people off-list.

--
Thomas Munro
http://www.enterprisedb.com

Attachments:

causal-reads-v7.patchapplication/octet-stream; name=causal-reads-v7.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index a09ceb2..eee6d0b 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -2664,6 +2664,35 @@ include_dir 'conf.d'
      across the cluster without problems if that is required.
     </para>
 
+    <sect2 id="runtime-config-replication-all">
+     <title>All Servers</title>
+     <para>
+      These parameters can be set on the primary or any standby.
+     </para>
+     <variablelist>
+      <varlistentry id="guc-causal-reads" xreflabel="causal_reads">
+       <term><varname>causal_reads</varname> (<type>boolean</type>)
+       <indexterm>
+        <primary><varname>causal_reads</> configuration parameter</primary>
+       </indexterm>
+       </term>
+       <listitem>
+        <para>
+         Enables causal consistency between transactions run on different
+         servers.  A transaction that is run with <varname>causal_reads</> set
+         to <literal>on</> is guaranteed either to see the effects of all
+         completed transactions run on the primary with the setting on, or to
+         receive an error "standby is not available for causal reads".  Note
+         that both transactions involved in a causal dependency (a write on
+         the primary followed by a read on any server which must see the
+         write) must be run with the setting on.
+         See <xref linkend="causal-reads"> for more details.
+        </para>
+       </listitem>
+      </varlistentry>
+     </variablelist>     
+    </sect2>
+
     <sect2 id="runtime-config-replication-sender">
      <title>Sending Server(s)</title>
 
@@ -2895,6 +2924,48 @@ include_dir 'conf.d'
       </listitem>
      </varlistentry>
 
+     <varlistentry>
+      <term><varname>causal_reads_timeout</varname> (<type>integer</type>)
+       <indexterm>
+        <primary><varname>causal_reads_timeout</> configuration parameter</primary>
+       </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies the maximum replay lag the primary will tolerate from a
+        standby before dropping it from the set of standbys available for
+        causal reads.
+       </para>
+       <para>
+        This setting is also used to control the <firstterm>leases</> used to
+        maintain the causal reads guarantee.  It must be set to a value which
+        is at least 4 times the maximum possible difference in system clocks
+        between the primary and standby servers, as described
+        in <xref linkend="causal-reads">.
+       </para>
+      </listitem>
+     </varlistentry>
+
+     <varlistentry id="guc-causal-reads-standby-names" xreflabel="causal-reads-standby-names">
+      <term><varname>causal_reads_standby_names</varname> (<type>string</type>)
+      <indexterm>
+       <primary><varname>causal_reads_standby_names</> configuration parameter</primary>
+      </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies a comma-separated list of standby names that can support
+        <firstterm>causal reads</>, as described in
+        <xref linkend="causal-reads">.  Follows the same convention
+        as <link linkend="guc-synchronous-standby-names"><literal>synchronous_standby_name</></>.
+        The default is <literal>*</>, matching all standbys.
+       </para>
+       <para>
+        This setting has no effect if <varname>causal_reads_timeout</> is not set.
+       </para>
+      </listitem>
+     </varlistentry>
+
      </variablelist>
     </sect2>
 
diff --git a/doc/src/sgml/high-availability.sgml b/doc/src/sgml/high-availability.sgml
index 6cb690c..a338fd8 100644
--- a/doc/src/sgml/high-availability.sgml
+++ b/doc/src/sgml/high-availability.sgml
@@ -1081,6 +1081,9 @@ primary_slot_name = 'node_a_slot'
     WAL record is then sent to the standby. The standby sends reply
     messages each time a new batch of WAL data is written to disk, unless
     <varname>wal_receiver_status_interval</> is set to zero on the standby.
+    In the case that <varname>synchronous_commit</> is set to
+    <literal>remote_apply</>, the standby sends reply messages when the commit
+    record is replayed, making the transaction visible.
     If the standby is the first matching standby, as specified in
     <varname>synchronous_standby_names</> on the primary, the reply
     messages from that standby will be used to wake users waiting for
@@ -1107,6 +1110,16 @@ primary_slot_name = 'node_a_slot'
    </para>
 
    <para>
+    Setting <varname>synchronous_commit</> to <literal>remote_apply</> will
+    cause each commit to wait until the current synchronous standby reports
+    that it has replayed the transaction, making it visible to user queries.
+    In simple cases, this allows for load balancing with causal consistency
+    on a single hot standby.  (See also
+    <xref linkend="causal-reads"> which deals with multiple standbys and
+    standby failure.)
+   </para>
+
+   <para>
     Users will stop waiting if a fast shutdown is requested.  However, as
     when using asynchronous replication, the server will not fully
     shutdown until all outstanding WAL records are transferred to the currently
@@ -1160,8 +1173,9 @@ primary_slot_name = 'node_a_slot'
     <title>Planning for High Availability</title>
 
    <para>
-    Commits made when <varname>synchronous_commit</> is set to <literal>on</>
-    or <literal>remote_write</> will wait until the synchronous standby responds. The response
+    Commits made when <varname>synchronous_commit</> is set to <literal>on</>,
+    <literal>remote_write</> or <literal>remote_apply</> will wait until the
+    synchronous standby responds. The response
     may never occur if the last, or only, standby should crash.
    </para>
 
@@ -1221,6 +1235,119 @@ primary_slot_name = 'node_a_slot'
    </sect3>
   </sect2>
 
+  <sect2 id="causal-reads">
+   <title>Causal reads</title>
+   <indexterm>
+    <primary>causal reads</primary>
+    <secondary>in standby</secondary>
+   </indexterm>
+
+   <para>
+    The causal reads feature allows read-only queries to run on hot standby
+    servers without exposing stale data to the client, providing a form of
+    causal consistency.  Transactions can run on any standby with the
+    following guarantee about the visibility of preceding transactions: If you
+    set <varname>causal_reads</> to <literal>on</> in any pair of consecutive
+    transactions tx1, tx2 where tx2 begins after tx1 successfully returns,
+    then tx2 will either see tx1 or fail with a new error "standby is not
+    available for causal reads", no matter which server it runs on.  Although
+    the guarantee is expressed in terms of two individual transactions, the
+    GUC can also be set at session, role or system level to make the guarantee
+    generally, allowing for load balancing of applications that were not
+    designed with load balancing in mind.
+   </para>
+
+   <para>
+    In order to enable the feature, <varname>causal_reads_timeout</> must be
+    set to a non-zero value on the primary server.  The
+    GUC <varname>causal_reads_standby_names</> can be used to limit the set of
+    standbys that can join the dynamic set of causal reads standbys by
+    providing a comma-separated list of application names.  By default, all
+    standbys are candidates, if the feature is enabled.
+   </para>
+
+   <para>
+    The current set of servers that the primary considers to be available for
+    causal reads can be seen in
+    the <link linkend="monitoring-stats-views-table"> <literal>pg_stat_replication</></>
+    view.  Administrators, applications and load balancing middleware can use
+    this view to discover standbys that can currently handle causal reads
+    transactions without raising the error.  Since that information is only an
+    instantantaneous snapshot, clients should still be prepared for the error
+    to be raised at any time, and consider redirecting transactions to another
+    standby.
+   </para>
+
+   <para>
+    The advantages of the causal reads feature over simply
+    setting <varname>synchronous_commit</> to <literal>remote_apply</> are:
+    <orderedlist>
+      <listitem>
+       <para>
+        It allows the primary to wait for multiple standbys to replay
+        transactions.
+       </para>
+      </listitem>
+      <listitem>
+       <para>
+        It places a configurable limit on how much replay lag (and therefore
+        delay at commit time) the primary tolerates from standbys before it
+        drops them from the dynamic set of standbys it waits for.
+       </para>   
+      </listitem>
+      <listitem>
+       <para>
+        It upholds the causal reads guarantee during the transitions that
+        occur when new standbys are added or removed from the set of standbys,
+        including scenarios where contact has been lost between the primary
+        and standbys but the standby is still alive and running client
+        queries.
+       </para>
+      </listitem>
+    </orderedlist>
+   </para>
+
+   <para>
+    The protocol used to uphold the guarantee even in the case of network
+    failure depends on the system clocks of the primary and standby servers
+    being synchronized, with an allowance for a difference up to one quarter
+    of <varname>causal_reads_timeout</>.  For example,
+    if <varname>causal_reads_timeout</> is set to <literal>4s</>, then the
+    clocks must not be further than 1 second apart for the guarantee to be
+    upheld reliably during transitions.  The ubiquity of the Network Time
+    Protocol (NTP) on modern operating systems and availability of high
+    quality time servers makes it possible to choose a tolerance significantly
+    higher than the maximum expected clock difference.  An effort is
+    nevertheless made to detect and report misconfigured and faulty systems
+    with clock differences greater than the configured tolerance.
+   </para>
+
+   <note>
+    <para>
+     Current hardware clocks, NTP implementations and public time servers are
+     unlikely to allow the system clocks to differ more than tens or hundreds
+     of milliseconds, and systems synchronized with dedicated local time
+     servers may be considerably more accurate, but you should only consider
+     setting <varname>causal_reads_timeout</> below 4 seconds (allowing up to
+     1 second of clock difference) after researching your time synchronization
+     infrastructure thoroughly.
+    </para>  
+   </note>
+
+   <note>
+    <para>
+      While similar to synchronous replication in the sense that both involve
+      the primary server waiting for responses from standby servers, the
+      causal reads feature is not concerned with avoiding data loss.  A
+      primary configured for causal reads will drop all standbys that stop
+      responding or replay too slowly from the dynamic set that it waits for,
+      so you should consider configuring both synchronous replication and
+      causal reads if you need data loss avoidance guarantees and causal
+      consistency guarantees for load balancing.
+    </para>
+   </note>
+  </sect2>
+
   <sect2 id="continuous-archiving-in-standby">
    <title>Continuous archiving in standby</title>
 
@@ -1569,7 +1696,16 @@ if (!triggered)
     so there will be a measurable delay between primary and standby. Running the
     same query nearly simultaneously on both primary and standby might therefore
     return differing results. We say that data on the standby is
-    <firstterm>eventually consistent</firstterm> with the primary.  Once the
+    <firstterm>eventually consistent</firstterm> with the primary by default.
+    The data visible to a transaction running on a standby can be
+    made <firstterm>causally consistent</> with respect to a transaction that
+    has completed on the primary by setting <varname>causal_reads</>
+    to <literal>on</> in both transactions.  For more details,
+    see <xref linkend="causal-reads">.
+   </para>
+
+   <para>
+    Once the
     commit record for a transaction is replayed on the standby, the changes
     made by that transaction will be visible to any new snapshots taken on
     the standby.  Snapshots may be taken at the start of each query or at the
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 85459d0..5a87f37 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -820,6 +820,12 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       standby server</entry>
     </row>
     <row>
+     <entry><structfield>replay_lag</></entry>
+     <entry><type>interval</></entry>
+     <entry>Estimated time taken for recent WAL records to be replayed on this
+      standby server</entry>
+    </row>
+    <row>
      <entry><structfield>sync_priority</></entry>
      <entry><type>integer</></entry>
      <entry>Priority of this standby server for being chosen as the
@@ -830,6 +836,17 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
      <entry><type>text</></entry>
      <entry>Synchronous state of this standby server</entry>
     </row>
+    <row>
+     <entry><structfield>causal_reads_state</></entry>
+     <entry><type>text</></entry>
+     <entry>Causal reads state of this standby server.  This field will be
+     non-null only if <varname>cause_reads_timeout</> is set.  If a standby is
+     in <literal>available</> state, then it can currently serve causal reads
+     queries.  If it is not replaying fast enough or not responding to
+     keepalive messages, it will be in <literal>unavailable</> state, and if
+     it is currently transitioning to availability it will be
+     in <literal>joining</> state for a short time.</entry>
+    </row>
    </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index b0d5440..8bfc510 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,10 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
+	{
+		CausalReadsWaitForLSN(XactLastRecEnd);
 		SyncRepWaitForLSN(XactLastRecEnd);
+	}
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5117,6 +5120,13 @@ XactLogCommitRecord(TimestampTz commit_time,
 		xl_xinfo.xinfo |= XACT_COMPLETION_FORCE_SYNC_COMMIT;
 
 	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this commit is applied.
+	 */
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY || causal_reads)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
+	/*
 	 * Relcache invalidations requires information about the current database
 	 * and so does logical decoding.
 	 */
@@ -5452,6 +5462,19 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 	if (XactCompletionForceSyncCommit(parsed->xinfo))
 		XLogFlush(lsn);
 
+	/*
+	 * Record the primary's timestamp for the commit record, so it can be used
+	 * for tracking replay lag.
+	 */
+	SetXLogReplayTimestamp(parsed->xact_time);
+
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit or causal reads), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 /*
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 94b79ac..b7348ab 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -81,6 +81,8 @@ extern uint32 bootstrap_data_checksum_version;
 #define PROMOTE_SIGNAL_FILE		"promote"
 #define FALLBACK_PROMOTE_SIGNAL_FILE "fallback_promote"
 
+/* Size of the circular buffer of timestamped LSNs. */
+#define MAX_TIMESTAMPED_LSNS 8192
 
 /* User-settable parameters */
 int			max_wal_size = 64;	/* 1 GB */
@@ -346,6 +348,12 @@ static XLogRecPtr RedoRecPtr;
 static bool doPageWrites;
 
 /*
+ * doRequestWalReceiverReply is used by recovery code to ask the main recovery
+ * loop to trigger a walreceiver reply.
+ */
+static bool doRequestWalReceiverReply;
+
+/*
  * RedoStartLSN points to the checkpoint's REDO location which is specified
  * in a backup label file, backup history file or control file. In standby
  * mode, XLOG streaming usually starts from the position where an invalid
@@ -357,6 +365,13 @@ static bool doPageWrites;
  */
 static XLogRecPtr RedoStartLSN = InvalidXLogRecPtr;
 
+/*
+ * LastReplayedTimestamp can be set by redo handlers when they apply a record
+ * that carries a timestamp, by calling SetXLogReplayedTimestamp.  The xlog
+ * apply loop can then update the value in shared memory.
+ */
+static TimestampTz LastReplayedTimestamp = 0;
+
 /*----------
  * Shared-memory data structures for XLOG control
  *
@@ -631,6 +646,21 @@ typedef struct XLogCtlData
 	/* current effective recovery target timeline */
 	TimeLineID	RecoveryTargetTLI;
 
+	/* timestamp from the most recently applied record that carried a timestamp. */
+	TimestampTz lastReplayedTimestamp;
+
+	/*
+	 * We maintain a circular buffer of LSNs and associated timestamps.
+	 * Walreceiver writes into it using information from timestamps, and the
+	 * startup recovery process reads from it and notifies walreceiver when
+	 * LSNs are replayed so that the timestamps can be fed back to the
+	 * upstream server, to track lag.
+	 */
+	Index		timestampedLsnRead;
+	Index		timestampedLsnWrite;
+	XLogRecPtr	timestampedLsn[MAX_TIMESTAMPED_LSNS];
+	TimestampTz	timestampedLsnTime[MAX_TIMESTAMPED_LSNS];
+
 	/*
 	 * timestamp of when we started replaying the current chunk of WAL data,
 	 * only relevant for replication or archive recovery
@@ -6897,14 +6927,58 @@ StartupXLOG(void)
 				error_context_stack = errcallback.previous;
 
 				/*
-				 * Update lastReplayedEndRecPtr after this record has been
-				 * successfully replayed.
+				 * Update lastReplayedEndRecPtr and lastReplayedTimestamp
+				 * after this record has been successfully replayed.
 				 */
 				SpinLockAcquire(&XLogCtl->info_lck);
 				XLogCtl->lastReplayedEndRecPtr = EndRecPtr;
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
+				if (LastReplayedTimestamp != 0)
+				{
+					/* If replaying a record produced a timestamp, use that. */
+					XLogCtl->lastReplayedTimestamp = LastReplayedTimestamp;
+					LastReplayedTimestamp = 0;
+				}
+				else
+				{
+					/*
+					 * If we have applied LSNs associated with timestamps
+					 * received by walreceiver, then use the recorded
+					 * timestamp.  We consume from the read end of the
+					 * circular buffer.
+					 */
+					while (XLogCtl->timestampedLsnRead !=
+						   XLogCtl->timestampedLsnWrite &&
+						   XLogCtl->timestampedLsn[XLogCtl->timestampedLsnRead]
+						   <= EndRecPtr)
+					{
+						if (XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead] >
+							XLogCtl->lastReplayedTimestamp)
+						{
+							XLogCtl->lastReplayedTimestamp =
+								XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead];
+							doRequestWalReceiverReply = true;
+						}
+						XLogCtl->timestampedLsnRead =
+							(XLogCtl->timestampedLsnRead + 1) % MAX_TIMESTAMPED_LSNS;
+					}
+				}
 				SpinLockRelease(&XLogCtl->info_lck);
 
+				/*
+				 * If rm_redo reported that it applied a commit record that
+				 * the master is waiting for by calling
+				 * XLogRequestWalReceiverReply, or we encountered a WAL
+				 * location that was associated with a timestamp above, then
+				 * we wake up the receiver so that it notices the updated
+				 * lastReplayedEndRecPtr and sends a reply to the master.
+				 */
+				if (doRequestWalReceiverReply)
+				{
+					doRequestWalReceiverReply = false;
+					WalRcvWakeup();
+				}
+
 				/* Remember this record as the last-applied one */
 				LastRec = ReadRecPtr;
 
@@ -11626,3 +11700,103 @@ SetWalWriterSleeping(bool sleeping)
 	XLogCtl->WalWriterSleeping = sleeping;
 	SpinLockRelease(&XLogCtl->info_lck);
 }
+
+/*
+ * Called by redo code to indicate that the xlog replay loop should wake up
+ * the walreceiver process so that a reply can be sent to the primary.
+ */
+void
+XLogRequestWalReceiverReply(void)
+{
+	doRequestWalReceiverReply = true;
+}
+
+/*
+ * Merge timestamps from keepalive messages with the timestamps from WAL
+ * records, so that we can track lag while idle or while replaying large
+ * amounts of WAL without commit records.  In the former case there is no lag,
+ * and in the latter case we will remember a timestamp that goes with an
+ * arbitrary LSN, and wait for that LSN to be replayed before using the
+ * timestamp.
+ *
+ * This is called by walreceiver on standby servers when keepalive messages
+ * arrive.
+ */
+void
+SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn)
+{
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn == XLogCtl->lastReplayedEndRecPtr)
+	{
+		/*
+		 * That is the last replayed LSN: we are fully replayed, so we can
+		 * update the replay timestamp immediately.
+		 */
+		XLogCtl->lastReplayedTimestamp = timestamp;
+	}
+	else
+	{
+		/*
+		 * There is WAL still to be applied.  We will associate the timestamp
+		 * with this WAL position and wait for it to be replayed.  We add it
+		 * at the 'write' end of the circular buffer of LSN/timestamp
+		 * mappings, which the replay loop will eventually read.
+		 */
+		Index w = XLogCtl->timestampedLsnWrite;
+		Index r = XLogCtl->timestampedLsnRead;
+
+		XLogCtl->timestampedLsn[w] = lsn;
+		XLogCtl->timestampedLsnTime[w] = timestamp;
+
+		/* Advance the write point. */
+		w = (w + 1) % MAX_TIMESTAMPED_LSNS;
+		XLogCtl->timestampedLsnWrite = w;
+		if (w == r)
+		{
+			/*
+			 * The buffer is full.  Advance the read point (throwing away
+			 * oldest values; we will begin to overestimate replay lag, until
+			 * lag decreases to a size our buffer can manage, or the next
+			 * commit record is replayed).
+			 */
+			r = (r + 1) % MAX_TIMESTAMPED_LSNS;
+			XLogCtl->timestampedLsnRead = r;
+		}
+	}
+	SpinLockRelease(&XLogCtl->info_lck);
+}
+
+/*
+ * Set the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary.  This can be called by redo handlers that have
+ * an appropriate timestamp (currently only commit records).  Updating the
+ * shared memory value is deferred until after the redo handler returns.
+ */
+void
+SetXLogReplayTimestamp(TimestampTz timestamp)
+{
+	LastReplayedTimestamp = timestamp;
+}
+
+/*
+ * Get the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the master, and also the most recently applied LSN.  (Note
+ * that the timestamp and the LSN don't necessarily relate to the same
+ * record.)
+ *
+ * This is similar to GetLatestXTime, except that it is not only advanced by
+ * commit records (see SetXLogReplayTimestampAtLsn).
+ */
+TimestampTz
+GetXLogReplayTimestamp(XLogRecPtr *lsn)
+{
+	TimestampTz result;
+
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn)
+		*lsn = XLogCtl->lastReplayedEndRecPtr;
+	result = XLogCtl->lastReplayedTimestamp;
+	SpinLockRelease(&XLogCtl->info_lck);
+
+	return result;
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index abf9a70..b80206e 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -661,8 +661,10 @@ CREATE VIEW pg_stat_replication AS
             W.write_location,
             W.flush_location,
             W.replay_location,
+            W.replay_lag,
             W.sync_priority,
-            W.sync_state
+            W.sync_state,
+            W.causal_reads_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
             pg_stat_get_wal_senders() AS W
     WHERE S.usesysid = U.oid AND
diff --git a/src/backend/replication/README.causal-reads b/src/backend/replication/README.causal-reads
new file mode 100644
index 0000000..b85d695
--- /dev/null
+++ b/src/backend/replication/README.causal-reads
@@ -0,0 +1,190 @@
+The causal reads guarantee says: If you run any two consecutive
+transactions tx1, tx2 where tx1 completes before tx2 begins, with
+causal_reads set to "on" in both transactions, tx2 will see tx1 or
+raise an error to complain that it can't guarantee causal consistency,
+no matter which servers (primary or any standby) you run each
+transaction on.
+
+When both transactions run on the primary, the guarantee is trivial.
+
+To deal with read-only physical streaming standbys, the primary keeps
+track of a set of standbys that it considers to be currently
+"available" for causal reads, and sends a stream of "leases" to those
+standbys granting them the right to handle causal reads transactions
+for a short time without any further communication with the primary.
+
+In general, the primary provides the guarantee by waiting for all of
+the "available" standbys to report that they have applied a
+transaction.  However, the set of available standbys is dynamic, and
+things get more complicated during state transitions.  There are two
+types of transitions to consider:
+
+1.  unavailable->joining->available
+
+Standbys start out as "unavailable".  If a standby is unavailable and
+is applying fast enough and matches causal_reads_standby_names, the
+primary transitions it to "available", but first it sets it to
+"joining" until it is sure that any transaction committed while it was
+unavailable has definitely been applied on the standby.  This closes a
+race that would otherwise exist if we moved directly to available
+state: tx1 might not wait for a given standby because it's
+unavailable, then a lease might be granted, and then tx2 might run a
+causal reads transaction without error but see stale data.  The
+joining state acts as an airlock: while in joining state, the primary
+waits for that standby to replay causal reads transactions in
+anticipation of the move to available, but it doesn't progress to
+available state and grant a lease to the standby until everything
+preceding joining state has also been applied.
+
+2.  available->unavailable
+
+If a standby is not applying fast enough or not responding to
+keepalive messages, then the primary kicks that standby out of the
+dynamic set of available standbys, that is, marks it as "unavailable".
+In order to make sure that the standby has started rejecting causal
+reads transactions, it needs to revoke the lease it most recently
+granted.  It does that by waiting for the lease to expire before
+allowing any causal reads commits to return.  (In future there could
+be a fast-path recovation message which waits for a serial-numbered
+acknowledgement to reduce waiting in the case where the standby is
+lagging but still reachable and responding).
+
+The rest of this document illustrates how clock skew affects the
+available->unavailable transition.
+
+The following 4 variables are derived from a single GUC, and these
+values will be used in the following illustrations:
+
+causal_reads_timeout = 4s
+lease_time           = 4s (= causal_reads_timeout)
+keepalive_time       = 2s (= lease_time / 2)
+max_clock_skew       = 1s (= lease_time / 4)
+
+Every keepalive_time, the primary transmits a lease that expires at
+local_clock_time + lease_time - max_clock_skew, shown in the following
+diagram as 't' for transmission time and '|' for expiry time.  If
+contact is lost with a standby, the primary will wait until sent_time
++ lease_time for the most recently granted lease to expire, shown on
+the following diagram 'x', to be sure that the standby's clock has
+reached the expiry time even if its clock differs by up to
+max_clock_skew.  In other words, the primary tells the standby that
+the expiry time is at one time, but it trusts that the standby will
+surely agree if it gives it some extra time.  The extra time is
+max_clock_skew.  If the clocks differ by more than max_clock_skew, all
+bets are off (but see below for attempt to detect obvious cases).
+
+0     1     2     3     4     5     6     7     8     9
+t-----------------|-----x
+            t-----------------|-----x
+                        t-----------------|-----x
+                                    t-----------------|...
+                                                t------...
+
+A standby whose clock is 2 seconds ahead of the primary's clock
+perceives gaps in the stream of leases, and will reject causal_reads
+transactions in those intervals.  The causal reads guarantee is
+upheld, but spurious errors are raised between leases, as a
+consequence of the clock skew being greater than max_clock_skew.  In
+the following diagram 'r' shows reception time, and the timeline along
+the top shows the standby's local clock time.
+
+2     3     4     5     6     7     8     9    10    11
+r-----|
+            r-----|
+                        r-----|
+                                    r-----|
+                                                r-----|
+
+If there were no network latency, a standby whose clock is exactly 1
+second ahead of the primary's clock would perceive the stream of
+leases as being replaced just in time, so there is no gap.  Since in
+reality the time of receipt is some time after the time of
+transmission due to network latency, if the standby's clock is exactly
+1 second behind, then there will be small network-latency-sized gaps
+before the next lease arrives, but still no correctness problem with
+respect to the causal reads guarantee.
+
+1     2     3     4     5     6     7     8     9    10
+r-----------|
+            r-----------|
+                        r-----------|
+                                    r-----------|
+                                                r------...
+
+A standby whose clock is perfectly in sync with the primary's
+perceives the stream of leases overlapping (this matches the primary's
+perception of the leases it sent):
+
+0     1     2     3     4     5     6     7     8     9
+r-----------------|
+            r-----------------|
+                        r-----------------|
+                                    r-----------------|
+                                                r------...
+
+A standby whose clock is exactly 1 second behind the primary's
+perceives the stream of leases as overlapping even more, but the time
+of expiry as judged by the standby is no later than the time the
+primary will wait for if required ('x').  That is, if contact is lost
+with the standby, the primary can still reliably hold up causal reads
+commits until the standby has started raising the error in
+causal_reads transactions.
+
+-1    0     1     2     3     4     5     6     7     8
+r-----------------------|
+            r-----------------------|
+                        r-----------------------|
+                                    r------------------...
+                                                r------...
+
+
+A standby whose clock is 2 seconds behind the primary's would perceive
+the stream of leases overlapping even more, and the primary would no
+longer be able to wait for a lease to expire if it wanted to revoke
+it.  But because the expiry time is after local_clock_time +
+lease_time, the standby can immediately see that its own clock must be
+more than 1 second behind the primary's, so it ignores the lease and
+logs a clock skew warning.  In the following diagram a lease expiry
+time that is obviously generated by a primary with a clock set too far
+in the future compared to the local clock is shown with a '!'.
+
+-2    -1    0     1     2     3     4     5     6     7
+r-----------------------------!
+            r-----------------------------!
+                        r-----------------------------!
+                                    r------------------...
+                                                r------...
+
+A danger window exists when the standby's clock is more than
+max_clock_skew behind the primary's clock, but not more than
+max_clock_skew + network latency time behind.  If the clock difference
+is in that range, then the algorithm presented above which is based on
+time of receipt cannot detect that the local clock is too far behind.
+The consequence of this problem could be as follows:
+
+1.  The standby loses contact with the primary due to a network fault.
+
+2.  The primary decides to drop the standby from the set of available
+    causal reads standbys due to lack of keepalive responses or
+    excessive lag, which necessitates holding up commits of causal
+    reads transactions until the most recently sent lease expires, in
+    the belief that the standby will definitely have started raising
+    the 'causal reads unavailable' error in causal reads transactions
+    by that time, if it is still alive and servicing requests.
+
+3.  The standby still has clients connected and running queries.
+
+4.  Due to clock skew in the problematic range, in the standby's
+    opinion the lease lasts slightly longer than the primary waits.
+
+5.  For a short window at most the duration of the network latency
+    time, clients running causal reads transactions are allowed to see
+    potentially stale data.
+
+For this reason we say that the causal reads guarantee only holds as
+long as the absolute difference between the system clocks of the
+machines is no more than max_clock_skew.  The theory is that NTP makes
+it possible to reason about the maximum possible clock difference
+between machines and choose a value that is safe even with a much
+larger difference.  However, we do make a best effort attempt to
+detect misconfigured systems as described above.
\ No newline at end of file
diff --git a/src/backend/replication/README.causal_reads b/src/backend/replication/README.causal_reads
new file mode 100644
index 0000000..35b29e9
--- /dev/null
+++ b/src/backend/replication/README.causal_reads
@@ -0,0 +1,193 @@
+The causal reads guarantee says: If you run any two consecutive
+transactions tx1, tx2 where tx1 completes before tx2 begins, with
+causal_reads set to "on" in both transactions, tx2 will see tx1 or
+raise an error to complain that it can't guarantee causal consistency,
+no matter which servers (primary or any standby) you run each
+transaction on.
+
+When both transactions run on the primary, the guarantee is trivially
+upheld.
+
+To deal with read-only physical streaming standbys, the primary keeps
+track of a set of standbys that it considers to be currently
+"available" for causal reads, and sends a stream of "leases" to those
+standbys granting them the right to handle causal reads transactions
+for a short time without any further communication with the primary.
+
+In general, the primary provides the guarantee by waiting for all of
+the "available" standbys to report that they have applied a
+transaction.  However, the set of available standbys is dynamic, and
+things get more complicated during state transitions.  There are two
+types of transitions to consider:
+
+1.  unavailable->joining->available
+
+Standbys start out as "unavailable".  If a standby is unavailable and
+is applying fast enough and matches causal_reads_standby_names, the
+primary transitions it to "available", but first it sets it to
+"joining" until it is sure that any transaction committed while it was
+unavailable has definitely been applied on the standby.  This closes a
+race that would otherwise exist if we moved directly to available
+state: tx1 might not wait for a given standby because it's
+unavailable, then a lease might be granted, and then tx2 might run a
+causal reads transaction without error but see stale data.  The
+joining state acts as an airlock: while in joining state, the primary
+waits for that standby to replay causal reads transactions in
+anticipation of the move to available, but it doesn't progress to
+available state and grant a lease to the standby until everything
+preceding joining state has also been applied.
+
+2.  available->unavailable
+
+If a standby is not applying fast enough or not responding to
+keepalive messages, then the primary kicks that standby out of the
+dynamic set of available standbys, that is, marks it as "unavailable".
+In order to make sure that the standby has started rejecting causal
+reads transactions, it needs to revoke the lease it most recently
+granted.  It does that by waiting for the lease to expire before
+allowing any causal reads commits to return.  (In future there could
+be a fast-path revocation message which waits for a serial-numbered
+acknowledgement to reduce waiting in the case where the standby is
+lagging but still reachable and responding).
+
+The rest of this document illustrates how clock skew affects the
+available->unavailable transition.
+
+The following 4 variables are derived from a single GUC, and these
+values will be used in the following illustrations:
+
+causal_reads_timeout = 4s
+lease_time           = 4s (= causal_reads_timeout)
+keepalive_time       = 2s (= lease_time / 2)
+max_clock_skew       = 1s (= lease_time / 4)
+
+Every keepalive_time, the primary transmits a lease that expires at
+local_clock_time + lease_time - max_clock_skew, shown in the following
+diagram as 't' for transmission time and '|' for expiry time.  If
+contact is lost with a standby, the primary will wait until sent_time
++ lease_time for the most recently granted lease to expire, shown on
+the following diagram 'x', to be sure that the standby's clock has
+reached the expiry time even if its clock differs by up to
+max_clock_skew.  In other words, the primary tells the standby that
+the expiry time is at one time, but it trusts that the standby will
+surely agree if it gives it some extra time.  The extra time is
+max_clock_skew.  If the clocks differ by more than max_clock_skew, all
+bets are off (but see below for attempt to detect obvious cases).
+
+0     1     2     3     4     5     6     7     8     9
+t-----------------|-----x
+            t-----------------|-----x
+                        t-----------------|-----x
+                                    t-----------------|...
+                                                t------...
+
+A standby whose clock is 2 seconds ahead of the primary's clock
+perceives gaps in the stream of leases, and will reject causal_reads
+transactions in those intervals.  The causal reads guarantee is
+upheld, but spurious errors are raised between leases, as a
+consequence of the clock skew being greater than max_clock_skew.  In
+the following diagram 'r' shows reception time, and the timeline along
+the top shows the standby's local clock time.
+
+2     3     4     5     6     7     8     9    10    11
+r-----|
+            r-----|
+                        r-----|
+                                    r-----|
+                                                r-----|
+
+If there were no network latency, a standby whose clock is exactly 1
+second ahead of the primary's clock would perceive the stream of
+leases as being replaced just in time, so there is no gap.  Since in
+reality the time of receipt is some time after the time of
+transmission due to network latency, if the standby's clock is exactly
+1 second behind, then there will be small network-latency-sized gaps
+before the next lease arrives, but still no correctness problem with
+respect to the causal reads guarantee.
+
+1     2     3     4     5     6     7     8     9    10
+r-----------|
+            r-----------|
+                        r-----------|
+                                    r-----------|
+                                                r------...
+
+A standby whose clock is perfectly in sync with the primary's
+perceives the stream of leases overlapping (this matches the primary's
+perception of the leases it sent):
+
+0     1     2     3     4     5     6     7     8     9
+r-----------------|
+            r-----------------|
+                        r-----------------|
+                                    r-----------------|
+                                                r------...
+
+A standby whose clock is exactly 1 second behind the primary's
+perceives the stream of leases as overlapping even more, but the time
+of expiry as judged by the standby is no later than the time the
+primary will wait for if required ('x').  That is, if contact is lost
+with the standby, the primary can still reliably hold up causal reads
+commits until the standby has started raising the error in
+causal_reads transactions.
+
+-1    0     1     2     3     4     5     6     7     8
+r-----------------------|
+            r-----------------------|
+                        r-----------------------|
+                                    r------------------...
+                                                r------...
+
+
+A standby whose clock is 2 seconds behind the primary's would perceive
+the stream of leases overlapping even more, and the primary would no
+longer be able to wait for a lease to expire if it wanted to revoke
+it.  But because the expiry time is after local_clock_time +
+lease_time, the standby can immediately see that its own clock must be
+more than 1 second behind the primary's, so it ignores the lease and
+logs a clock skew warning.  In the following diagram a lease expiry
+time that is obviously generated by a primary with a clock set too far
+in the future compared to the local clock is shown with a '!'.
+
+-2    -1    0     1     2     3     4     5     6     7
+r-----------------------------!
+            r-----------------------------!
+                        r-----------------------------!
+                                    r------------------...
+                                                r------...
+
+A danger window exists when the standby's clock is more than
+max_clock_skew behind the primary's clock, but not more than
+max_clock_skew + network latency time behind.  If the clock difference
+is in that range, then the algorithm presented above which is based on
+time of receipt cannot detect that the local clock is too far behind.
+The consequence of this problem could be as follows:
+
+1.  The standby loses contact with the primary due to a network fault.
+
+2.  The primary decides to drop the standby from the set of available
+    causal reads standbys due to lack of keepalive responses or
+    excessive lag, which necessitates holding up commits of causal
+    reads transactions until the most recently sent lease expires, in
+    the belief that the standby will definitely have started raising
+    the 'causal reads unavailable' error in causal reads transactions
+    by that time, if it is still alive and servicing requests.
+
+3.  The standby still has clients connected and running queries.
+
+4.  Due to clock skew in the problematic range, in the standby's
+    opinion the lease lasts slightly longer than the primary waits.
+
+5.  For a short window at most the duration of the network latency
+    time, clients running causal reads transactions are allowed to see
+    potentially stale data.
+
+For this reason we say that the causal reads guarantee only holds as
+long as the absolute difference between the system clocks of the
+machines is no more than max_clock_skew.  The theory is that NTP makes
+it possible to reason about the maximum possible clock difference
+between machines and choose a value that allows for a much larger
+difference.  However, we do make a best effort attempt to detect
+misconfigured systems as described above, to catch the case of servers
+not running ntp a correctly configured ntp daemon, or with a clock so
+far out of whack that ntp refuses to fix it.
\ No newline at end of file
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 7f85b88..e0ea7b7 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -57,6 +57,11 @@
 #include "utils/builtins.h"
 #include "utils/ps_status.h"
 
+/* GUC variables */
+int causal_reads_timeout;
+bool causal_reads;
+char *causal_reads_standby_names;
+
 /* User-settable parameters for sync rep */
 char	   *SyncRepStandbyNames;
 
@@ -69,7 +74,7 @@ static int	SyncRepWaitMode = SYNC_REP_NO_WAIT;
 
 static void SyncRepQueueInsert(int mode);
 static void SyncRepCancelWait(void);
-static int	SyncRepWakeQueue(bool all, int mode);
+static int	SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn);
 
 static int	SyncRepGetStandbyPriority(void);
 
@@ -83,6 +88,255 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * ===========================================================
  */
 
+static bool
+SyncRepCheckEarlyExit(void)
+{
+	/*
+	 * If a wait for synchronous replication is pending, we can neither
+	 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
+	 * lead the client to believe that the transaction aborted, which
+	 * is not true: it's already committed locally. The former is no good
+	 * either: the client has requested synchronous replication, and is
+	 * entitled to assume that an acknowledged commit is also replicated,
+	 * which might not be true. So in this case we issue a WARNING (which
+	 * some clients may be able to interpret) and shut off further output.
+	 * We do NOT reset ProcDiePending, so that the process will die after
+	 * the commit is cleaned up.
+	 */
+	if (ProcDiePending)
+	{
+		ereport(WARNING,
+				(errcode(ERRCODE_ADMIN_SHUTDOWN),
+				 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * It's unclear what to do if a query cancel interrupt arrives.  We
+	 * can't actually abort at this point, but ignoring the interrupt
+	 * altogether is not helpful, so we just terminate the wait with a
+	 * suitable warning.
+	 */
+	if (QueryCancelPending)
+	{
+		QueryCancelPending = false;
+		ereport(WARNING,
+				(errmsg("canceling wait for synchronous replication due to user request"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * If the postmaster dies, we'll probably never get an
+	 * acknowledgement, because all the wal sender processes will exit. So
+	 * just bail out.
+	 */
+	if (!PostmasterIsAlive())
+	{
+		ProcDiePending = true;
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	return false;
+}
+
+/*
+ * Check if we can stop waiting for causal consistency.  We can stop waiting
+ * when the following conditions are met:
+ *
+ * 1.  All walsenders currently in 'joining' or 'available' state have
+ * applied the target LSN.
+ *
+ * 2.  Any stall periods caused by standbys dropping out of 'available' state
+ * have passed, so that we can be sure that their leases have expired and they
+ * have started rejecting causal reads transactions.
+ *
+ * The output parameter 'waitingFor' is set to the number of nodes we are
+ * currently waiting for.  The output parameters 'stallTimeMillis' is set to
+ * the number of milliseconds we need to wait for to observe any current
+ * commit stall.
+ *
+ * Returns true if commit can return control, because every standby has either
+ * applied the LSN or started rejecting causal_reads transactions.
+ */
+static bool
+CausalReadsCommitCanReturn(XLogRecPtr XactCommitLSN,
+						   int *waitingFor,
+						   long *stallTimeMillis)
+{
+	int i;
+	TimestampTz now;
+
+	/* Count how many joining/available nodes we are waiting for. */
+	*waitingFor = 0;
+	for (i = 0; i < max_wal_senders; ++i)
+	{
+		WalSnd *walsnd = &WalSndCtl->walsnds[i];
+
+		/*
+		 * Assuming atomic read of pid_t, we can check walsnd->pid without
+		 * acquiring the spinlock to avoid memory synchronization costs for
+		 * unused walsender slots.  We see a value that existed sometime at
+		 * least as recently as the last memory barrier.
+		 */
+		if (walsnd->pid != 0)
+		{
+			/*
+			 * We need to hold the spinlock to read LSNs, because we can't be
+			 * sure they can be read atomically.
+			 */
+			SpinLockAcquire(&walsnd->mutex);
+			if (walsnd->pid != 0 && walsnd->causal_reads_state >= WALSNDCRSTATE_JOINING)
+			{
+				if (walsnd->apply < XactCommitLSN)
+					++*waitingFor;
+			}
+			SpinLockRelease(&walsnd->mutex);
+		}
+	}
+
+	/* Check if there is a stall in progress that we need to observe. */
+	now = GetCurrentTimestamp();
+	LWLockAcquire(SyncRepLock, LW_SHARED);
+	if (WalSndCtl->stall_causal_reads_until > now)
+	{
+		long seconds;
+		int usecs;
+
+		/* Compute how long we have to wait, rounded up to nearest ms. */
+		TimestampDifference(now, WalSndCtl->stall_causal_reads_until,
+							&seconds, &usecs);
+		*stallTimeMillis = seconds * 1000 + (usecs + 999) / 1000;
+	}
+	else
+		*stallTimeMillis = 0;
+	LWLockRelease(SyncRepLock);
+
+	/* We are done if we are not waiting for any nodes or stalls. */
+	return *waitingFor == 0 && *stallTimeMillis == 0;
+}
+
+/*
+ * Wait for causal consistency in causal_reads mode, if requested by user.
+ */
+void
+CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN)
+{
+	long stallTimeMillis;
+	int waitingFor;
+	char *ps_display_buffer = NULL;
+
+	/* Leave if we aren't in causal_reads mode. */
+	if (!causal_reads)
+		return;
+
+	for (;;)
+	{
+		/* Reset latch before checking state. */
+		ResetLatch(MyLatch);
+
+		/*
+		 * Join the queue to be woken up if any causal reads joining/available
+		 * standby applies XactCommitLSN, if we aren't already in it.  We
+		 * don't actually know if we need to wait for any peers yet, but we
+		 * have to register just in case before checking the walsenders' state
+		 * to avoid a race condition that could occur if we did it after
+		 * calling CausalReadsCommitCanReturn.  (SyncRepWaitForLSN doesn't
+		 * have to do this because it can check the highest-seen LSN in
+		 * walsndctl->lsn[mode] which is protected by SyncRepLock, the same
+		 * lock as the queues.  We can't do that here, because there is no
+		 * single highest-seen LSN that is useful.  We must check
+		 * walsnd->apply for all relevant walsenders.  Therefore we must
+		 * register for notifications first, so that we can be notified via
+		 * our latch of any standby applying the LSN we're interested in after
+		 * we check but before we start waiting, or we could wait forever for
+		 * something that has already happened.)
+		 */
+		LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+		if (MyProc->syncRepState != SYNC_REP_WAITING)
+		{
+			MyProc->waitLSN = XactCommitLSN;
+			MyProc->syncRepState = SYNC_REP_WAITING;
+			SyncRepQueueInsert(SYNC_REP_WAIT_CAUSAL_READS_APPLY);
+			Assert(SyncRepQueueIsOrderedByLSN(SYNC_REP_WAIT_CAUSAL_READS_APPLY));
+		}
+		LWLockRelease(SyncRepLock);
+
+		/* Check if we're done. */
+		if (CausalReadsCommitCanReturn(XactCommitLSN, &waitingFor, &stallTimeMillis))
+		{
+			SyncRepCancelWait();
+			break;
+		}
+
+		Assert(waitingFor > 0 || stallTimeMillis > 0);
+
+		/* If we aren't actually waiting for any standbys, leave the queue. */
+		if (waitingFor == 0)
+			SyncRepCancelWait();
+
+		/* Update the ps title. */
+		if (update_process_title)
+		{
+			char buffer[80];
+
+			/* Remember the old value if this is our first update. */
+			if (ps_display_buffer == NULL)
+			{
+				int len;
+				const char *ps_display = get_ps_display(&len);
+
+				ps_display_buffer = palloc(len + 1);
+				memcpy(ps_display_buffer, ps_display, len);
+				ps_display_buffer[len] = '\0';
+			}
+
+			snprintf(buffer, sizeof(buffer),
+					 "waiting for %d peer(s) to apply %X/%X%s",
+					 waitingFor,
+					 (uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN,
+					 stallTimeMillis > 0 ? " (stalling)" : "");
+			set_ps_display(buffer, false);
+		}
+
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit()) /* Calls SyncRepCancelWait() if true. */
+			break;
+
+		/*
+		 * If are still waiting for peers, then we wait for any joining or
+		 * available peer to reach the LSN (or possibly stop being in one of
+		 * those states or go away).
+		 *
+		 * If not, there must be a non-zero stall time, so we wait for that to
+		 * elapse.
+		 */
+		if (waitingFor > 0)
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH, -1);
+		else
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH | WL_TIMEOUT,
+					  stallTimeMillis);
+	}
+
+	/* There is no way out of the loop that could leave us in the queue. */
+	Assert(SHMQueueIsDetached(&(MyProc->syncRepLinks)));
+	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
+	MyProc->waitLSN = 0;
+
+	/* Restore the ps display. */
+	if (ps_display_buffer != NULL)
+	{
+		set_ps_display(ps_display_buffer, false);
+		pfree(ps_display_buffer);
+	}
+}
+
 /*
  * Wait for synchronous replication, if requested by user.
  *
@@ -180,57 +434,9 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 		if (syncRepState == SYNC_REP_WAIT_COMPLETE)
 			break;
 
-		/*
-		 * If a wait for synchronous replication is pending, we can neither
-		 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
-		 * lead the client to believe that the transaction aborted, which
-		 * is not true: it's already committed locally. The former is no good
-		 * either: the client has requested synchronous replication, and is
-		 * entitled to assume that an acknowledged commit is also replicated,
-		 * which might not be true. So in this case we issue a WARNING (which
-		 * some clients may be able to interpret) and shut off further output.
-		 * We do NOT reset ProcDiePending, so that the process will die after
-		 * the commit is cleaned up.
-		 */
-		if (ProcDiePending)
-		{
-			ereport(WARNING,
-					(errcode(ERRCODE_ADMIN_SHUTDOWN),
-					 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * It's unclear what to do if a query cancel interrupt arrives.  We
-		 * can't actually abort at this point, but ignoring the interrupt
-		 * altogether is not helpful, so we just terminate the wait with a
-		 * suitable warning.
-		 */
-		if (QueryCancelPending)
-		{
-			QueryCancelPending = false;
-			ereport(WARNING,
-					(errmsg("canceling wait for synchronous replication due to user request"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * If the postmaster dies, we'll probably never get an
-		 * acknowledgement, because all the wal sender processes will exit. So
-		 * just bail out.
-		 */
-		if (!PostmasterIsAlive())
-		{
-			ProcDiePending = true;
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit())
 			break;
-		}
 
 		/*
 		 * Wait on latch.  Any condition that should wake us up will set the
@@ -403,6 +609,49 @@ SyncRepGetSynchronousStandby(void)
 }
 
 /*
+ * Check if the current WALSender process's application_name matches a name in
+ * causal_reads_standby_names (including '*' for wildcard).
+ */
+bool
+CausalReadsPotentialStandby(void)
+{
+	char *rawstring;
+	List	   *elemlist;
+	ListCell   *l;
+	bool		found = false;
+
+	/* Need a modifiable copy of string */
+	rawstring = pstrdup(causal_reads_standby_names);
+
+	/* Parse string into list of identifiers */
+	if (!SplitIdentifierString(rawstring, ',', &elemlist))
+	{
+		/* syntax error in list */
+		pfree(rawstring);
+		list_free(elemlist);
+		/* GUC machinery will have already complained - no need to do again */
+		return 0;
+	}
+
+	foreach(l, elemlist)
+	{
+		char	   *standby_name = (char *) lfirst(l);
+
+		if (pg_strcasecmp(standby_name, application_name) == 0 ||
+			pg_strcasecmp(standby_name, "*") == 0)
+		{
+			found = true;
+			break;
+		}
+	}
+
+	pfree(rawstring);
+	list_free(elemlist);
+
+	return found;
+}
+
+/*
  * Update the LSNs on each queue based upon our latest state. This
  * implements a simple policy of first-valid-standby-releases-waiter.
  *
@@ -410,22 +659,27 @@ SyncRepGetSynchronousStandby(void)
  * perhaps also which information we store as well.
  */
 void
-SyncRepReleaseWaiters(void)
+SyncRepReleaseWaiters(bool walsender_cr_available_or_joining)
 {
 	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
+	int			numapply = 0;
+	int			numcausalreadsapply = 0;
+	bool		is_highest_priority_sync_standby;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
-	 * potential standbys then we have nothing to do. If we are still starting
-	 * up, still running base backup or the current flush position is still
-	 * invalid, then leave quickly also.
+	 * potential standbys and not in a state that causal_reads waits for, then
+	 * we have nothing to do. If we are still starting up, still running base
+	 * backup or the current flush position is still invalid, then leave
+	 * quickly also.
 	 */
-	if (MyWalSnd->sync_standby_priority == 0 ||
-		MyWalSnd->state < WALSNDSTATE_STREAMING ||
-		XLogRecPtrIsInvalid(MyWalSnd->flush))
+	if (!walsender_cr_available_or_joining &&
+		(MyWalSnd->sync_standby_priority == 0 ||
+		 MyWalSnd->state < WALSNDSTATE_STREAMING ||
+		 XLogRecPtrIsInvalid(MyWalSnd->flush)))
 		return;
 
 	/*
@@ -435,45 +689,77 @@ SyncRepReleaseWaiters(void)
 	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
 	syncWalSnd = SyncRepGetSynchronousStandby();
 
-	/* We should have found ourselves at least */
-	Assert(syncWalSnd != NULL);
+	/*
+	 * If we aren't managing the highest priority standby then make a note of
+	 * that so we can announce a takeover in the log if we ever get that job.
+	 */
+	is_highest_priority_sync_standby = syncWalSnd == MyWalSnd;
+	if (!is_highest_priority_sync_standby)
+		announce_next_takeover = true;
 
 	/*
-	 * If we aren't managing the highest priority standby then just leave.
+	 * If we aren't managing the highest priority standby or a standby in
+	 * causal reads 'joining' or 'available' state, then just leave.
 	 */
-	if (syncWalSnd != MyWalSnd)
+	if (!is_highest_priority_sync_standby && !walsender_cr_available_or_joining)
 	{
 		LWLockRelease(SyncRepLock);
-		announce_next_takeover = true;
 		return;
 	}
 
 	/*
 	 * Set the lsn first so that when we wake backends they will release up to
-	 * this location.
+	 * this location.  For the single-standby synchronous commit levels, we
+	 * only do this if we are the current synchronous standby and we are
+	 * advancing the LSN further than it has been advanced before, so that
+	 * SyncRepWaitForLSN can skip waiting in some cases.
 	 */
-	if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+	if (is_highest_priority_sync_standby)
 	{
-		walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
-		numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
-		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
+		if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
+			numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE,
+										MyWalSnd->write);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
+			numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH,
+										MyWalSnd->flush);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+			numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY,
+										MyWalSnd->apply);
+		}
 	}
+	/*
+	 * For causal_reads, all walsenders currently in available or joining
+	 * state must reach the LSN on their own, and standbys will reach LSNs in
+	 * any order.  It doesn't make sense to keep the highest seen LSN in a
+	 * single walsndctl->lsn element.  (CausalReadsWaitForLSN has handling for
+	 * LSNs that have already been reached).
+	 */
+	if (walsender_cr_available_or_joining)
+		numcausalreadsapply =
+			SyncRepWakeQueue(false, SYNC_REP_WAIT_CAUSAL_READS_APPLY,
+							 MyWalSnd->apply);
 
 	LWLockRelease(SyncRepLock);
 
-	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X",
+	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%X, %d procs to causal_reads apply",
 		 numwrite, (uint32) (MyWalSnd->write >> 32), (uint32) MyWalSnd->write,
-	   numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush);
+		 numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush,
+		 numapply, (uint32) (MyWalSnd->apply >> 32), (uint32) MyWalSnd->apply,
+		 numcausalreadsapply);
 
 	/*
 	 * If we are managing the highest priority standby, though we weren't
 	 * prior to this, then announce we are now the sync standby.
 	 */
-	if (announce_next_takeover)
+	if (is_highest_priority_sync_standby && announce_next_takeover)
 	{
 		announce_next_takeover = false;
 		ereport(LOG,
@@ -548,9 +834,8 @@ SyncRepGetStandbyPriority(void)
  * Must hold SyncRepLock.
  */
 static int
-SyncRepWakeQueue(bool all, int mode)
+SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn)
 {
-	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	PGPROC	   *proc = NULL;
 	PGPROC	   *thisproc = NULL;
 	int			numprocs = 0;
@@ -567,7 +852,7 @@ SyncRepWakeQueue(bool all, int mode)
 		/*
 		 * Assume the queue is ordered by LSN
 		 */
-		if (!all && walsndctl->lsn[mode] < proc->waitLSN)
+		if (!all && lsn < proc->waitLSN)
 			return numprocs;
 
 		/*
@@ -627,7 +912,7 @@ SyncRepUpdateSyncStandbysDefined(void)
 			int			i;
 
 			for (i = 0; i < NUM_SYNC_REP_WAIT_MODE; i++)
-				SyncRepWakeQueue(true, i);
+				SyncRepWakeQueue(true, i, InvalidXLogRecPtr);
 		}
 
 		/*
@@ -679,13 +964,31 @@ SyncRepQueueIsOrderedByLSN(int mode)
 #endif
 
 /*
+ * Make sure that CausalReadsWaitForLSN can't return until after the given
+ * lease expiry time has been reached.
+ *
+ * Wake up all backends waiting in CausalReadsWaitForLSN, because the set of
+ * available/joining peers has changed, and there is a new stall time they
+ * need to observe.
+ */
+void
+CausalReadsBeginStall(TimestampTz lease_expiry_time)
+{
+	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+	WalSndCtl->stall_causal_reads_until =
+		Max(WalSndCtl->stall_causal_reads_until, lease_expiry_time);
+	SyncRepWakeQueue(true, SYNC_REP_WAIT_CAUSAL_READS_APPLY, InvalidXLogRecPtr);
+	LWLockRelease(SyncRepLock);
+}
+
+/*
  * ===========================================================
  * Synchronous Replication functions executed by any process
  * ===========================================================
  */
 
 bool
-check_synchronous_standby_names(char **newval, void **extra, GucSource source)
+check_standby_names(char **newval, void **extra, GucSource source)
 {
 	char	   *rawstring;
 	List	   *elemlist;
@@ -728,6 +1031,9 @@ assign_synchronous_commit(int newval, void *extra)
 		case SYNCHRONOUS_COMMIT_REMOTE_FLUSH:
 			SyncRepWaitMode = SYNC_REP_WAIT_FLUSH;
 			break;
+		case SYNCHRONOUS_COMMIT_REMOTE_APPLY:
+			SyncRepWaitMode = SYNC_REP_WAIT_APPLY;
+			break;
 		default:
 			SyncRepWaitMode = SYNC_REP_NO_WAIT;
 			break;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 7b36e02..4526d73 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -55,6 +55,7 @@
 #include "libpq/pqformat.h"
 #include "libpq/pqsignal.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "replication/walsender.h"
 #include "storage/ipc.h"
@@ -101,6 +102,7 @@ static uint32 recvOff = 0;
  */
 static volatile sig_atomic_t got_SIGHUP = false;
 static volatile sig_atomic_t got_SIGTERM = false;
+static volatile sig_atomic_t got_SIGUSR2 = false;
 
 /*
  * LogstreamResult indicates the byte positions that we have already
@@ -145,14 +147,33 @@ static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
 static void XLogWalRcvSendReply(bool force, bool requestReply);
 static void XLogWalRcvSendHSFeedback(bool immed);
-static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
+static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+								  TimestampTz *causalReadsUntil);
 
 /* Signal handlers */
 static void WalRcvSigHupHandler(SIGNAL_ARGS);
 static void WalRcvSigUsr1Handler(SIGNAL_ARGS);
+static void WalRcvSigUsr2Handler(SIGNAL_ARGS);
 static void WalRcvShutdownHandler(SIGNAL_ARGS);
 static void WalRcvQuickDieHandler(SIGNAL_ARGS);
 
+static void WalRcvBlockSigUsr2(void)
+{
+	sigset_t mask;
+
+	sigemptyset(&mask);
+	sigaddset(&mask, SIGUSR2);
+	sigprocmask(SIG_BLOCK, &mask, NULL);
+}
+
+static void WalRcvUnblockSigUsr2(void)
+{
+	sigset_t mask;
+
+	sigemptyset(&mask);
+	sigaddset(&mask, SIGUSR2);
+	sigprocmask(SIG_UNBLOCK, &mask, NULL);
+}
 
 static void
 ProcessWalRcvInterrupts(void)
@@ -200,6 +221,7 @@ WalReceiverMain(void)
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz last_recv_timestamp;
 	bool		ping_sent;
+	bool		forceReply;
 
 	/*
 	 * WalRcv should be set up already (if we are a backend, we inherit this
@@ -251,6 +273,7 @@ WalReceiverMain(void)
 
 	/* Initialise to a sanish value */
 	walrcv->lastMsgSendTime = walrcv->lastMsgReceiptTime = walrcv->latestWalEndTime = GetCurrentTimestamp();
+	walrcv->causalReadsLease = 0;
 
 	SpinLockRelease(&walrcv->mutex);
 
@@ -268,7 +291,7 @@ WalReceiverMain(void)
 	pqsignal(SIGALRM, SIG_IGN);
 	pqsignal(SIGPIPE, SIG_IGN);
 	pqsignal(SIGUSR1, WalRcvSigUsr1Handler);
-	pqsignal(SIGUSR2, SIG_IGN);
+	pqsignal(SIGUSR2, WalRcvSigUsr2Handler);
 
 	/* Reset some signals that are accepted by postmaster but not here */
 	pqsignal(SIGCHLD, SIG_DFL);
@@ -299,6 +322,10 @@ WalReceiverMain(void)
 	/* Unblock signals (they were blocked when the postmaster forked us) */
 	PG_SETMASK(&UnBlockSig);
 
+	/* Block SIGUSR2 (we unblock it only during network waits). */
+	WalRcvBlockSigUsr2();
+	got_SIGUSR2 = false;
+
 	/* Establish the connection to the primary for XLOG streaming */
 	EnableWalRcvImmediateExit();
 	walrcv_connect(conninfo);
@@ -408,7 +435,9 @@ WalReceiverMain(void)
 				}
 
 				/* Wait a while for data to arrive */
+				WalRcvUnblockSigUsr2();
 				len = walrcv_receive(NAPTIME_PER_CYCLE, &buf);
+				WalRcvBlockSigUsr2();
 				if (len != 0)
 				{
 					/*
@@ -439,11 +468,21 @@ WalReceiverMain(void)
 							endofwal = true;
 							break;
 						}
+						WalRcvUnblockSigUsr2();
 						len = walrcv_receive(0, &buf);
+						WalRcvBlockSigUsr2();
+					}
+
+					if (got_SIGUSR2)
+					{
+						/* The recovery process asked us to force a reply. */
+						got_SIGUSR2 = false;
+						forceReply = true;
 					}
 
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(false, false);
+					XLogWalRcvSendReply(forceReply, false);
+					forceReply = false;
 
 					/*
 					 * If we've written some records, flush them to disk and
@@ -498,7 +537,15 @@ WalReceiverMain(void)
 						}
 					}
 
-					XLogWalRcvSendReply(requestReply, requestReply);
+					/* Check if the startup process has signaled us. */
+					if (got_SIGUSR2)
+					{
+						got_SIGUSR2 = false;
+						forceReply = true;
+					}
+
+					XLogWalRcvSendReply(requestReply || forceReply, requestReply);
+					forceReply = false;
 					XLogWalRcvSendHSFeedback(false);
 				}
 			}
@@ -740,6 +787,13 @@ WalRcvSigUsr1Handler(SIGNAL_ARGS)
 	errno = save_errno;
 }
 
+/* SIGUSR2: used to receive wakeups from recovery */
+static void
+WalRcvSigUsr2Handler(SIGNAL_ARGS)
+{
+	got_SIGUSR2 = true;
+}
+
 /* SIGTERM: set flag for main loop, or shutdown immediately if safe */
 static void
 WalRcvShutdownHandler(SIGNAL_ARGS)
@@ -800,6 +854,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 	XLogRecPtr	walEnd;
 	TimestampTz sendTime;
 	bool		replyRequested;
+	TimestampTz causalReadsLease;
 
 	resetStringInfo(&incoming_message);
 
@@ -820,7 +875,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				walEnd = pq_getmsgint64(&incoming_message);
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, NULL);
 
 				buf += hdrlen;
 				len -= hdrlen;
@@ -830,7 +885,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 		case 'k':				/* Keepalive */
 			{
 				/* copy message to StringInfo */
-				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
+				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char) + sizeof(int64);
 				if (len != hdrlen)
 					ereport(ERROR,
 							(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -842,8 +897,12 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
 				replyRequested = pq_getmsgbyte(&incoming_message);
+				causalReadsLease = IntegerTimestampToTimestampTz(
+					pq_getmsgint64(&incoming_message));
+				ProcessWalSndrMessage(walEnd, sendTime, &causalReadsLease);
 
-				ProcessWalSndrMessage(walEnd, sendTime);
+				/* Remember primary's timestamp at this WAL location. */
+				SetXLogReplayTimestampAtLsn(sendTime, walEnd);
 
 				/* If the primary requested a reply, send one immediately */
 				if (replyRequested)
@@ -1037,6 +1096,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	XLogRecPtr	applyPtr;
 	static TimestampTz sendTime = 0;
 	TimestampTz now;
+	TimestampTz applyTimestamp = 0;
 
 	/*
 	 * If the user doesn't want status to be reported to the master, be sure
@@ -1068,7 +1128,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	/* Construct a new message */
 	writePtr = LogstreamResult.Write;
 	flushPtr = LogstreamResult.Flush;
-	applyPtr = GetXLogReplayRecPtr(NULL);
+	applyTimestamp = GetXLogReplayTimestamp(&applyPtr);
 
 	resetStringInfo(&reply_message);
 	pq_sendbyte(&reply_message, 'r');
@@ -1076,6 +1136,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	pq_sendint64(&reply_message, flushPtr);
 	pq_sendint64(&reply_message, applyPtr);
 	pq_sendint64(&reply_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&reply_message, TimestampTzToIntegerTimestamp(applyTimestamp));
 	pq_sendbyte(&reply_message, requestReply ? 1 : 0);
 
 	/* Send it */
@@ -1174,15 +1235,52 @@ XLogWalRcvSendHSFeedback(bool immed)
  * Update shared memory status upon receiving a message from primary.
  *
  * 'walEnd' and 'sendTime' are the end-of-WAL and timestamp of the latest
- * message, reported by primary.
+ * message, reported by primary.  'causalReadsLease' is a pointer to
+ * the time the primary promises that this standby can safely claim to be
+ * causally consistent, to 0 if it cannot, or a NULL pointer for no change.
  */
 static void
-ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
+ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+					  TimestampTz *causalReadsLease)
 {
 	WalRcvData *walrcv = WalRcv;
 
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
 
+	/* Sanity check for the causalReadsLease time. */
+	if (causalReadsLease != NULL && *causalReadsLease != 0)
+	{
+		/* Deduce max_clock_skew from the causalReadsLease and sendTime. */
+#ifdef HAVE_INT64_TIMESTAMP
+		int64 diffMillis = (*causalReadsLease - sendTime) / 1000;
+#else
+		int64 diffMillis = (*causalReadsLease - sendTime) * 1000;
+#endif
+		int64 max_clock_skew = diffMillis / (CAUSAL_READS_CLOCK_SKEW_RATIO - 1);
+
+		if (sendTime > TimestampTzPlusMilliseconds(lastMsgReceiptTime, max_clock_skew))
+		{
+			/*
+			 * The primary's clock is more than max_clock_skew + network
+			 * latency ahead of the standby's clock.  (If the primary's clock
+			 * is more than max_clock_skew ahead of the standby's clock, but
+			 * by less than the network latency, then there isn't much we can
+			 * do to detect that; but it still seems useful to have this basic
+			 * sanity check for wildly misconfigured servers.)
+			 */
+			elog(LOG, "the primary server's clock time is too far ahead");
+			causalReadsLease = NULL;
+		}
+		/*
+		 * We could also try to detect cases where sendTime is more than
+		 * max_clock_skew in the past according to the standby's clock, but
+		 * that is indistinguishable from network latency/buffering, so we
+		 * could produce misleading error messages; if we do nothing, the
+		 * consequence is 'standby is not available for causal reads' errors
+		 * which should cause the user to investigate.
+		 */
+	}
+
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
 	if (walrcv->latestWalEnd < walEnd)
@@ -1190,6 +1288,8 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->latestWalEnd = walEnd;
 	walrcv->lastMsgSendTime = sendTime;
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
+	if (causalReadsLease != NULL)
+		walrcv->causalReadsLease = *causalReadsLease;
 	SpinLockRelease(&walrcv->mutex);
 
 	if (log_min_messages <= DEBUG2)
@@ -1222,6 +1322,22 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 }
 
 /*
+ * Wake up the walreceiver if it happens to be blocked in walrcv_receive,
+ * and tell it that a commit record has been applied.
+ *
+ * This is called by the startup process whenever interesting xlog records
+ * are applied, so that walreceiver can check if it needs to send an apply
+ * notification back to the master which may be waiting in a COMMIT with
+ * synchronous_commit = apply or causal_reads = on.
+ */
+void
+WalRcvWakeup(void)
+{
+	if (WalRcv->pid != 0)
+		kill(WalRcv->pid, SIGUSR2);
+}
+
+/*
  * Return a string constant representing the state. This is used
  * in system functions and views, and should *not* be translated.
  */
diff --git a/src/backend/replication/walreceiverfuncs.c b/src/backend/replication/walreceiverfuncs.c
index 5f6e423..f398a75 100644
--- a/src/backend/replication/walreceiverfuncs.c
+++ b/src/backend/replication/walreceiverfuncs.c
@@ -25,9 +25,11 @@
 
 #include "access/xlog_internal.h"
 #include "postmaster/startup.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "storage/pmsignal.h"
 #include "storage/shmem.h"
+#include "utils/guc.h"
 #include "utils/timestamp.h"
 
 WalRcvData *WalRcv = NULL;
@@ -374,3 +376,21 @@ GetReplicationTransferLatency(void)
 
 	return ms;
 }
+
+/*
+ * Used by snapmgr to check if this standby has a valid lease, granting it the
+ * right to consider itself available for causal reads.
+ */
+bool
+WalRcvCausalReadsAvailable(void)
+{
+	WalRcvData *walrcv = WalRcv;
+	TimestampTz now = GetCurrentTimestamp();
+	bool result;
+
+	SpinLockAcquire(&walrcv->mutex);
+	result = walrcv->causalReadsLease != 0 && now <= walrcv->causalReadsLease;
+	SpinLockRelease(&walrcv->mutex);
+
+	return result;
+}
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index c03e045..55c10e4 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -153,9 +153,20 @@ static StringInfoData tmpbuf;
  */
 static TimestampTz last_reply_timestamp = 0;
 
+static TimestampTz last_keepalive_timestamp = 0;
+
 /* Have we sent a heartbeat message asking for reply, since last reply? */
 static bool waiting_for_ping_response = false;
 
+/* How long do need to stay in JOINING state? */
+static XLogRecPtr causal_reads_joining_until = 0;
+
+/* The last causal reads lease sent to the standby. */
+static TimestampTz causal_reads_last_lease = 0;
+
+/* Is this WALSender listed in causal_reads_standby_names? */
+static bool am_potential_causal_reads_standby = false;
+
 /*
  * While streaming WAL in Copy mode, streamingDoneSending is set to true
  * after we have sent CopyDone. We should not send any more CopyData messages
@@ -242,6 +253,57 @@ InitWalSender(void)
 }
 
 /*
+ * If we are exiting unexpectedly, we may need to communicate with concurrent
+ * causal_reads commits to maintain the causal consistency guarantee.
+ */
+static void
+PrepareUncleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * We've lost contact with the standby, but it may still be alive.  We
+		 * can't let any causal_reads transactions return until we've stalled
+		 * for long enough for a zombie standby to start raising errors
+		 * because its lease has expired.
+		 */
+		elog(LOG, "standby \"%s\" is lost (no longer available for causal reads)", application_name);
+		CausalReadsBeginStall(causal_reads_last_lease);
+
+		/*
+		 * We set the state to a lower level _after_ beginning the stall,
+		 * otherwise there would be a tiny window where commits could return
+		 * without observing the stall.
+		 */
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
+ * We are shutting down because we received a goodbye message from the
+ * walreceiver.
+ */
+static void
+PrepareCleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * The standby is shutting down, so it won't be running any more
+		 * transactions.  It is therefore safe to stop waiting for it, and no
+		 * stall is necessary.
+		 */
+		elog(LOG, "standby \"%s\" is leaving (no longer available for causal reads)", application_name);
+
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
  * Clean up after an error.
  *
  * WAL sender processes don't use transactions like regular backends do.
@@ -264,7 +326,10 @@ WalSndErrorCleanup(void)
 
 	replication_active = false;
 	if (walsender_ready_to_stop)
+	{
+		PrepareUncleanExit();
 		proc_exit(0);
+	}
 
 	/* Revert back to startup state */
 	WalSndSetState(WALSNDSTATE_STARTUP);
@@ -276,6 +341,8 @@ WalSndErrorCleanup(void)
 static void
 WalSndShutdown(void)
 {
+	PrepareUncleanExit();
+
 	/*
 	 * Reset whereToSendOutput to prevent ereport from attempting to send any
 	 * more messages to the standby.
@@ -1386,6 +1453,7 @@ ProcessRepliesIfAny(void)
 		if (r < 0)
 		{
 			/* unexpected error or EOF */
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1402,6 +1470,7 @@ ProcessRepliesIfAny(void)
 		resetStringInfo(&reply_message);
 		if (pq_getmessage(&reply_message, 0))
 		{
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1451,6 +1520,7 @@ ProcessRepliesIfAny(void)
 				 * 'X' means that the standby is closing down the socket.
 				 */
 			case 'X':
+				PrepareCleanExit();
 				proc_exit(0);
 
 			default:
@@ -1543,15 +1613,29 @@ ProcessStandbyReplyMessage(void)
 	XLogRecPtr	writePtr,
 				flushPtr,
 				applyPtr;
+	int			applyLagMs;
 	bool		replyRequested;
+	TimestampTz now = GetCurrentTimestamp();
+	TimestampTz applyTimestamp;
 
 	/* the caller already consumed the msgtype byte */
 	writePtr = pq_getmsgint64(&reply_message);
 	flushPtr = pq_getmsgint64(&reply_message);
 	applyPtr = pq_getmsgint64(&reply_message);
 	(void) pq_getmsgint64(&reply_message);		/* sendTime; not used ATM */
+	applyTimestamp = IntegerTimestampToTimestampTz(pq_getmsgint64(&reply_message));
 	replyRequested = pq_getmsgbyte(&reply_message);
 
+	/* Compute the apply lag in milliseconds. */
+	if (applyTimestamp == 0)
+		applyLagMs = -1;
+	else
+#ifdef HAVE_INT64_TIMESTAMP
+		applyLagMs = (now - applyTimestamp) / 1000;
+#else
+		applyLagMs = (now - applyTimestamp) * 1000.0;
+#endif
+
 	elog(DEBUG2, "write %X/%X flush %X/%X apply %X/%X%s",
 		 (uint32) (writePtr >> 32), (uint32) writePtr,
 		 (uint32) (flushPtr >> 32), (uint32) flushPtr,
@@ -1568,16 +1652,116 @@ ProcessStandbyReplyMessage(void)
 	 */
 	{
 		WalSnd *walsnd = MyWalSnd;
+		WalSndCausalReadsState causal_reads_state = walsnd->causal_reads_state;
+		bool causal_reads_state_changed = false;
+		bool causal_reads_set_joining_until = false;
+
+		/*
+		 * Handle causal reads state transitions, if a causal_reads_timeout is
+		 * configured, this standby is listed in causal_reads_standby_names,
+		 * and we are a primary database (not a cascading standby).
+		 */
+		if (causal_reads_timeout != 0 &&
+			am_potential_causal_reads_standby &&
+			!am_cascading_walsender)
+		{
+			if (applyLagMs >= 0 && applyLagMs < causal_reads_timeout)
+			{
+				if (causal_reads_state == WALSNDCRSTATE_UNAVAILABLE)
+				{
+					/*
+					 * The standby is applying fast enough.  We can't grant a
+					 * lease yet though, we need to wait for everything that
+					 * was committed while this standby was unavailable to be
+					 * applied first.  We move to joining state while we wait
+					 * for the standby to catch up.
+					 */
+					causal_reads_state = WALSNDCRSTATE_JOINING;
+					causal_reads_set_joining_until = true;
+					causal_reads_state_changed = true;
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING &&
+						 applyPtr >= causal_reads_joining_until)
+				{
+					/*
+					 * The standby has applied everything committed before we
+					 * reached joining state, and has been waiting for remote
+					 * apply on this standby while it's been in joining state,
+					 * so it is safe to move to available state and send a
+					 * lease.
+					 */
+					causal_reads_state = WALSNDCRSTATE_AVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+			else
+			{
+				if (causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+					/*
+					 * We are dropping a causal reads available standby, so we
+					 * mustn't let any commit command that is waiting in
+					 * CausalReadsWaitForLSN return until we are sure that the
+					 * standby definitely knows that it's not available and
+					 * starts raising errors for causal_reads transactions.
+					 * TODO: We could just wait until the standby acks that
+					 * its lease has been cancelled, and start numbering
+					 * keepalives and sending the number back in replies, so
+					 * we know it's acking the right message; then lagging
+					 * standbys would be less disruptive, but for now we just
+					 * wait for the lease to expire, as we do when we lose
+					 * contact with a standby, for the sake of simplicity.
+					 */
+					CausalReadsBeginStall(causal_reads_last_lease);
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING)
+				{
+					/*
+					 * Dropping a joining standby doesn't require a stall,
+					 * because the standby doesn't think it's available, so
+					 * it's already raising the error for causal_reads
+					 * transactions.
+					 */
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+		}
 
 		SpinLockAcquire(&walsnd->mutex);
 		walsnd->write = writePtr;
 		walsnd->flush = flushPtr;
 		walsnd->apply = applyPtr;
+		walsnd->applyLagMs = applyLagMs;
+		walsnd->causal_reads_state = causal_reads_state;
 		SpinLockRelease(&walsnd->mutex);
+
+		if (causal_reads_set_joining_until)
+		{
+			/*
+			 * Record the end of the primary's WAL at some arbitrary point
+			 * observed _after_ we moved to joining state (so that causal
+			 * reads commits start waiting, closing a race).  The standby
+			 * won't become available until it has replayed up to here.
+			 */
+			causal_reads_joining_until = GetFlushRecPtr();
+		}
+
+		if (causal_reads_state_changed)
+		{
+			WalSndKeepalive(true);
+			elog(LOG, "standby \"%s\" is %s", application_name,
+				 causal_reads_state == WALSNDCRSTATE_UNAVAILABLE ? "unavailable for causal reads" :
+				 causal_reads_state == WALSNDCRSTATE_JOINING ? "joining as a causal reads standby..." :
+				 causal_reads_state == WALSNDCRSTATE_AVAILABLE ? "available for causal reads" :
+				 "UNKNOWN");
+		}
 	}
 
 	if (!am_cascading_walsender)
-		SyncRepReleaseWaiters();
+		SyncRepReleaseWaiters(MyWalSnd->causal_reads_state >= WALSNDCRSTATE_JOINING);
 
 	/*
 	 * Advance our local xmin horizon when the client confirmed a flush.
@@ -1724,27 +1908,34 @@ WalSndComputeSleeptime(TimestampTz now)
 {
 	long		sleeptime = 10000;		/* 10 s */
 
-	if (wal_sender_timeout > 0 && last_reply_timestamp > 0)
+	if ((wal_sender_timeout > 0 || causal_reads_timeout > 0) && last_reply_timestamp > 0)
 	{
 		TimestampTz wakeup_time;
 		long		sec_to_timeout;
 		int			microsec_to_timeout;
 
-		/*
-		 * At the latest stop sleeping once wal_sender_timeout has been
-		 * reached.
-		 */
-		wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-												  wal_sender_timeout);
-
-		/*
-		 * If no ping has been sent yet, wakeup when it's time to do so.
-		 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
-		 * the timeout passed without a response.
-		 */
-		if (!waiting_for_ping_response)
+		if (causal_reads_timeout != 0)
+			wakeup_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+													  causal_reads_timeout /
+													  CAUSAL_READS_KEEPALIVE_RATIO);
+		else
+		{
+			/*
+			 * At the latest stop sleeping once wal_sender_timeout has been
+			 * reached.
+			 */
 			wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-													  wal_sender_timeout / 2);
+													  wal_sender_timeout);
+
+			/*
+			 * If no ping has been sent yet, wakeup when it's time to do so.
+			 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
+			 * the timeout passed without a response.
+			 */
+			if (!waiting_for_ping_response)
+				wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+														  wal_sender_timeout / 2);
+		}
 
 		/* Compute relative time until wakeup. */
 		TimestampDifference(now, wakeup_time,
@@ -1765,15 +1956,28 @@ static void
 WalSndCheckTimeOut(TimestampTz now)
 {
 	TimestampTz timeout;
+	int allowed_time;
 
 	/* don't bail out if we're doing something that doesn't require timeouts */
 	if (last_reply_timestamp <= 0)
 		return;
 
+	/*
+	 * If a causal_reads_timeout is configured, it is used instead of
+	 * wal_sender_timeout.  Ideally we'd use causal_reads_timeout / 2 +
+	 * allowance for network latency, but since walreceiver can become quite
+	 * bogged down fsyncing WAL we allow more tolerance.  (This could be
+	 * tightened up once standbys hand writing off to the WAL writer).
+	 */
+	if (causal_reads_timeout != 0)
+		allowed_time = causal_reads_timeout;
+	else
+		allowed_time = wal_sender_timeout;
+
 	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
-										  wal_sender_timeout);
+										  allowed_time);
 
-	if (wal_sender_timeout > 0 && now >= timeout)
+	if (allowed_time > 0 && now >= timeout)
 	{
 		/*
 		 * Since typically expiration of replication timeout means
@@ -1806,6 +2010,9 @@ WalSndLoop(WalSndSendDataCallback send_data)
 	last_reply_timestamp = GetCurrentTimestamp();
 	waiting_for_ping_response = false;
 
+	/* Check if we are managing potential causal_reads standby. */
+	am_potential_causal_reads_standby = CausalReadsPotentialStandby();
+
 	/*
 	 * Loop until we reach the end of this timeline or the client requests to
 	 * stop streaming.
@@ -1966,6 +2173,7 @@ InitWalSenderSlot(void)
 			walsnd->flush = InvalidXLogRecPtr;
 			walsnd->apply = InvalidXLogRecPtr;
 			walsnd->state = WALSNDSTATE_STARTUP;
+			walsnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
 			walsnd->latch = &MyProc->procLatch;
 			SpinLockRelease(&walsnd->mutex);
 			/* don't need the lock anymore */
@@ -2735,6 +2943,24 @@ WalSndGetStateString(WalSndState state)
 	return "UNKNOWN";
 }
 
+/*
+ * Return a string constant representing the causal reads state. This is used
+ * in system views, and should *not* be translated.
+ */
+static const char *
+WalSndGetCausalReadsStateString(WalSndCausalReadsState causal_reads_state)
+{
+	switch (causal_reads_state)
+	{
+		case WALSNDCRSTATE_UNAVAILABLE:
+			return "unavailable";
+		case WALSNDCRSTATE_JOINING:
+			return "joining";
+		case WALSNDCRSTATE_AVAILABLE:
+			return "available";
+	}
+	return "UNKNOWN";
+}
 
 /*
  * Returns activity of walsenders, including pids and xlog locations sent to
@@ -2743,7 +2969,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	8
+#define PG_STAT_GET_WAL_SENDERS_COLS	10
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2791,8 +3017,10 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		XLogRecPtr	write;
 		XLogRecPtr	flush;
 		XLogRecPtr	apply;
+		int			applyLagMs;
 		int			priority;
 		WalSndState state;
+		WalSndCausalReadsState causalReadsState;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
 		bool		nulls[PG_STAT_GET_WAL_SENDERS_COLS];
 
@@ -2802,9 +3030,11 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		SpinLockAcquire(&walsnd->mutex);
 		sentPtr = walsnd->sentPtr;
 		state = walsnd->state;
+		causalReadsState = walsnd->causal_reads_state;
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
+		applyLagMs = walsnd->applyLagMs;
 		priority = walsnd->sync_standby_priority;
 		SpinLockRelease(&walsnd->mutex);
 
@@ -2839,6 +3069,23 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				nulls[5] = true;
 			values[5] = LSNGetDatum(apply);
 
+			if (applyLagMs < 0)
+				nulls[6] = true;
+			else
+			{
+				Interval *applyLagInterval = palloc(sizeof(Interval));
+
+				applyLagInterval->month = 0;
+				applyLagInterval->day = 0;
+#ifdef HAVE_INT64_TIMESTAMP
+				applyLagInterval->time = applyLagMs * 1000;
+#else
+				applyLagInterval->time = applyLagMs / 1000.0;
+#endif
+				nulls[6] = false;
+				values[6] = IntervalPGetDatum(applyLagInterval);
+			}
+
 			/*
 			 * Treat a standby such as a pg_basebackup background process
 			 * which always returns an invalid flush location, as an
@@ -2846,18 +3093,21 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 			 */
 			priority = XLogRecPtrIsInvalid(walsnd->flush) ? 0 : priority;
 
-			values[6] = Int32GetDatum(priority);
+			values[7] = Int32GetDatum(priority);
 
 			/*
 			 * More easily understood version of standby state. This is purely
 			 * informational, not different from priority.
 			 */
 			if (priority == 0)
-				values[7] = CStringGetTextDatum("async");
+				values[8] = CStringGetTextDatum("async");
 			else if (walsnd == sync_standby)
-				values[7] = CStringGetTextDatum("sync");
+				values[8] = CStringGetTextDatum("sync");
 			else
-				values[7] = CStringGetTextDatum("potential");
+				values[8] = CStringGetTextDatum("potential");
+
+			values[9] =
+				CStringGetTextDatum(WalSndGetCausalReadsStateString(causalReadsState));
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
@@ -2877,14 +3127,52 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 static void
 WalSndKeepalive(bool requestReply)
 {
+	TimestampTz now;
+	TimestampTz causal_reads_lease;
+
 	elog(DEBUG2, "sending replication keepalive");
 
+	/*
+	 * If the walsender currently deems the standby to be available for causal
+	 * reads, then it grants a causal reads lease.  The lease authorizes the
+	 * standby to consider itself available for causal reads until a short
+	 * time in the future.  The primary promises to uphold the causal reads
+	 * guarantee until that time, by stalling commits until the the lease has
+	 * expired if necessary.
+	 */
+	now = GetCurrentTimestamp();
+	if (MyWalSnd->causal_reads_state < WALSNDCRSTATE_AVAILABLE)
+		causal_reads_lease = 0; /* Not available, no lease granted. */
+	else
+	{
+		/*
+		 * Since this timestamp is being sent to the standby where it will be
+		 * compared against a time generated by the standby's system clock, we
+		 * must consider clock skew.  First, we decide on a maximum tolerable
+		 * difference between system clocks.  If the primary's clock is ahead
+		 * of the standby's by more than this, then all bets are off (the
+		 * standby could falsely believe it has a valid lease).  If the
+		 * primary's clock is behind the standby's by more than this, then the
+		 * standby will err the other way and generate spurious errors in
+		 * causal_reads mode.  Rather than having a separate GUC for this, we
+		 * derive it from causal_reads_timeout.
+		 */
+		int max_clock_skew = causal_reads_timeout / CAUSAL_READS_CLOCK_SKEW_RATIO;
+
+		/* Compute and remember the expiry time of the lease we're granting. */
+		causal_reads_last_lease = TimestampTzPlusMilliseconds(now, causal_reads_timeout);
+		/* The version we'll send to the standby is adjusted to tolerate clock skew. */
+		causal_reads_lease =
+			TimestampTzPlusMilliseconds(causal_reads_last_lease, -max_clock_skew);
+	}
+
 	/* construct the message... */
 	resetStringInfo(&output_message);
 	pq_sendbyte(&output_message, 'k');
 	pq_sendint64(&output_message, sentPtr);
-	pq_sendint64(&output_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(now));
 	pq_sendbyte(&output_message, requestReply ? 1 : 0);
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(causal_reads_lease));
 
 	/* ... and send it wrapped in CopyData */
 	pq_putmessage_noblock('d', output_message.data, output_message.len);
@@ -2902,23 +3190,32 @@ WalSndKeepaliveIfNecessary(TimestampTz now)
 	 * Don't send keepalive messages if timeouts are globally disabled or
 	 * we're doing something not partaking in timeouts.
 	 */
-	if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
+	if ((wal_sender_timeout <= 0 && causal_reads_timeout == 0) || last_reply_timestamp <= 0)
 		return;
 
-	if (waiting_for_ping_response)
+	if (waiting_for_ping_response && causal_reads_timeout == 0)
 		return;
 
 	/*
 	 * If half of wal_sender_timeout has lapsed without receiving any reply
 	 * from the standby, send a keep-alive message to the standby requesting
 	 * an immediate reply.
+	 *
+	 * If causal_reads_timeout has been configured, use it to control
+	 * keepalive intervals rather than wal_sender_timeout.
 	 */
-	ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-											wal_sender_timeout / 2);
+	if (causal_reads_timeout != 0)
+		ping_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+												causal_reads_timeout /
+												CAUSAL_READS_KEEPALIVE_RATIO);
+	else
+		ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+												wal_sender_timeout / 2);
 	if (now >= ping_time)
 	{
 		WalSndKeepalive(true);
 		waiting_for_ping_response = true;
+		last_keepalive_timestamp = now;
 
 		/* Try to flush pending output to the client */
 		if (pq_flush_if_writable() != 0)
diff --git a/src/backend/utils/adt/timestamp.c b/src/backend/utils/adt/timestamp.c
index 1525d2a..6ff111f 100644
--- a/src/backend/utils/adt/timestamp.c
+++ b/src/backend/utils/adt/timestamp.c
@@ -1611,6 +1611,20 @@ IntegerTimestampToTimestampTz(int64 timestamp)
 #endif
 
 /*
+ * TimestampTzToIntegerTimestamp -- convert a native timestamp to int64 format
+ *
+ * When compiled with --enable-integer-datetimes, this is implemented as a
+ * no-op macro.
+ */
+#ifndef HAVE_INT64_TIMESTAMP
+int64
+TimestampTzToIntegerTimestamp(TimestampTz timestamp)
+{
+	return timestamp * 1000000;
+}
+#endif
+
+/*
  * TimestampDifference -- convert the difference between two timestamps
  *		into integer seconds and microseconds
  *
diff --git a/src/backend/utils/errcodes.txt b/src/backend/utils/errcodes.txt
index 04c9c00..d4bf0c0 100644
--- a/src/backend/utils/errcodes.txt
+++ b/src/backend/utils/errcodes.txt
@@ -302,6 +302,7 @@ Section: Class 40 - Transaction Rollback
 40001    E    ERRCODE_T_R_SERIALIZATION_FAILURE                              serialization_failure
 40003    E    ERRCODE_T_R_STATEMENT_COMPLETION_UNKNOWN                       statement_completion_unknown
 40P01    E    ERRCODE_T_R_DEADLOCK_DETECTED                                  deadlock_detected
+40P02    E    ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE                         causal_reads_not_available
 
 Section: Class 42 - Syntax Error or Access Rule Violation
 
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index ea5a09a..fb91cad 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -345,12 +345,13 @@ static const struct config_enum_entry constraint_exclusion_options[] = {
 };
 
 /*
- * Although only "on", "off", "remote_write", and "local" are documented, we
- * accept all the likely variants of "on" and "off".
+ * Although only "on", "off", "remote_apply", "remote_write", and "local" are
+ * documented, we accept all the likely variants of "on" and "off".
  */
 static const struct config_enum_entry synchronous_commit_options[] = {
 	{"local", SYNCHRONOUS_COMMIT_LOCAL_FLUSH, false},
 	{"remote_write", SYNCHRONOUS_COMMIT_REMOTE_WRITE, false},
+	{"remote_apply", SYNCHRONOUS_COMMIT_REMOTE_APPLY, false},
 	{"on", SYNCHRONOUS_COMMIT_ON, false},
 	{"off", SYNCHRONOUS_COMMIT_OFF, false},
 	{"true", SYNCHRONOUS_COMMIT_ON, true},
@@ -1632,6 +1633,16 @@ static struct config_bool ConfigureNamesBool[] =
 		NULL, NULL, NULL
 	},
 
+	{
+		{"causal_reads", PGC_USERSET, REPLICATION_STANDBY,
+		 gettext_noop("Enables causal reads."),
+		 NULL
+		},
+		&causal_reads,
+		false,
+		NULL, NULL, NULL
+	},
+
 	/* End-of-list marker */
 	{
 		{NULL, 0, 0, NULL, NULL}, NULL, false, NULL, NULL, NULL
@@ -1790,6 +1801,17 @@ static struct config_int ConfigureNamesInt[] =
 	},
 
 	{
+		{"causal_reads_timeout", PGC_SIGHUP, REPLICATION_STANDBY,
+			gettext_noop("Sets the maximum apply lag before causal reads standbys are no longer available."),
+			NULL,
+			GUC_UNIT_MS
+		},
+		&causal_reads_timeout,
+		0, 0, INT_MAX,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"max_connections", PGC_POSTMASTER, CONN_AUTH_SETTINGS,
 			gettext_noop("Sets the maximum number of concurrent connections."),
 			NULL
@@ -3386,7 +3408,18 @@ static struct config_string ConfigureNamesString[] =
 		},
 		&SyncRepStandbyNames,
 		"",
-		check_synchronous_standby_names, NULL, NULL
+		check_standby_names, NULL, NULL
+	},
+
+	{
+		{"causal_reads_standby_names", PGC_SIGHUP, REPLICATION_MASTER,
+			gettext_noop("List of names of potential causal reads standbys."),
+			NULL,
+			GUC_LIST_INPUT
+		},
+		&causal_reads_standby_names,
+		"*",
+		check_standby_names, NULL, NULL
 	},
 
 	{
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index 63e908d..b1455e1 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -46,8 +46,11 @@
 
 #include "access/transam.h"
 #include "access/xact.h"
+#include "access/xlog.h"
 #include "lib/pairingheap.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
+#include "replication/walreceiver.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
 #include "storage/procarray.h"
@@ -209,6 +212,18 @@ GetTransactionSnapshot(void)
 				 "cannot take query snapshot during a parallel operation");
 
 		/*
+		 * In causal_reads mode on a standby, check if we have definitely
+		 * applied WAL for any COMMIT that returned successfully on the
+		 * primary.
+		 *
+		 * TODO: Machine readable error code?
+		 */
+		if (causal_reads && RecoveryInProgress() && !WalRcvCausalReadsAvailable())
+			ereport(ERROR,
+					(errcode(ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE),
+					 errmsg("standby is not available for causal reads")));
+
+		/*
 		 * In transaction-snapshot mode, the first snapshot must live until
 		 * end of xact regardless of what the caller does with it, so we must
 		 * make a copy of it rather than returning CurrentSnapshotData
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index ebeb582..4037dc6 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -60,7 +60,11 @@ typedef enum
 	SYNCHRONOUS_COMMIT_LOCAL_FLUSH,		/* wait for local flush only */
 	SYNCHRONOUS_COMMIT_REMOTE_WRITE,	/* wait for local flush and remote
 										 * write */
-	SYNCHRONOUS_COMMIT_REMOTE_FLUSH		/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_FLUSH,	/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_APPLY,	/* wait for local flush and remote
+										 * apply */
+	SYNCHRONOUS_COMMIT_CONSISTENT_APPLY /* wait for local flusha and remote
+										   apply with causal consistency */
 }	SyncCommitLevel;
 
 /* Define the default setting for synchonous_commit */
@@ -144,10 +148,13 @@ typedef void (*SubXactCallback) (SubXactEvent event, SubTransactionId mySubid,
  * EOXact... routines which run at the end of the original transaction
  * completion.
  */
+#define XACT_COMPLETION_SYNC_APPLY_FEEDBACK		(1U << 29)
 #define XACT_COMPLETION_UPDATE_RELCACHE_FILE	(1U << 30)
 #define XACT_COMPLETION_FORCE_SYNC_COMMIT		(1U << 31)
 
 /* Access macros for above flags */
+#define XactCompletionSyncApplyFeedback(xinfo) \
+	(!!(xinfo & XACT_COMPLETION_SYNC_APPLY_FEEDBACK))
 #define XactCompletionRelcacheInitFileInval(xinfo) \
 	(!!(xinfo & XACT_COMPLETION_UPDATE_RELCACHE_FILE))
 #define XactCompletionForceSyncCommit(xinfo) \
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index ecd30ce..efb9719 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -236,6 +236,9 @@ extern void GetXLogReceiptTime(TimestampTz *rtime, bool *fromStream);
 extern XLogRecPtr GetXLogReplayRecPtr(TimeLineID *replayTLI);
 extern XLogRecPtr GetXLogInsertRecPtr(void);
 extern XLogRecPtr GetXLogWriteRecPtr(void);
+extern void SetXLogReplayTimestamp(TimestampTz timestamp);
+extern void SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn);
+extern TimestampTz GetXLogReplayTimestamp(XLogRecPtr *lsn);
 extern bool RecoveryIsPaused(void);
 extern void SetRecoveryPause(bool recoveryPause);
 extern TimestampTz GetLatestXTime(void);
@@ -268,6 +271,8 @@ extern bool CheckPromoteSignal(void);
 extern void WakeupRecovery(void);
 extern void SetWalWriterSleeping(bool sleeping);
 
+extern void XLogRequestWalReceiverReply(void);
+
 extern void assign_max_wal_size(int newval, void *extra);
 extern void assign_checkpoint_completion_target(double newval, void *extra);
 
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 62b9125..fa9b184 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2710,7 +2710,7 @@ DATA(insert OID = 1936 (  pg_stat_get_backend_idset		PGNSP PGUID 12 1 100 0 0 f
 DESCR("statistics: currently active backend IDs");
 DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f f f t s r 1 0 2249 "23" "{23,26,23,26,25,25,25,16,1184,1184,1184,1184,869,25,23,28,28,16,25,25,23,16,25}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{pid,datid,pid,usesysid,application_name,state,query,waiting,xact_start,query_start,backend_start,state_change,client_addr,client_hostname,client_port,backend_xid,backend_xmin,ssl,sslversion,sslcipher,sslbits,sslcompression,sslclientdn}" _null_ _null_ pg_stat_get_activity _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active backends");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,23,25}" "{o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25,25}" "{o,o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state,causal_reads_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index 96e059b..76a4ee9 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -23,14 +23,34 @@
 #define SYNC_REP_NO_WAIT		-1
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
+#define SYNC_REP_WAIT_APPLY		2
+#define SYNC_REP_WAIT_CAUSAL_READS_APPLY 3
 
-#define NUM_SYNC_REP_WAIT_MODE	2
+#define NUM_SYNC_REP_WAIT_MODE	4
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
 #define SYNC_REP_WAITING			1
 #define SYNC_REP_WAIT_COMPLETE		2
 
+/*
+ * ratio of causal_read_timeout to max_clock_skew (4 means than the maximum
+ * tolerated clock difference between primary and standbys using causal_reads
+ * is 1/4 of causal_reads_timeout)
+ */
+#define CAUSAL_READS_CLOCK_SKEW_RATIO 4
+
+/*
+ * ratio of causal_reads_timeout to keepalive time (2 means that the effective
+ * keepalive time is 1/2 of the causal_reads_timeout GUC when it is non-zero)
+ */
+#define CAUSAL_READS_KEEPALIVE_RATIO 2
+
+/* GUC variables */
+extern int causal_reads_timeout;
+extern bool causal_reads;
+extern char *causal_reads_standby_names;
+
 /* user-settable parameters for synchronous replication */
 extern char *SyncRepStandbyNames;
 
@@ -42,16 +62,23 @@ extern void SyncRepCleanupAtProcExit(void);
 
 /* called by wal sender */
 extern void SyncRepInitConfig(void);
-extern void SyncRepReleaseWaiters(void);
+extern void SyncRepReleaseWaiters(bool walsender_cr_available_or_joining);
 
 /* called by checkpointer */
 extern void SyncRepUpdateSyncStandbysDefined(void);
 
+/* called by user backend (xact.c) */
+extern void CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN);
+
+/* called by wal sender */
+extern void CausalReadsBeginStall(TimestampTz lease_expiry_time);
+extern bool CausalReadsPotentialStandby(void);
+
 /* forward declaration to avoid pulling in walsender_private.h */
 struct WalSnd;
 extern struct WalSnd *SyncRepGetSynchronousStandby(void);
 
-extern bool check_synchronous_standby_names(char **newval, void **extra, GucSource source);
+extern bool check_standby_names(char **newval, void **extra, GucSource source);
 extern void assign_synchronous_commit(int newval, void *extra);
 
 #endif   /* _SYNCREP_H */
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 6eacb09..7f83934 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -80,6 +80,13 @@ typedef struct
 	TimeLineID	receivedTLI;
 
 	/*
+	 * causalReadsLease is the time until which the primary has authorized
+	 * this standby to consider itself available for causal_reads mode, or 0
+	 * for not authorized.
+	 */
+	TimestampTz causalReadsLease;
+
+	/*
 	 * latestChunkStart is the starting byte position of the current "batch"
 	 * of received WAL.  It's actually the same as the previous value of
 	 * receivedUpto before the last flush to disk.  Startup process can use
@@ -162,5 +169,8 @@ extern void RequestXLogStreaming(TimeLineID tli, XLogRecPtr recptr,
 extern XLogRecPtr GetWalRcvWriteRecPtr(XLogRecPtr *latestChunkStart, TimeLineID *receiveTLI);
 extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
+extern void WalRcvWakeup(void);
+
+extern bool WalRcvCausalReadsAvailable(void);
 
 #endif   /* _WALRECEIVER_H */
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 7794aa5..81a2776 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -27,6 +27,13 @@ typedef enum WalSndState
 	WALSNDSTATE_STREAMING
 } WalSndState;
 
+typedef enum WalSndCausalReadsState
+{
+	WALSNDCRSTATE_UNAVAILABLE = 0,
+	WALSNDCRSTATE_JOINING,
+	WALSNDCRSTATE_AVAILABLE
+} WalSndCausalReadsState;
+
 /*
  * Each walsender has a WalSnd struct in shared memory.
  */
@@ -34,6 +41,7 @@ typedef struct WalSnd
 {
 	pid_t		pid;			/* this walsender's process id, or 0 */
 	WalSndState state;			/* this walsender's state */
+	WalSndCausalReadsState causal_reads_state; /* the walsender's causal reads state */
 	XLogRecPtr	sentPtr;		/* WAL has been sent up to this point */
 	bool		needreload;		/* does currently-open file need to be
 								 * reloaded? */
@@ -46,6 +54,7 @@ typedef struct WalSnd
 	XLogRecPtr	write;
 	XLogRecPtr	flush;
 	XLogRecPtr	apply;
+	int			applyLagMs;
 
 	/* Protects shared variables shown above. */
 	slock_t		mutex;
@@ -88,6 +97,12 @@ typedef struct
 	 */
 	bool		sync_standbys_defined;
 
+	/*
+	 * Until when must commits in causal_reads stall?  This is used to wait
+	 * for causal reads leases to expire.
+	 */
+	TimestampTz	stall_causal_reads_until;
+
 	WalSnd		walsnds[FLEXIBLE_ARRAY_MEMBER];
 } WalSndCtlData;
 
diff --git a/src/include/utils/timestamp.h b/src/include/utils/timestamp.h
index fbead3a..297e151 100644
--- a/src/include/utils/timestamp.h
+++ b/src/include/utils/timestamp.h
@@ -227,9 +227,11 @@ extern bool TimestampDifferenceExceeds(TimestampTz start_time,
 #ifndef HAVE_INT64_TIMESTAMP
 extern int64 GetCurrentIntegerTimestamp(void);
 extern TimestampTz IntegerTimestampToTimestampTz(int64 timestamp);
+extern int64 TimestampTzToIntegerTimestamp(TimestampTz timestamp);
 #else
 #define GetCurrentIntegerTimestamp()	GetCurrentTimestamp()
 #define IntegerTimestampToTimestampTz(timestamp) (timestamp)
+#define TimestampTzToIntegerTimestamp(timestamp) (timestamp)
 #endif
 
 extern TimestampTz time_t_to_timestamptz(pg_time_t tm);
#36Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Thomas Munro (#35)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

Hi Thomas,

On 2016/02/29 15:20, Thomas Munro wrote:

Thanks for looking at the patch! Here is a new version with the
following changes:

1. Some draft user documentation has been added, as requested.

Just to clarify, in:

+         servers.  A transaction that is run with
<varname>causal_reads</> set
+         to <literal>on</> is guaranteed either to see the effects of all
+         completed transactions run on the primary with the setting on, or to
+         receive an error "standby is not available for causal reads".

"A transaction that is run" means "A transaction that is run on a
standby", right? By the way, is there some discussion in our existing
documentation to refer to about causal consistency in single node case? I
don't know maybe that will help ease into the new feature. Grepping the
existing source tree doesn't reveal the term "causal", so maybe even a
single line in the patch mentioning "single node operation trivially
implies (or does it?) causal consistency" would help. Thoughts?

Thanks,
Amit

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#37Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Amit Langote (#36)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Mon, Feb 29, 2016 at 9:05 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

Hi Thomas,

On 2016/02/29 15:20, Thomas Munro wrote:

Thanks for looking at the patch! Here is a new version with the
following changes:

1. Some draft user documentation has been added, as requested.

Just to clarify, in:

+         servers.  A transaction that is run with
<varname>causal_reads</> set
+         to <literal>on</> is guaranteed either to see the effects of all
+         completed transactions run on the primary with the setting on, or to
+         receive an error "standby is not available for causal reads".

"A transaction that is run" means "A transaction that is run on a
standby", right?

Well, it could be any server, standby or primary. Of course standbys
are the interesting case since it it was already true that if you run
two sequential transactions run on the primary, the second can see the
effect of the first, but I like the idea of a general rule that
applies anywhere, allowing you not to care which server it is.

By the way, is there some discussion in our existing
documentation to refer to about causal consistency in single node case? I
don't know maybe that will help ease into the new feature. Grepping the
existing source tree doesn't reveal the term "causal", so maybe even a
single line in the patch mentioning "single node operation trivially
implies (or does it?) causal consistency" would help. Thoughts?

Hmm. Where should such a thing go? I probably haven't introduced the
term well enough. I thought for a moment about putting something
here:

http://www.postgresql.org/docs/devel/static/sql-commit.html

"All changes made by the transaction become visible to others ..." --
which others? But I backed out, that succinct account of COMMIT is 20
years old, and in any case visibility is tied to committing, not
specifically to the COMMIT command. But perhaps this patch really
should include something there that refers back to the causal reads
section.

--
Thomas Munro
http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#38Michael Paquier
michael.paquier@gmail.com
In reply to: Thomas Munro (#37)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Mon, Feb 29, 2016 at 6:05 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

"All changes made by the transaction become visible to others ..." --
which others? But I backed out, that succinct account of COMMIT is 20
years old, and in any case visibility is tied to committing, not
specifically to the COMMIT command. But perhaps this patch really
should include something there that refers back to the causal reads
section.

Luckily enough, read uncommitted behaves like read-committed in PG,
making this true :)
--
Michael

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#39Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Thomas Munro (#37)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

Hi,

On 2016/02/29 18:05, Thomas Munro wrote:

On Mon, Feb 29, 2016 at 9:05 PM, Amit Langote wrote:

+         servers.  A transaction that is run with
<varname>causal_reads</> set
+         to <literal>on</> is guaranteed either to see the effects of all
+         completed transactions run on the primary with the setting on, or to
+         receive an error "standby is not available for causal reads".

"A transaction that is run" means "A transaction that is run on a
standby", right?

Well, it could be any server, standby or primary. Of course standbys
are the interesting case since it it was already true that if you run
two sequential transactions run on the primary, the second can see the
effect of the first, but I like the idea of a general rule that
applies anywhere, allowing you not to care which server it is.

I meant actually in context of that sentence only.

By the way, is there some discussion in our existing
documentation to refer to about causal consistency in single node case? I
don't know maybe that will help ease into the new feature. Grepping the
existing source tree doesn't reveal the term "causal", so maybe even a
single line in the patch mentioning "single node operation trivially
implies (or does it?) causal consistency" would help. Thoughts?

Hmm. Where should such a thing go? I probably haven't introduced the
term well enough. I thought for a moment about putting something
here:

http://www.postgresql.org/docs/devel/static/sql-commit.html

"All changes made by the transaction become visible to others ..." --
which others? But I backed out, that succinct account of COMMIT is 20
years old, and in any case visibility is tied to committing, not
specifically to the COMMIT command. But perhaps this patch really
should include something there that refers back to the causal reads
section.

I see. I agree this is not exactly material for the COMMIT page. Perhaps
somewhere under "Chapter 13. Concurrency Control" with cross-reference
to/from "25.5. Hot Standby". Might be interesting to hear from others as
well.

Thanks,
Amit

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#40Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Amit Langote (#39)
1 attachment(s)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Tue, Mar 1, 2016 at 2:46 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

Hi,

On 2016/02/29 18:05, Thomas Munro wrote:

On Mon, Feb 29, 2016 at 9:05 PM, Amit Langote wrote:

+         servers.  A transaction that is run with
<varname>causal_reads</> set
+         to <literal>on</> is guaranteed either to see the effects of all
+         completed transactions run on the primary with the setting on, or to
+         receive an error "standby is not available for causal reads".

"A transaction that is run" means "A transaction that is run on a
standby", right?

Well, it could be any server, standby or primary. Of course standbys
are the interesting case since it it was already true that if you run
two sequential transactions run on the primary, the second can see the
effect of the first, but I like the idea of a general rule that
applies anywhere, allowing you not to care which server it is.

I meant actually in context of that sentence only.

Ok, here's a new version that includes that change, fixes a conflict
with recent commit 10b48522 and removes an accidental duplicate copy
of the README file.

--
Thomas Munro
http://www.enterprisedb.com

Attachments:

causal-reads-v8.patchapplication/octet-stream; name=causal-reads-v8.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index a09ceb2..9dbd5cd 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -2664,6 +2664,35 @@ include_dir 'conf.d'
      across the cluster without problems if that is required.
     </para>
 
+    <sect2 id="runtime-config-replication-all">
+     <title>All Servers</title>
+     <para>
+      These parameters can be set on the primary or any standby.
+     </para>
+     <variablelist>
+      <varlistentry id="guc-causal-reads" xreflabel="causal_reads">
+       <term><varname>causal_reads</varname> (<type>boolean</type>)
+       <indexterm>
+        <primary><varname>causal_reads</> configuration parameter</primary>
+       </indexterm>
+       </term>
+       <listitem>
+        <para>
+         Enables causal consistency between transactions run on different
+         servers.  A transaction that is run on a standby
+         with <varname>causal_reads</> set to <literal>on</> is guaranteed
+         either to see the effects of all completed transactions run on the
+         primary with the setting on, or to receive an error "standby is not
+         available for causal reads".  Note that both transactions involved in
+         a causal dependency (a write on the primary followed by a read on any
+         server which must see the write) must be run with the setting on.
+         See <xref linkend="causal-reads"> for more details.
+        </para>
+       </listitem>
+      </varlistentry>
+     </variablelist>     
+    </sect2>
+
     <sect2 id="runtime-config-replication-sender">
      <title>Sending Server(s)</title>
 
@@ -2895,6 +2924,48 @@ include_dir 'conf.d'
       </listitem>
      </varlistentry>
 
+     <varlistentry>
+      <term><varname>causal_reads_timeout</varname> (<type>integer</type>)
+       <indexterm>
+        <primary><varname>causal_reads_timeout</> configuration parameter</primary>
+       </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies the maximum replay lag the primary will tolerate from a
+        standby before dropping it from the set of standbys available for
+        causal reads.
+       </para>
+       <para>
+        This setting is also used to control the <firstterm>leases</> used to
+        maintain the causal reads guarantee.  It must be set to a value which
+        is at least 4 times the maximum possible difference in system clocks
+        between the primary and standby servers, as described
+        in <xref linkend="causal-reads">.
+       </para>
+      </listitem>
+     </varlistentry>
+
+     <varlistentry id="guc-causal-reads-standby-names" xreflabel="causal-reads-standby-names">
+      <term><varname>causal_reads_standby_names</varname> (<type>string</type>)
+      <indexterm>
+       <primary><varname>causal_reads_standby_names</> configuration parameter</primary>
+      </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies a comma-separated list of standby names that can support
+        <firstterm>causal reads</>, as described in
+        <xref linkend="causal-reads">.  Follows the same convention
+        as <link linkend="guc-synchronous-standby-names"><literal>synchronous_standby_name</></>.
+        The default is <literal>*</>, matching all standbys.
+       </para>
+       <para>
+        This setting has no effect if <varname>causal_reads_timeout</> is not set.
+       </para>
+      </listitem>
+     </varlistentry>
+
      </variablelist>
     </sect2>
 
diff --git a/doc/src/sgml/high-availability.sgml b/doc/src/sgml/high-availability.sgml
index 6cb690c..a338fd8 100644
--- a/doc/src/sgml/high-availability.sgml
+++ b/doc/src/sgml/high-availability.sgml
@@ -1081,6 +1081,9 @@ primary_slot_name = 'node_a_slot'
     WAL record is then sent to the standby. The standby sends reply
     messages each time a new batch of WAL data is written to disk, unless
     <varname>wal_receiver_status_interval</> is set to zero on the standby.
+    In the case that <varname>synchronous_commit</> is set to
+    <literal>remote_apply</>, the standby sends reply messages when the commit
+    record is replayed, making the transaction visible.
     If the standby is the first matching standby, as specified in
     <varname>synchronous_standby_names</> on the primary, the reply
     messages from that standby will be used to wake users waiting for
@@ -1107,6 +1110,16 @@ primary_slot_name = 'node_a_slot'
    </para>
 
    <para>
+    Setting <varname>synchronous_commit</> to <literal>remote_apply</> will
+    cause each commit to wait until the current synchronous standby reports
+    that it has replayed the transaction, making it visible to user queries.
+    In simple cases, this allows for load balancing with causal consistency
+    on a single hot standby.  (See also
+    <xref linkend="causal-reads"> which deals with multiple standbys and
+    standby failure.)
+   </para>
+
+   <para>
     Users will stop waiting if a fast shutdown is requested.  However, as
     when using asynchronous replication, the server will not fully
     shutdown until all outstanding WAL records are transferred to the currently
@@ -1160,8 +1173,9 @@ primary_slot_name = 'node_a_slot'
     <title>Planning for High Availability</title>
 
    <para>
-    Commits made when <varname>synchronous_commit</> is set to <literal>on</>
-    or <literal>remote_write</> will wait until the synchronous standby responds. The response
+    Commits made when <varname>synchronous_commit</> is set to <literal>on</>,
+    <literal>remote_write</> or <literal>remote_apply</> will wait until the
+    synchronous standby responds. The response
     may never occur if the last, or only, standby should crash.
    </para>
 
@@ -1221,6 +1235,119 @@ primary_slot_name = 'node_a_slot'
    </sect3>
   </sect2>
 
+  <sect2 id="causal-reads">
+   <title>Causal reads</title>
+   <indexterm>
+    <primary>causal reads</primary>
+    <secondary>in standby</secondary>
+   </indexterm>
+
+   <para>
+    The causal reads feature allows read-only queries to run on hot standby
+    servers without exposing stale data to the client, providing a form of
+    causal consistency.  Transactions can run on any standby with the
+    following guarantee about the visibility of preceding transactions: If you
+    set <varname>causal_reads</> to <literal>on</> in any pair of consecutive
+    transactions tx1, tx2 where tx2 begins after tx1 successfully returns,
+    then tx2 will either see tx1 or fail with a new error "standby is not
+    available for causal reads", no matter which server it runs on.  Although
+    the guarantee is expressed in terms of two individual transactions, the
+    GUC can also be set at session, role or system level to make the guarantee
+    generally, allowing for load balancing of applications that were not
+    designed with load balancing in mind.
+   </para>
+
+   <para>
+    In order to enable the feature, <varname>causal_reads_timeout</> must be
+    set to a non-zero value on the primary server.  The
+    GUC <varname>causal_reads_standby_names</> can be used to limit the set of
+    standbys that can join the dynamic set of causal reads standbys by
+    providing a comma-separated list of application names.  By default, all
+    standbys are candidates, if the feature is enabled.
+   </para>
+
+   <para>
+    The current set of servers that the primary considers to be available for
+    causal reads can be seen in
+    the <link linkend="monitoring-stats-views-table"> <literal>pg_stat_replication</></>
+    view.  Administrators, applications and load balancing middleware can use
+    this view to discover standbys that can currently handle causal reads
+    transactions without raising the error.  Since that information is only an
+    instantantaneous snapshot, clients should still be prepared for the error
+    to be raised at any time, and consider redirecting transactions to another
+    standby.
+   </para>
+
+   <para>
+    The advantages of the causal reads feature over simply
+    setting <varname>synchronous_commit</> to <literal>remote_apply</> are:
+    <orderedlist>
+      <listitem>
+       <para>
+        It allows the primary to wait for multiple standbys to replay
+        transactions.
+       </para>
+      </listitem>
+      <listitem>
+       <para>
+        It places a configurable limit on how much replay lag (and therefore
+        delay at commit time) the primary tolerates from standbys before it
+        drops them from the dynamic set of standbys it waits for.
+       </para>   
+      </listitem>
+      <listitem>
+       <para>
+        It upholds the causal reads guarantee during the transitions that
+        occur when new standbys are added or removed from the set of standbys,
+        including scenarios where contact has been lost between the primary
+        and standbys but the standby is still alive and running client
+        queries.
+       </para>
+      </listitem>
+    </orderedlist>
+   </para>
+
+   <para>
+    The protocol used to uphold the guarantee even in the case of network
+    failure depends on the system clocks of the primary and standby servers
+    being synchronized, with an allowance for a difference up to one quarter
+    of <varname>causal_reads_timeout</>.  For example,
+    if <varname>causal_reads_timeout</> is set to <literal>4s</>, then the
+    clocks must not be further than 1 second apart for the guarantee to be
+    upheld reliably during transitions.  The ubiquity of the Network Time
+    Protocol (NTP) on modern operating systems and availability of high
+    quality time servers makes it possible to choose a tolerance significantly
+    higher than the maximum expected clock difference.  An effort is
+    nevertheless made to detect and report misconfigured and faulty systems
+    with clock differences greater than the configured tolerance.
+   </para>
+
+   <note>
+    <para>
+     Current hardware clocks, NTP implementations and public time servers are
+     unlikely to allow the system clocks to differ more than tens or hundreds
+     of milliseconds, and systems synchronized with dedicated local time
+     servers may be considerably more accurate, but you should only consider
+     setting <varname>causal_reads_timeout</> below 4 seconds (allowing up to
+     1 second of clock difference) after researching your time synchronization
+     infrastructure thoroughly.
+    </para>  
+   </note>
+
+   <note>
+    <para>
+      While similar to synchronous replication in the sense that both involve
+      the primary server waiting for responses from standby servers, the
+      causal reads feature is not concerned with avoiding data loss.  A
+      primary configured for causal reads will drop all standbys that stop
+      responding or replay too slowly from the dynamic set that it waits for,
+      so you should consider configuring both synchronous replication and
+      causal reads if you need data loss avoidance guarantees and causal
+      consistency guarantees for load balancing.
+    </para>
+   </note>
+  </sect2>
+
   <sect2 id="continuous-archiving-in-standby">
    <title>Continuous archiving in standby</title>
 
@@ -1569,7 +1696,16 @@ if (!triggered)
     so there will be a measurable delay between primary and standby. Running the
     same query nearly simultaneously on both primary and standby might therefore
     return differing results. We say that data on the standby is
-    <firstterm>eventually consistent</firstterm> with the primary.  Once the
+    <firstterm>eventually consistent</firstterm> with the primary by default.
+    The data visible to a transaction running on a standby can be
+    made <firstterm>causally consistent</> with respect to a transaction that
+    has completed on the primary by setting <varname>causal_reads</>
+    to <literal>on</> in both transactions.  For more details,
+    see <xref linkend="causal-reads">.
+   </para>
+
+   <para>
+    Once the
     commit record for a transaction is replayed on the standby, the changes
     made by that transaction will be visible to any new snapshots taken on
     the standby.  Snapshots may be taken at the start of each query or at the
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 85459d0..5a87f37 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -820,6 +820,12 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       standby server</entry>
     </row>
     <row>
+     <entry><structfield>replay_lag</></entry>
+     <entry><type>interval</></entry>
+     <entry>Estimated time taken for recent WAL records to be replayed on this
+      standby server</entry>
+    </row>
+    <row>
      <entry><structfield>sync_priority</></entry>
      <entry><type>integer</></entry>
      <entry>Priority of this standby server for being chosen as the
@@ -830,6 +836,17 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
      <entry><type>text</></entry>
      <entry>Synchronous state of this standby server</entry>
     </row>
+    <row>
+     <entry><structfield>causal_reads_state</></entry>
+     <entry><type>text</></entry>
+     <entry>Causal reads state of this standby server.  This field will be
+     non-null only if <varname>cause_reads_timeout</> is set.  If a standby is
+     in <literal>available</> state, then it can currently serve causal reads
+     queries.  If it is not replaying fast enough or not responding to
+     keepalive messages, it will be in <literal>unavailable</> state, and if
+     it is currently transitioning to availability it will be
+     in <literal>joining</> state for a short time.</entry>
+    </row>
    </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index b0d5440..8bfc510 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,10 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
+	{
+		CausalReadsWaitForLSN(XactLastRecEnd);
 		SyncRepWaitForLSN(XactLastRecEnd);
+	}
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5117,6 +5120,13 @@ XactLogCommitRecord(TimestampTz commit_time,
 		xl_xinfo.xinfo |= XACT_COMPLETION_FORCE_SYNC_COMMIT;
 
 	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this commit is applied.
+	 */
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY || causal_reads)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
+	/*
 	 * Relcache invalidations requires information about the current database
 	 * and so does logical decoding.
 	 */
@@ -5452,6 +5462,19 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 	if (XactCompletionForceSyncCommit(parsed->xinfo))
 		XLogFlush(lsn);
 
+	/*
+	 * Record the primary's timestamp for the commit record, so it can be used
+	 * for tracking replay lag.
+	 */
+	SetXLogReplayTimestamp(parsed->xact_time);
+
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit or causal reads), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 /*
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 94b79ac..b7348ab 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -81,6 +81,8 @@ extern uint32 bootstrap_data_checksum_version;
 #define PROMOTE_SIGNAL_FILE		"promote"
 #define FALLBACK_PROMOTE_SIGNAL_FILE "fallback_promote"
 
+/* Size of the circular buffer of timestamped LSNs. */
+#define MAX_TIMESTAMPED_LSNS 8192
 
 /* User-settable parameters */
 int			max_wal_size = 64;	/* 1 GB */
@@ -346,6 +348,12 @@ static XLogRecPtr RedoRecPtr;
 static bool doPageWrites;
 
 /*
+ * doRequestWalReceiverReply is used by recovery code to ask the main recovery
+ * loop to trigger a walreceiver reply.
+ */
+static bool doRequestWalReceiverReply;
+
+/*
  * RedoStartLSN points to the checkpoint's REDO location which is specified
  * in a backup label file, backup history file or control file. In standby
  * mode, XLOG streaming usually starts from the position where an invalid
@@ -357,6 +365,13 @@ static bool doPageWrites;
  */
 static XLogRecPtr RedoStartLSN = InvalidXLogRecPtr;
 
+/*
+ * LastReplayedTimestamp can be set by redo handlers when they apply a record
+ * that carries a timestamp, by calling SetXLogReplayedTimestamp.  The xlog
+ * apply loop can then update the value in shared memory.
+ */
+static TimestampTz LastReplayedTimestamp = 0;
+
 /*----------
  * Shared-memory data structures for XLOG control
  *
@@ -631,6 +646,21 @@ typedef struct XLogCtlData
 	/* current effective recovery target timeline */
 	TimeLineID	RecoveryTargetTLI;
 
+	/* timestamp from the most recently applied record that carried a timestamp. */
+	TimestampTz lastReplayedTimestamp;
+
+	/*
+	 * We maintain a circular buffer of LSNs and associated timestamps.
+	 * Walreceiver writes into it using information from timestamps, and the
+	 * startup recovery process reads from it and notifies walreceiver when
+	 * LSNs are replayed so that the timestamps can be fed back to the
+	 * upstream server, to track lag.
+	 */
+	Index		timestampedLsnRead;
+	Index		timestampedLsnWrite;
+	XLogRecPtr	timestampedLsn[MAX_TIMESTAMPED_LSNS];
+	TimestampTz	timestampedLsnTime[MAX_TIMESTAMPED_LSNS];
+
 	/*
 	 * timestamp of when we started replaying the current chunk of WAL data,
 	 * only relevant for replication or archive recovery
@@ -6897,14 +6927,58 @@ StartupXLOG(void)
 				error_context_stack = errcallback.previous;
 
 				/*
-				 * Update lastReplayedEndRecPtr after this record has been
-				 * successfully replayed.
+				 * Update lastReplayedEndRecPtr and lastReplayedTimestamp
+				 * after this record has been successfully replayed.
 				 */
 				SpinLockAcquire(&XLogCtl->info_lck);
 				XLogCtl->lastReplayedEndRecPtr = EndRecPtr;
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
+				if (LastReplayedTimestamp != 0)
+				{
+					/* If replaying a record produced a timestamp, use that. */
+					XLogCtl->lastReplayedTimestamp = LastReplayedTimestamp;
+					LastReplayedTimestamp = 0;
+				}
+				else
+				{
+					/*
+					 * If we have applied LSNs associated with timestamps
+					 * received by walreceiver, then use the recorded
+					 * timestamp.  We consume from the read end of the
+					 * circular buffer.
+					 */
+					while (XLogCtl->timestampedLsnRead !=
+						   XLogCtl->timestampedLsnWrite &&
+						   XLogCtl->timestampedLsn[XLogCtl->timestampedLsnRead]
+						   <= EndRecPtr)
+					{
+						if (XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead] >
+							XLogCtl->lastReplayedTimestamp)
+						{
+							XLogCtl->lastReplayedTimestamp =
+								XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead];
+							doRequestWalReceiverReply = true;
+						}
+						XLogCtl->timestampedLsnRead =
+							(XLogCtl->timestampedLsnRead + 1) % MAX_TIMESTAMPED_LSNS;
+					}
+				}
 				SpinLockRelease(&XLogCtl->info_lck);
 
+				/*
+				 * If rm_redo reported that it applied a commit record that
+				 * the master is waiting for by calling
+				 * XLogRequestWalReceiverReply, or we encountered a WAL
+				 * location that was associated with a timestamp above, then
+				 * we wake up the receiver so that it notices the updated
+				 * lastReplayedEndRecPtr and sends a reply to the master.
+				 */
+				if (doRequestWalReceiverReply)
+				{
+					doRequestWalReceiverReply = false;
+					WalRcvWakeup();
+				}
+
 				/* Remember this record as the last-applied one */
 				LastRec = ReadRecPtr;
 
@@ -11626,3 +11700,103 @@ SetWalWriterSleeping(bool sleeping)
 	XLogCtl->WalWriterSleeping = sleeping;
 	SpinLockRelease(&XLogCtl->info_lck);
 }
+
+/*
+ * Called by redo code to indicate that the xlog replay loop should wake up
+ * the walreceiver process so that a reply can be sent to the primary.
+ */
+void
+XLogRequestWalReceiverReply(void)
+{
+	doRequestWalReceiverReply = true;
+}
+
+/*
+ * Merge timestamps from keepalive messages with the timestamps from WAL
+ * records, so that we can track lag while idle or while replaying large
+ * amounts of WAL without commit records.  In the former case there is no lag,
+ * and in the latter case we will remember a timestamp that goes with an
+ * arbitrary LSN, and wait for that LSN to be replayed before using the
+ * timestamp.
+ *
+ * This is called by walreceiver on standby servers when keepalive messages
+ * arrive.
+ */
+void
+SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn)
+{
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn == XLogCtl->lastReplayedEndRecPtr)
+	{
+		/*
+		 * That is the last replayed LSN: we are fully replayed, so we can
+		 * update the replay timestamp immediately.
+		 */
+		XLogCtl->lastReplayedTimestamp = timestamp;
+	}
+	else
+	{
+		/*
+		 * There is WAL still to be applied.  We will associate the timestamp
+		 * with this WAL position and wait for it to be replayed.  We add it
+		 * at the 'write' end of the circular buffer of LSN/timestamp
+		 * mappings, which the replay loop will eventually read.
+		 */
+		Index w = XLogCtl->timestampedLsnWrite;
+		Index r = XLogCtl->timestampedLsnRead;
+
+		XLogCtl->timestampedLsn[w] = lsn;
+		XLogCtl->timestampedLsnTime[w] = timestamp;
+
+		/* Advance the write point. */
+		w = (w + 1) % MAX_TIMESTAMPED_LSNS;
+		XLogCtl->timestampedLsnWrite = w;
+		if (w == r)
+		{
+			/*
+			 * The buffer is full.  Advance the read point (throwing away
+			 * oldest values; we will begin to overestimate replay lag, until
+			 * lag decreases to a size our buffer can manage, or the next
+			 * commit record is replayed).
+			 */
+			r = (r + 1) % MAX_TIMESTAMPED_LSNS;
+			XLogCtl->timestampedLsnRead = r;
+		}
+	}
+	SpinLockRelease(&XLogCtl->info_lck);
+}
+
+/*
+ * Set the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary.  This can be called by redo handlers that have
+ * an appropriate timestamp (currently only commit records).  Updating the
+ * shared memory value is deferred until after the redo handler returns.
+ */
+void
+SetXLogReplayTimestamp(TimestampTz timestamp)
+{
+	LastReplayedTimestamp = timestamp;
+}
+
+/*
+ * Get the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the master, and also the most recently applied LSN.  (Note
+ * that the timestamp and the LSN don't necessarily relate to the same
+ * record.)
+ *
+ * This is similar to GetLatestXTime, except that it is not only advanced by
+ * commit records (see SetXLogReplayTimestampAtLsn).
+ */
+TimestampTz
+GetXLogReplayTimestamp(XLogRecPtr *lsn)
+{
+	TimestampTz result;
+
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn)
+		*lsn = XLogCtl->lastReplayedEndRecPtr;
+	result = XLogCtl->lastReplayedTimestamp;
+	SpinLockRelease(&XLogCtl->info_lck);
+
+	return result;
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index abf9a70..b80206e 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -661,8 +661,10 @@ CREATE VIEW pg_stat_replication AS
             W.write_location,
             W.flush_location,
             W.replay_location,
+            W.replay_lag,
             W.sync_priority,
-            W.sync_state
+            W.sync_state,
+            W.causal_reads_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
             pg_stat_get_wal_senders() AS W
     WHERE S.usesysid = U.oid AND
diff --git a/src/backend/replication/README.causal_reads b/src/backend/replication/README.causal_reads
new file mode 100644
index 0000000..35b29e9
--- /dev/null
+++ b/src/backend/replication/README.causal_reads
@@ -0,0 +1,193 @@
+The causal reads guarantee says: If you run any two consecutive
+transactions tx1, tx2 where tx1 completes before tx2 begins, with
+causal_reads set to "on" in both transactions, tx2 will see tx1 or
+raise an error to complain that it can't guarantee causal consistency,
+no matter which servers (primary or any standby) you run each
+transaction on.
+
+When both transactions run on the primary, the guarantee is trivially
+upheld.
+
+To deal with read-only physical streaming standbys, the primary keeps
+track of a set of standbys that it considers to be currently
+"available" for causal reads, and sends a stream of "leases" to those
+standbys granting them the right to handle causal reads transactions
+for a short time without any further communication with the primary.
+
+In general, the primary provides the guarantee by waiting for all of
+the "available" standbys to report that they have applied a
+transaction.  However, the set of available standbys is dynamic, and
+things get more complicated during state transitions.  There are two
+types of transitions to consider:
+
+1.  unavailable->joining->available
+
+Standbys start out as "unavailable".  If a standby is unavailable and
+is applying fast enough and matches causal_reads_standby_names, the
+primary transitions it to "available", but first it sets it to
+"joining" until it is sure that any transaction committed while it was
+unavailable has definitely been applied on the standby.  This closes a
+race that would otherwise exist if we moved directly to available
+state: tx1 might not wait for a given standby because it's
+unavailable, then a lease might be granted, and then tx2 might run a
+causal reads transaction without error but see stale data.  The
+joining state acts as an airlock: while in joining state, the primary
+waits for that standby to replay causal reads transactions in
+anticipation of the move to available, but it doesn't progress to
+available state and grant a lease to the standby until everything
+preceding joining state has also been applied.
+
+2.  available->unavailable
+
+If a standby is not applying fast enough or not responding to
+keepalive messages, then the primary kicks that standby out of the
+dynamic set of available standbys, that is, marks it as "unavailable".
+In order to make sure that the standby has started rejecting causal
+reads transactions, it needs to revoke the lease it most recently
+granted.  It does that by waiting for the lease to expire before
+allowing any causal reads commits to return.  (In future there could
+be a fast-path revocation message which waits for a serial-numbered
+acknowledgement to reduce waiting in the case where the standby is
+lagging but still reachable and responding).
+
+The rest of this document illustrates how clock skew affects the
+available->unavailable transition.
+
+The following 4 variables are derived from a single GUC, and these
+values will be used in the following illustrations:
+
+causal_reads_timeout = 4s
+lease_time           = 4s (= causal_reads_timeout)
+keepalive_time       = 2s (= lease_time / 2)
+max_clock_skew       = 1s (= lease_time / 4)
+
+Every keepalive_time, the primary transmits a lease that expires at
+local_clock_time + lease_time - max_clock_skew, shown in the following
+diagram as 't' for transmission time and '|' for expiry time.  If
+contact is lost with a standby, the primary will wait until sent_time
++ lease_time for the most recently granted lease to expire, shown on
+the following diagram 'x', to be sure that the standby's clock has
+reached the expiry time even if its clock differs by up to
+max_clock_skew.  In other words, the primary tells the standby that
+the expiry time is at one time, but it trusts that the standby will
+surely agree if it gives it some extra time.  The extra time is
+max_clock_skew.  If the clocks differ by more than max_clock_skew, all
+bets are off (but see below for attempt to detect obvious cases).
+
+0     1     2     3     4     5     6     7     8     9
+t-----------------|-----x
+            t-----------------|-----x
+                        t-----------------|-----x
+                                    t-----------------|...
+                                                t------...
+
+A standby whose clock is 2 seconds ahead of the primary's clock
+perceives gaps in the stream of leases, and will reject causal_reads
+transactions in those intervals.  The causal reads guarantee is
+upheld, but spurious errors are raised between leases, as a
+consequence of the clock skew being greater than max_clock_skew.  In
+the following diagram 'r' shows reception time, and the timeline along
+the top shows the standby's local clock time.
+
+2     3     4     5     6     7     8     9    10    11
+r-----|
+            r-----|
+                        r-----|
+                                    r-----|
+                                                r-----|
+
+If there were no network latency, a standby whose clock is exactly 1
+second ahead of the primary's clock would perceive the stream of
+leases as being replaced just in time, so there is no gap.  Since in
+reality the time of receipt is some time after the time of
+transmission due to network latency, if the standby's clock is exactly
+1 second behind, then there will be small network-latency-sized gaps
+before the next lease arrives, but still no correctness problem with
+respect to the causal reads guarantee.
+
+1     2     3     4     5     6     7     8     9    10
+r-----------|
+            r-----------|
+                        r-----------|
+                                    r-----------|
+                                                r------...
+
+A standby whose clock is perfectly in sync with the primary's
+perceives the stream of leases overlapping (this matches the primary's
+perception of the leases it sent):
+
+0     1     2     3     4     5     6     7     8     9
+r-----------------|
+            r-----------------|
+                        r-----------------|
+                                    r-----------------|
+                                                r------...
+
+A standby whose clock is exactly 1 second behind the primary's
+perceives the stream of leases as overlapping even more, but the time
+of expiry as judged by the standby is no later than the time the
+primary will wait for if required ('x').  That is, if contact is lost
+with the standby, the primary can still reliably hold up causal reads
+commits until the standby has started raising the error in
+causal_reads transactions.
+
+-1    0     1     2     3     4     5     6     7     8
+r-----------------------|
+            r-----------------------|
+                        r-----------------------|
+                                    r------------------...
+                                                r------...
+
+
+A standby whose clock is 2 seconds behind the primary's would perceive
+the stream of leases overlapping even more, and the primary would no
+longer be able to wait for a lease to expire if it wanted to revoke
+it.  But because the expiry time is after local_clock_time +
+lease_time, the standby can immediately see that its own clock must be
+more than 1 second behind the primary's, so it ignores the lease and
+logs a clock skew warning.  In the following diagram a lease expiry
+time that is obviously generated by a primary with a clock set too far
+in the future compared to the local clock is shown with a '!'.
+
+-2    -1    0     1     2     3     4     5     6     7
+r-----------------------------!
+            r-----------------------------!
+                        r-----------------------------!
+                                    r------------------...
+                                                r------...
+
+A danger window exists when the standby's clock is more than
+max_clock_skew behind the primary's clock, but not more than
+max_clock_skew + network latency time behind.  If the clock difference
+is in that range, then the algorithm presented above which is based on
+time of receipt cannot detect that the local clock is too far behind.
+The consequence of this problem could be as follows:
+
+1.  The standby loses contact with the primary due to a network fault.
+
+2.  The primary decides to drop the standby from the set of available
+    causal reads standbys due to lack of keepalive responses or
+    excessive lag, which necessitates holding up commits of causal
+    reads transactions until the most recently sent lease expires, in
+    the belief that the standby will definitely have started raising
+    the 'causal reads unavailable' error in causal reads transactions
+    by that time, if it is still alive and servicing requests.
+
+3.  The standby still has clients connected and running queries.
+
+4.  Due to clock skew in the problematic range, in the standby's
+    opinion the lease lasts slightly longer than the primary waits.
+
+5.  For a short window at most the duration of the network latency
+    time, clients running causal reads transactions are allowed to see
+    potentially stale data.
+
+For this reason we say that the causal reads guarantee only holds as
+long as the absolute difference between the system clocks of the
+machines is no more than max_clock_skew.  The theory is that NTP makes
+it possible to reason about the maximum possible clock difference
+between machines and choose a value that allows for a much larger
+difference.  However, we do make a best effort attempt to detect
+misconfigured systems as described above, to catch the case of servers
+not running ntp a correctly configured ntp daemon, or with a clock so
+far out of whack that ntp refuses to fix it.
\ No newline at end of file
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 92faf4e..3b6ab3f 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -57,6 +57,11 @@
 #include "utils/builtins.h"
 #include "utils/ps_status.h"
 
+/* GUC variables */
+int causal_reads_timeout;
+bool causal_reads;
+char *causal_reads_standby_names;
+
 /* User-settable parameters for sync rep */
 char	   *SyncRepStandbyNames;
 
@@ -69,7 +74,7 @@ static int	SyncRepWaitMode = SYNC_REP_NO_WAIT;
 
 static void SyncRepQueueInsert(int mode);
 static void SyncRepCancelWait(void);
-static int	SyncRepWakeQueue(bool all, int mode);
+static int	SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn);
 
 static int	SyncRepGetStandbyPriority(void);
 
@@ -83,6 +88,255 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * ===========================================================
  */
 
+static bool
+SyncRepCheckEarlyExit(void)
+{
+	/*
+	 * If a wait for synchronous replication is pending, we can neither
+	 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
+	 * lead the client to believe that the transaction aborted, which
+	 * is not true: it's already committed locally. The former is no good
+	 * either: the client has requested synchronous replication, and is
+	 * entitled to assume that an acknowledged commit is also replicated,
+	 * which might not be true. So in this case we issue a WARNING (which
+	 * some clients may be able to interpret) and shut off further output.
+	 * We do NOT reset ProcDiePending, so that the process will die after
+	 * the commit is cleaned up.
+	 */
+	if (ProcDiePending)
+	{
+		ereport(WARNING,
+				(errcode(ERRCODE_ADMIN_SHUTDOWN),
+				 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * It's unclear what to do if a query cancel interrupt arrives.  We
+	 * can't actually abort at this point, but ignoring the interrupt
+	 * altogether is not helpful, so we just terminate the wait with a
+	 * suitable warning.
+	 */
+	if (QueryCancelPending)
+	{
+		QueryCancelPending = false;
+		ereport(WARNING,
+				(errmsg("canceling wait for synchronous replication due to user request"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * If the postmaster dies, we'll probably never get an
+	 * acknowledgement, because all the wal sender processes will exit. So
+	 * just bail out.
+	 */
+	if (!PostmasterIsAlive())
+	{
+		ProcDiePending = true;
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	return false;
+}
+
+/*
+ * Check if we can stop waiting for causal consistency.  We can stop waiting
+ * when the following conditions are met:
+ *
+ * 1.  All walsenders currently in 'joining' or 'available' state have
+ * applied the target LSN.
+ *
+ * 2.  Any stall periods caused by standbys dropping out of 'available' state
+ * have passed, so that we can be sure that their leases have expired and they
+ * have started rejecting causal reads transactions.
+ *
+ * The output parameter 'waitingFor' is set to the number of nodes we are
+ * currently waiting for.  The output parameters 'stallTimeMillis' is set to
+ * the number of milliseconds we need to wait for to observe any current
+ * commit stall.
+ *
+ * Returns true if commit can return control, because every standby has either
+ * applied the LSN or started rejecting causal_reads transactions.
+ */
+static bool
+CausalReadsCommitCanReturn(XLogRecPtr XactCommitLSN,
+						   int *waitingFor,
+						   long *stallTimeMillis)
+{
+	int i;
+	TimestampTz now;
+
+	/* Count how many joining/available nodes we are waiting for. */
+	*waitingFor = 0;
+	for (i = 0; i < max_wal_senders; ++i)
+	{
+		WalSnd *walsnd = &WalSndCtl->walsnds[i];
+
+		/*
+		 * Assuming atomic read of pid_t, we can check walsnd->pid without
+		 * acquiring the spinlock to avoid memory synchronization costs for
+		 * unused walsender slots.  We see a value that existed sometime at
+		 * least as recently as the last memory barrier.
+		 */
+		if (walsnd->pid != 0)
+		{
+			/*
+			 * We need to hold the spinlock to read LSNs, because we can't be
+			 * sure they can be read atomically.
+			 */
+			SpinLockAcquire(&walsnd->mutex);
+			if (walsnd->pid != 0 && walsnd->causal_reads_state >= WALSNDCRSTATE_JOINING)
+			{
+				if (walsnd->apply < XactCommitLSN)
+					++*waitingFor;
+			}
+			SpinLockRelease(&walsnd->mutex);
+		}
+	}
+
+	/* Check if there is a stall in progress that we need to observe. */
+	now = GetCurrentTimestamp();
+	LWLockAcquire(SyncRepLock, LW_SHARED);
+	if (WalSndCtl->stall_causal_reads_until > now)
+	{
+		long seconds;
+		int usecs;
+
+		/* Compute how long we have to wait, rounded up to nearest ms. */
+		TimestampDifference(now, WalSndCtl->stall_causal_reads_until,
+							&seconds, &usecs);
+		*stallTimeMillis = seconds * 1000 + (usecs + 999) / 1000;
+	}
+	else
+		*stallTimeMillis = 0;
+	LWLockRelease(SyncRepLock);
+
+	/* We are done if we are not waiting for any nodes or stalls. */
+	return *waitingFor == 0 && *stallTimeMillis == 0;
+}
+
+/*
+ * Wait for causal consistency in causal_reads mode, if requested by user.
+ */
+void
+CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN)
+{
+	long stallTimeMillis;
+	int waitingFor;
+	char *ps_display_buffer = NULL;
+
+	/* Leave if we aren't in causal_reads mode. */
+	if (!causal_reads)
+		return;
+
+	for (;;)
+	{
+		/* Reset latch before checking state. */
+		ResetLatch(MyLatch);
+
+		/*
+		 * Join the queue to be woken up if any causal reads joining/available
+		 * standby applies XactCommitLSN, if we aren't already in it.  We
+		 * don't actually know if we need to wait for any peers yet, but we
+		 * have to register just in case before checking the walsenders' state
+		 * to avoid a race condition that could occur if we did it after
+		 * calling CausalReadsCommitCanReturn.  (SyncRepWaitForLSN doesn't
+		 * have to do this because it can check the highest-seen LSN in
+		 * walsndctl->lsn[mode] which is protected by SyncRepLock, the same
+		 * lock as the queues.  We can't do that here, because there is no
+		 * single highest-seen LSN that is useful.  We must check
+		 * walsnd->apply for all relevant walsenders.  Therefore we must
+		 * register for notifications first, so that we can be notified via
+		 * our latch of any standby applying the LSN we're interested in after
+		 * we check but before we start waiting, or we could wait forever for
+		 * something that has already happened.)
+		 */
+		LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+		if (MyProc->syncRepState != SYNC_REP_WAITING)
+		{
+			MyProc->waitLSN = XactCommitLSN;
+			MyProc->syncRepState = SYNC_REP_WAITING;
+			SyncRepQueueInsert(SYNC_REP_WAIT_CAUSAL_READS_APPLY);
+			Assert(SyncRepQueueIsOrderedByLSN(SYNC_REP_WAIT_CAUSAL_READS_APPLY));
+		}
+		LWLockRelease(SyncRepLock);
+
+		/* Check if we're done. */
+		if (CausalReadsCommitCanReturn(XactCommitLSN, &waitingFor, &stallTimeMillis))
+		{
+			SyncRepCancelWait();
+			break;
+		}
+
+		Assert(waitingFor > 0 || stallTimeMillis > 0);
+
+		/* If we aren't actually waiting for any standbys, leave the queue. */
+		if (waitingFor == 0)
+			SyncRepCancelWait();
+
+		/* Update the ps title. */
+		if (update_process_title)
+		{
+			char buffer[80];
+
+			/* Remember the old value if this is our first update. */
+			if (ps_display_buffer == NULL)
+			{
+				int len;
+				const char *ps_display = get_ps_display(&len);
+
+				ps_display_buffer = palloc(len + 1);
+				memcpy(ps_display_buffer, ps_display, len);
+				ps_display_buffer[len] = '\0';
+			}
+
+			snprintf(buffer, sizeof(buffer),
+					 "waiting for %d peer(s) to apply %X/%X%s",
+					 waitingFor,
+					 (uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN,
+					 stallTimeMillis > 0 ? " (stalling)" : "");
+			set_ps_display(buffer, false);
+		}
+
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit()) /* Calls SyncRepCancelWait() if true. */
+			break;
+
+		/*
+		 * If are still waiting for peers, then we wait for any joining or
+		 * available peer to reach the LSN (or possibly stop being in one of
+		 * those states or go away).
+		 *
+		 * If not, there must be a non-zero stall time, so we wait for that to
+		 * elapse.
+		 */
+		if (waitingFor > 0)
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH, -1);
+		else
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH | WL_TIMEOUT,
+					  stallTimeMillis);
+	}
+
+	/* There is no way out of the loop that could leave us in the queue. */
+	Assert(SHMQueueIsDetached(&(MyProc->syncRepLinks)));
+	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
+	MyProc->waitLSN = 0;
+
+	/* Restore the ps display. */
+	if (ps_display_buffer != NULL)
+	{
+		set_ps_display(ps_display_buffer, false);
+		pfree(ps_display_buffer);
+	}
+}
+
 /*
  * Wait for synchronous replication, if requested by user.
  *
@@ -180,57 +434,9 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 		if (syncRepState == SYNC_REP_WAIT_COMPLETE)
 			break;
 
-		/*
-		 * If a wait for synchronous replication is pending, we can neither
-		 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
-		 * lead the client to believe that the transaction aborted, which
-		 * is not true: it's already committed locally. The former is no good
-		 * either: the client has requested synchronous replication, and is
-		 * entitled to assume that an acknowledged commit is also replicated,
-		 * which might not be true. So in this case we issue a WARNING (which
-		 * some clients may be able to interpret) and shut off further output.
-		 * We do NOT reset ProcDiePending, so that the process will die after
-		 * the commit is cleaned up.
-		 */
-		if (ProcDiePending)
-		{
-			ereport(WARNING,
-					(errcode(ERRCODE_ADMIN_SHUTDOWN),
-					 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * It's unclear what to do if a query cancel interrupt arrives.  We
-		 * can't actually abort at this point, but ignoring the interrupt
-		 * altogether is not helpful, so we just terminate the wait with a
-		 * suitable warning.
-		 */
-		if (QueryCancelPending)
-		{
-			QueryCancelPending = false;
-			ereport(WARNING,
-					(errmsg("canceling wait for synchronous replication due to user request"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * If the postmaster dies, we'll probably never get an
-		 * acknowledgement, because all the wal sender processes will exit. So
-		 * just bail out.
-		 */
-		if (!PostmasterIsAlive())
-		{
-			ProcDiePending = true;
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit())
 			break;
-		}
 
 		/*
 		 * Wait on latch.  Any condition that should wake us up will set the
@@ -403,6 +609,49 @@ SyncRepGetSynchronousStandby(void)
 }
 
 /*
+ * Check if the current WALSender process's application_name matches a name in
+ * causal_reads_standby_names (including '*' for wildcard).
+ */
+bool
+CausalReadsPotentialStandby(void)
+{
+	char *rawstring;
+	List	   *elemlist;
+	ListCell   *l;
+	bool		found = false;
+
+	/* Need a modifiable copy of string */
+	rawstring = pstrdup(causal_reads_standby_names);
+
+	/* Parse string into list of identifiers */
+	if (!SplitIdentifierString(rawstring, ',', &elemlist))
+	{
+		/* syntax error in list */
+		pfree(rawstring);
+		list_free(elemlist);
+		/* GUC machinery will have already complained - no need to do again */
+		return 0;
+	}
+
+	foreach(l, elemlist)
+	{
+		char	   *standby_name = (char *) lfirst(l);
+
+		if (pg_strcasecmp(standby_name, application_name) == 0 ||
+			pg_strcasecmp(standby_name, "*") == 0)
+		{
+			found = true;
+			break;
+		}
+	}
+
+	pfree(rawstring);
+	list_free(elemlist);
+
+	return found;
+}
+
+/*
  * Update the LSNs on each queue based upon our latest state. This
  * implements a simple policy of first-valid-standby-releases-waiter.
  *
@@ -410,22 +659,27 @@ SyncRepGetSynchronousStandby(void)
  * perhaps also which information we store as well.
  */
 void
-SyncRepReleaseWaiters(void)
+SyncRepReleaseWaiters(bool walsender_cr_available_or_joining)
 {
 	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
+	int			numapply = 0;
+	int			numcausalreadsapply = 0;
+	bool		is_highest_priority_sync_standby;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
-	 * potential sync standbys then we have nothing to do. If we are still
-	 * starting up, still running base backup or the current flush position
-	 * is still invalid, then leave quickly also.
+	 * potential sync standbys and not in a state that causal_reads waits for,
+	 * then we have nothing to do. If we are still starting up, still running
+	 * base backup or the current flush position is still invalid, then leave
+	 * quickly also.
 	 */
-	if (MyWalSnd->sync_standby_priority == 0 ||
-		MyWalSnd->state < WALSNDSTATE_STREAMING ||
-		XLogRecPtrIsInvalid(MyWalSnd->flush))
+	if (!walsender_cr_available_or_joining &&
+		(MyWalSnd->sync_standby_priority == 0 ||
+		 MyWalSnd->state < WALSNDSTATE_STREAMING ||
+		 XLogRecPtrIsInvalid(MyWalSnd->flush)))
 		return;
 
 	/*
@@ -435,45 +689,77 @@ SyncRepReleaseWaiters(void)
 	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
 	syncWalSnd = SyncRepGetSynchronousStandby();
 
-	/* We should have found ourselves at least */
-	Assert(syncWalSnd != NULL);
+	/*
+	 * If we aren't managing the highest priority standby then make a note of
+	 * that so we can announce a takeover in the log if we ever get that job.
+	 */
+	is_highest_priority_sync_standby = syncWalSnd == MyWalSnd;
+	if (!is_highest_priority_sync_standby)
+		announce_next_takeover = true;
 
 	/*
-	 * If we aren't managing the highest priority standby then just leave.
+	 * If we aren't managing the highest priority standby or a standby in
+	 * causal reads 'joining' or 'available' state, then just leave.
 	 */
-	if (syncWalSnd != MyWalSnd)
+	if (!is_highest_priority_sync_standby && !walsender_cr_available_or_joining)
 	{
 		LWLockRelease(SyncRepLock);
-		announce_next_takeover = true;
 		return;
 	}
 
 	/*
 	 * Set the lsn first so that when we wake backends they will release up to
-	 * this location.
+	 * this location.  For the single-standby synchronous commit levels, we
+	 * only do this if we are the current synchronous standby and we are
+	 * advancing the LSN further than it has been advanced before, so that
+	 * SyncRepWaitForLSN can skip waiting in some cases.
 	 */
-	if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+	if (is_highest_priority_sync_standby)
 	{
-		walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
-		numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
-		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
+		if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
+			numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE,
+										MyWalSnd->write);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
+			numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH,
+										MyWalSnd->flush);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+			numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY,
+										MyWalSnd->apply);
+		}
 	}
+	/*
+	 * For causal_reads, all walsenders currently in available or joining
+	 * state must reach the LSN on their own, and standbys will reach LSNs in
+	 * any order.  It doesn't make sense to keep the highest seen LSN in a
+	 * single walsndctl->lsn element.  (CausalReadsWaitForLSN has handling for
+	 * LSNs that have already been reached).
+	 */
+	if (walsender_cr_available_or_joining)
+		numcausalreadsapply =
+			SyncRepWakeQueue(false, SYNC_REP_WAIT_CAUSAL_READS_APPLY,
+							 MyWalSnd->apply);
 
 	LWLockRelease(SyncRepLock);
 
-	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X",
+	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%X, %d procs to causal_reads apply",
 		 numwrite, (uint32) (MyWalSnd->write >> 32), (uint32) MyWalSnd->write,
-	   numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush);
+		 numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush,
+		 numapply, (uint32) (MyWalSnd->apply >> 32), (uint32) MyWalSnd->apply,
+		 numcausalreadsapply);
 
 	/*
 	 * If we are managing the highest priority standby, though we weren't
 	 * prior to this, then announce we are now the sync standby.
 	 */
-	if (announce_next_takeover)
+	if (is_highest_priority_sync_standby && announce_next_takeover)
 	{
 		announce_next_takeover = false;
 		ereport(LOG,
@@ -548,9 +834,8 @@ SyncRepGetStandbyPriority(void)
  * Must hold SyncRepLock.
  */
 static int
-SyncRepWakeQueue(bool all, int mode)
+SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn)
 {
-	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	PGPROC	   *proc = NULL;
 	PGPROC	   *thisproc = NULL;
 	int			numprocs = 0;
@@ -567,7 +852,7 @@ SyncRepWakeQueue(bool all, int mode)
 		/*
 		 * Assume the queue is ordered by LSN
 		 */
-		if (!all && walsndctl->lsn[mode] < proc->waitLSN)
+		if (!all && lsn < proc->waitLSN)
 			return numprocs;
 
 		/*
@@ -627,7 +912,7 @@ SyncRepUpdateSyncStandbysDefined(void)
 			int			i;
 
 			for (i = 0; i < NUM_SYNC_REP_WAIT_MODE; i++)
-				SyncRepWakeQueue(true, i);
+				SyncRepWakeQueue(true, i, InvalidXLogRecPtr);
 		}
 
 		/*
@@ -679,13 +964,31 @@ SyncRepQueueIsOrderedByLSN(int mode)
 #endif
 
 /*
+ * Make sure that CausalReadsWaitForLSN can't return until after the given
+ * lease expiry time has been reached.
+ *
+ * Wake up all backends waiting in CausalReadsWaitForLSN, because the set of
+ * available/joining peers has changed, and there is a new stall time they
+ * need to observe.
+ */
+void
+CausalReadsBeginStall(TimestampTz lease_expiry_time)
+{
+	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+	WalSndCtl->stall_causal_reads_until =
+		Max(WalSndCtl->stall_causal_reads_until, lease_expiry_time);
+	SyncRepWakeQueue(true, SYNC_REP_WAIT_CAUSAL_READS_APPLY, InvalidXLogRecPtr);
+	LWLockRelease(SyncRepLock);
+}
+
+/*
  * ===========================================================
  * Synchronous Replication functions executed by any process
  * ===========================================================
  */
 
 bool
-check_synchronous_standby_names(char **newval, void **extra, GucSource source)
+check_standby_names(char **newval, void **extra, GucSource source)
 {
 	char	   *rawstring;
 	List	   *elemlist;
@@ -728,6 +1031,9 @@ assign_synchronous_commit(int newval, void *extra)
 		case SYNCHRONOUS_COMMIT_REMOTE_FLUSH:
 			SyncRepWaitMode = SYNC_REP_WAIT_FLUSH;
 			break;
+		case SYNCHRONOUS_COMMIT_REMOTE_APPLY:
+			SyncRepWaitMode = SYNC_REP_WAIT_APPLY;
+			break;
 		default:
 			SyncRepWaitMode = SYNC_REP_NO_WAIT;
 			break;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 7b36e02..4526d73 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -55,6 +55,7 @@
 #include "libpq/pqformat.h"
 #include "libpq/pqsignal.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "replication/walsender.h"
 #include "storage/ipc.h"
@@ -101,6 +102,7 @@ static uint32 recvOff = 0;
  */
 static volatile sig_atomic_t got_SIGHUP = false;
 static volatile sig_atomic_t got_SIGTERM = false;
+static volatile sig_atomic_t got_SIGUSR2 = false;
 
 /*
  * LogstreamResult indicates the byte positions that we have already
@@ -145,14 +147,33 @@ static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
 static void XLogWalRcvSendReply(bool force, bool requestReply);
 static void XLogWalRcvSendHSFeedback(bool immed);
-static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
+static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+								  TimestampTz *causalReadsUntil);
 
 /* Signal handlers */
 static void WalRcvSigHupHandler(SIGNAL_ARGS);
 static void WalRcvSigUsr1Handler(SIGNAL_ARGS);
+static void WalRcvSigUsr2Handler(SIGNAL_ARGS);
 static void WalRcvShutdownHandler(SIGNAL_ARGS);
 static void WalRcvQuickDieHandler(SIGNAL_ARGS);
 
+static void WalRcvBlockSigUsr2(void)
+{
+	sigset_t mask;
+
+	sigemptyset(&mask);
+	sigaddset(&mask, SIGUSR2);
+	sigprocmask(SIG_BLOCK, &mask, NULL);
+}
+
+static void WalRcvUnblockSigUsr2(void)
+{
+	sigset_t mask;
+
+	sigemptyset(&mask);
+	sigaddset(&mask, SIGUSR2);
+	sigprocmask(SIG_UNBLOCK, &mask, NULL);
+}
 
 static void
 ProcessWalRcvInterrupts(void)
@@ -200,6 +221,7 @@ WalReceiverMain(void)
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz last_recv_timestamp;
 	bool		ping_sent;
+	bool		forceReply;
 
 	/*
 	 * WalRcv should be set up already (if we are a backend, we inherit this
@@ -251,6 +273,7 @@ WalReceiverMain(void)
 
 	/* Initialise to a sanish value */
 	walrcv->lastMsgSendTime = walrcv->lastMsgReceiptTime = walrcv->latestWalEndTime = GetCurrentTimestamp();
+	walrcv->causalReadsLease = 0;
 
 	SpinLockRelease(&walrcv->mutex);
 
@@ -268,7 +291,7 @@ WalReceiverMain(void)
 	pqsignal(SIGALRM, SIG_IGN);
 	pqsignal(SIGPIPE, SIG_IGN);
 	pqsignal(SIGUSR1, WalRcvSigUsr1Handler);
-	pqsignal(SIGUSR2, SIG_IGN);
+	pqsignal(SIGUSR2, WalRcvSigUsr2Handler);
 
 	/* Reset some signals that are accepted by postmaster but not here */
 	pqsignal(SIGCHLD, SIG_DFL);
@@ -299,6 +322,10 @@ WalReceiverMain(void)
 	/* Unblock signals (they were blocked when the postmaster forked us) */
 	PG_SETMASK(&UnBlockSig);
 
+	/* Block SIGUSR2 (we unblock it only during network waits). */
+	WalRcvBlockSigUsr2();
+	got_SIGUSR2 = false;
+
 	/* Establish the connection to the primary for XLOG streaming */
 	EnableWalRcvImmediateExit();
 	walrcv_connect(conninfo);
@@ -408,7 +435,9 @@ WalReceiverMain(void)
 				}
 
 				/* Wait a while for data to arrive */
+				WalRcvUnblockSigUsr2();
 				len = walrcv_receive(NAPTIME_PER_CYCLE, &buf);
+				WalRcvBlockSigUsr2();
 				if (len != 0)
 				{
 					/*
@@ -439,11 +468,21 @@ WalReceiverMain(void)
 							endofwal = true;
 							break;
 						}
+						WalRcvUnblockSigUsr2();
 						len = walrcv_receive(0, &buf);
+						WalRcvBlockSigUsr2();
+					}
+
+					if (got_SIGUSR2)
+					{
+						/* The recovery process asked us to force a reply. */
+						got_SIGUSR2 = false;
+						forceReply = true;
 					}
 
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(false, false);
+					XLogWalRcvSendReply(forceReply, false);
+					forceReply = false;
 
 					/*
 					 * If we've written some records, flush them to disk and
@@ -498,7 +537,15 @@ WalReceiverMain(void)
 						}
 					}
 
-					XLogWalRcvSendReply(requestReply, requestReply);
+					/* Check if the startup process has signaled us. */
+					if (got_SIGUSR2)
+					{
+						got_SIGUSR2 = false;
+						forceReply = true;
+					}
+
+					XLogWalRcvSendReply(requestReply || forceReply, requestReply);
+					forceReply = false;
 					XLogWalRcvSendHSFeedback(false);
 				}
 			}
@@ -740,6 +787,13 @@ WalRcvSigUsr1Handler(SIGNAL_ARGS)
 	errno = save_errno;
 }
 
+/* SIGUSR2: used to receive wakeups from recovery */
+static void
+WalRcvSigUsr2Handler(SIGNAL_ARGS)
+{
+	got_SIGUSR2 = true;
+}
+
 /* SIGTERM: set flag for main loop, or shutdown immediately if safe */
 static void
 WalRcvShutdownHandler(SIGNAL_ARGS)
@@ -800,6 +854,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 	XLogRecPtr	walEnd;
 	TimestampTz sendTime;
 	bool		replyRequested;
+	TimestampTz causalReadsLease;
 
 	resetStringInfo(&incoming_message);
 
@@ -820,7 +875,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				walEnd = pq_getmsgint64(&incoming_message);
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, NULL);
 
 				buf += hdrlen;
 				len -= hdrlen;
@@ -830,7 +885,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 		case 'k':				/* Keepalive */
 			{
 				/* copy message to StringInfo */
-				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
+				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char) + sizeof(int64);
 				if (len != hdrlen)
 					ereport(ERROR,
 							(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -842,8 +897,12 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
 				replyRequested = pq_getmsgbyte(&incoming_message);
+				causalReadsLease = IntegerTimestampToTimestampTz(
+					pq_getmsgint64(&incoming_message));
+				ProcessWalSndrMessage(walEnd, sendTime, &causalReadsLease);
 
-				ProcessWalSndrMessage(walEnd, sendTime);
+				/* Remember primary's timestamp at this WAL location. */
+				SetXLogReplayTimestampAtLsn(sendTime, walEnd);
 
 				/* If the primary requested a reply, send one immediately */
 				if (replyRequested)
@@ -1037,6 +1096,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	XLogRecPtr	applyPtr;
 	static TimestampTz sendTime = 0;
 	TimestampTz now;
+	TimestampTz applyTimestamp = 0;
 
 	/*
 	 * If the user doesn't want status to be reported to the master, be sure
@@ -1068,7 +1128,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	/* Construct a new message */
 	writePtr = LogstreamResult.Write;
 	flushPtr = LogstreamResult.Flush;
-	applyPtr = GetXLogReplayRecPtr(NULL);
+	applyTimestamp = GetXLogReplayTimestamp(&applyPtr);
 
 	resetStringInfo(&reply_message);
 	pq_sendbyte(&reply_message, 'r');
@@ -1076,6 +1136,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	pq_sendint64(&reply_message, flushPtr);
 	pq_sendint64(&reply_message, applyPtr);
 	pq_sendint64(&reply_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&reply_message, TimestampTzToIntegerTimestamp(applyTimestamp));
 	pq_sendbyte(&reply_message, requestReply ? 1 : 0);
 
 	/* Send it */
@@ -1174,15 +1235,52 @@ XLogWalRcvSendHSFeedback(bool immed)
  * Update shared memory status upon receiving a message from primary.
  *
  * 'walEnd' and 'sendTime' are the end-of-WAL and timestamp of the latest
- * message, reported by primary.
+ * message, reported by primary.  'causalReadsLease' is a pointer to
+ * the time the primary promises that this standby can safely claim to be
+ * causally consistent, to 0 if it cannot, or a NULL pointer for no change.
  */
 static void
-ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
+ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+					  TimestampTz *causalReadsLease)
 {
 	WalRcvData *walrcv = WalRcv;
 
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
 
+	/* Sanity check for the causalReadsLease time. */
+	if (causalReadsLease != NULL && *causalReadsLease != 0)
+	{
+		/* Deduce max_clock_skew from the causalReadsLease and sendTime. */
+#ifdef HAVE_INT64_TIMESTAMP
+		int64 diffMillis = (*causalReadsLease - sendTime) / 1000;
+#else
+		int64 diffMillis = (*causalReadsLease - sendTime) * 1000;
+#endif
+		int64 max_clock_skew = diffMillis / (CAUSAL_READS_CLOCK_SKEW_RATIO - 1);
+
+		if (sendTime > TimestampTzPlusMilliseconds(lastMsgReceiptTime, max_clock_skew))
+		{
+			/*
+			 * The primary's clock is more than max_clock_skew + network
+			 * latency ahead of the standby's clock.  (If the primary's clock
+			 * is more than max_clock_skew ahead of the standby's clock, but
+			 * by less than the network latency, then there isn't much we can
+			 * do to detect that; but it still seems useful to have this basic
+			 * sanity check for wildly misconfigured servers.)
+			 */
+			elog(LOG, "the primary server's clock time is too far ahead");
+			causalReadsLease = NULL;
+		}
+		/*
+		 * We could also try to detect cases where sendTime is more than
+		 * max_clock_skew in the past according to the standby's clock, but
+		 * that is indistinguishable from network latency/buffering, so we
+		 * could produce misleading error messages; if we do nothing, the
+		 * consequence is 'standby is not available for causal reads' errors
+		 * which should cause the user to investigate.
+		 */
+	}
+
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
 	if (walrcv->latestWalEnd < walEnd)
@@ -1190,6 +1288,8 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->latestWalEnd = walEnd;
 	walrcv->lastMsgSendTime = sendTime;
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
+	if (causalReadsLease != NULL)
+		walrcv->causalReadsLease = *causalReadsLease;
 	SpinLockRelease(&walrcv->mutex);
 
 	if (log_min_messages <= DEBUG2)
@@ -1222,6 +1322,22 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 }
 
 /*
+ * Wake up the walreceiver if it happens to be blocked in walrcv_receive,
+ * and tell it that a commit record has been applied.
+ *
+ * This is called by the startup process whenever interesting xlog records
+ * are applied, so that walreceiver can check if it needs to send an apply
+ * notification back to the master which may be waiting in a COMMIT with
+ * synchronous_commit = apply or causal_reads = on.
+ */
+void
+WalRcvWakeup(void)
+{
+	if (WalRcv->pid != 0)
+		kill(WalRcv->pid, SIGUSR2);
+}
+
+/*
  * Return a string constant representing the state. This is used
  * in system functions and views, and should *not* be translated.
  */
diff --git a/src/backend/replication/walreceiverfuncs.c b/src/backend/replication/walreceiverfuncs.c
index 5f6e423..f398a75 100644
--- a/src/backend/replication/walreceiverfuncs.c
+++ b/src/backend/replication/walreceiverfuncs.c
@@ -25,9 +25,11 @@
 
 #include "access/xlog_internal.h"
 #include "postmaster/startup.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "storage/pmsignal.h"
 #include "storage/shmem.h"
+#include "utils/guc.h"
 #include "utils/timestamp.h"
 
 WalRcvData *WalRcv = NULL;
@@ -374,3 +376,21 @@ GetReplicationTransferLatency(void)
 
 	return ms;
 }
+
+/*
+ * Used by snapmgr to check if this standby has a valid lease, granting it the
+ * right to consider itself available for causal reads.
+ */
+bool
+WalRcvCausalReadsAvailable(void)
+{
+	WalRcvData *walrcv = WalRcv;
+	TimestampTz now = GetCurrentTimestamp();
+	bool result;
+
+	SpinLockAcquire(&walrcv->mutex);
+	result = walrcv->causalReadsLease != 0 && now <= walrcv->causalReadsLease;
+	SpinLockRelease(&walrcv->mutex);
+
+	return result;
+}
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index c03e045..55c10e4 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -153,9 +153,20 @@ static StringInfoData tmpbuf;
  */
 static TimestampTz last_reply_timestamp = 0;
 
+static TimestampTz last_keepalive_timestamp = 0;
+
 /* Have we sent a heartbeat message asking for reply, since last reply? */
 static bool waiting_for_ping_response = false;
 
+/* How long do need to stay in JOINING state? */
+static XLogRecPtr causal_reads_joining_until = 0;
+
+/* The last causal reads lease sent to the standby. */
+static TimestampTz causal_reads_last_lease = 0;
+
+/* Is this WALSender listed in causal_reads_standby_names? */
+static bool am_potential_causal_reads_standby = false;
+
 /*
  * While streaming WAL in Copy mode, streamingDoneSending is set to true
  * after we have sent CopyDone. We should not send any more CopyData messages
@@ -242,6 +253,57 @@ InitWalSender(void)
 }
 
 /*
+ * If we are exiting unexpectedly, we may need to communicate with concurrent
+ * causal_reads commits to maintain the causal consistency guarantee.
+ */
+static void
+PrepareUncleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * We've lost contact with the standby, but it may still be alive.  We
+		 * can't let any causal_reads transactions return until we've stalled
+		 * for long enough for a zombie standby to start raising errors
+		 * because its lease has expired.
+		 */
+		elog(LOG, "standby \"%s\" is lost (no longer available for causal reads)", application_name);
+		CausalReadsBeginStall(causal_reads_last_lease);
+
+		/*
+		 * We set the state to a lower level _after_ beginning the stall,
+		 * otherwise there would be a tiny window where commits could return
+		 * without observing the stall.
+		 */
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
+ * We are shutting down because we received a goodbye message from the
+ * walreceiver.
+ */
+static void
+PrepareCleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * The standby is shutting down, so it won't be running any more
+		 * transactions.  It is therefore safe to stop waiting for it, and no
+		 * stall is necessary.
+		 */
+		elog(LOG, "standby \"%s\" is leaving (no longer available for causal reads)", application_name);
+
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
  * Clean up after an error.
  *
  * WAL sender processes don't use transactions like regular backends do.
@@ -264,7 +326,10 @@ WalSndErrorCleanup(void)
 
 	replication_active = false;
 	if (walsender_ready_to_stop)
+	{
+		PrepareUncleanExit();
 		proc_exit(0);
+	}
 
 	/* Revert back to startup state */
 	WalSndSetState(WALSNDSTATE_STARTUP);
@@ -276,6 +341,8 @@ WalSndErrorCleanup(void)
 static void
 WalSndShutdown(void)
 {
+	PrepareUncleanExit();
+
 	/*
 	 * Reset whereToSendOutput to prevent ereport from attempting to send any
 	 * more messages to the standby.
@@ -1386,6 +1453,7 @@ ProcessRepliesIfAny(void)
 		if (r < 0)
 		{
 			/* unexpected error or EOF */
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1402,6 +1470,7 @@ ProcessRepliesIfAny(void)
 		resetStringInfo(&reply_message);
 		if (pq_getmessage(&reply_message, 0))
 		{
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1451,6 +1520,7 @@ ProcessRepliesIfAny(void)
 				 * 'X' means that the standby is closing down the socket.
 				 */
 			case 'X':
+				PrepareCleanExit();
 				proc_exit(0);
 
 			default:
@@ -1543,15 +1613,29 @@ ProcessStandbyReplyMessage(void)
 	XLogRecPtr	writePtr,
 				flushPtr,
 				applyPtr;
+	int			applyLagMs;
 	bool		replyRequested;
+	TimestampTz now = GetCurrentTimestamp();
+	TimestampTz applyTimestamp;
 
 	/* the caller already consumed the msgtype byte */
 	writePtr = pq_getmsgint64(&reply_message);
 	flushPtr = pq_getmsgint64(&reply_message);
 	applyPtr = pq_getmsgint64(&reply_message);
 	(void) pq_getmsgint64(&reply_message);		/* sendTime; not used ATM */
+	applyTimestamp = IntegerTimestampToTimestampTz(pq_getmsgint64(&reply_message));
 	replyRequested = pq_getmsgbyte(&reply_message);
 
+	/* Compute the apply lag in milliseconds. */
+	if (applyTimestamp == 0)
+		applyLagMs = -1;
+	else
+#ifdef HAVE_INT64_TIMESTAMP
+		applyLagMs = (now - applyTimestamp) / 1000;
+#else
+		applyLagMs = (now - applyTimestamp) * 1000.0;
+#endif
+
 	elog(DEBUG2, "write %X/%X flush %X/%X apply %X/%X%s",
 		 (uint32) (writePtr >> 32), (uint32) writePtr,
 		 (uint32) (flushPtr >> 32), (uint32) flushPtr,
@@ -1568,16 +1652,116 @@ ProcessStandbyReplyMessage(void)
 	 */
 	{
 		WalSnd *walsnd = MyWalSnd;
+		WalSndCausalReadsState causal_reads_state = walsnd->causal_reads_state;
+		bool causal_reads_state_changed = false;
+		bool causal_reads_set_joining_until = false;
+
+		/*
+		 * Handle causal reads state transitions, if a causal_reads_timeout is
+		 * configured, this standby is listed in causal_reads_standby_names,
+		 * and we are a primary database (not a cascading standby).
+		 */
+		if (causal_reads_timeout != 0 &&
+			am_potential_causal_reads_standby &&
+			!am_cascading_walsender)
+		{
+			if (applyLagMs >= 0 && applyLagMs < causal_reads_timeout)
+			{
+				if (causal_reads_state == WALSNDCRSTATE_UNAVAILABLE)
+				{
+					/*
+					 * The standby is applying fast enough.  We can't grant a
+					 * lease yet though, we need to wait for everything that
+					 * was committed while this standby was unavailable to be
+					 * applied first.  We move to joining state while we wait
+					 * for the standby to catch up.
+					 */
+					causal_reads_state = WALSNDCRSTATE_JOINING;
+					causal_reads_set_joining_until = true;
+					causal_reads_state_changed = true;
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING &&
+						 applyPtr >= causal_reads_joining_until)
+				{
+					/*
+					 * The standby has applied everything committed before we
+					 * reached joining state, and has been waiting for remote
+					 * apply on this standby while it's been in joining state,
+					 * so it is safe to move to available state and send a
+					 * lease.
+					 */
+					causal_reads_state = WALSNDCRSTATE_AVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+			else
+			{
+				if (causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+					/*
+					 * We are dropping a causal reads available standby, so we
+					 * mustn't let any commit command that is waiting in
+					 * CausalReadsWaitForLSN return until we are sure that the
+					 * standby definitely knows that it's not available and
+					 * starts raising errors for causal_reads transactions.
+					 * TODO: We could just wait until the standby acks that
+					 * its lease has been cancelled, and start numbering
+					 * keepalives and sending the number back in replies, so
+					 * we know it's acking the right message; then lagging
+					 * standbys would be less disruptive, but for now we just
+					 * wait for the lease to expire, as we do when we lose
+					 * contact with a standby, for the sake of simplicity.
+					 */
+					CausalReadsBeginStall(causal_reads_last_lease);
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING)
+				{
+					/*
+					 * Dropping a joining standby doesn't require a stall,
+					 * because the standby doesn't think it's available, so
+					 * it's already raising the error for causal_reads
+					 * transactions.
+					 */
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+		}
 
 		SpinLockAcquire(&walsnd->mutex);
 		walsnd->write = writePtr;
 		walsnd->flush = flushPtr;
 		walsnd->apply = applyPtr;
+		walsnd->applyLagMs = applyLagMs;
+		walsnd->causal_reads_state = causal_reads_state;
 		SpinLockRelease(&walsnd->mutex);
+
+		if (causal_reads_set_joining_until)
+		{
+			/*
+			 * Record the end of the primary's WAL at some arbitrary point
+			 * observed _after_ we moved to joining state (so that causal
+			 * reads commits start waiting, closing a race).  The standby
+			 * won't become available until it has replayed up to here.
+			 */
+			causal_reads_joining_until = GetFlushRecPtr();
+		}
+
+		if (causal_reads_state_changed)
+		{
+			WalSndKeepalive(true);
+			elog(LOG, "standby \"%s\" is %s", application_name,
+				 causal_reads_state == WALSNDCRSTATE_UNAVAILABLE ? "unavailable for causal reads" :
+				 causal_reads_state == WALSNDCRSTATE_JOINING ? "joining as a causal reads standby..." :
+				 causal_reads_state == WALSNDCRSTATE_AVAILABLE ? "available for causal reads" :
+				 "UNKNOWN");
+		}
 	}
 
 	if (!am_cascading_walsender)
-		SyncRepReleaseWaiters();
+		SyncRepReleaseWaiters(MyWalSnd->causal_reads_state >= WALSNDCRSTATE_JOINING);
 
 	/*
 	 * Advance our local xmin horizon when the client confirmed a flush.
@@ -1724,27 +1908,34 @@ WalSndComputeSleeptime(TimestampTz now)
 {
 	long		sleeptime = 10000;		/* 10 s */
 
-	if (wal_sender_timeout > 0 && last_reply_timestamp > 0)
+	if ((wal_sender_timeout > 0 || causal_reads_timeout > 0) && last_reply_timestamp > 0)
 	{
 		TimestampTz wakeup_time;
 		long		sec_to_timeout;
 		int			microsec_to_timeout;
 
-		/*
-		 * At the latest stop sleeping once wal_sender_timeout has been
-		 * reached.
-		 */
-		wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-												  wal_sender_timeout);
-
-		/*
-		 * If no ping has been sent yet, wakeup when it's time to do so.
-		 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
-		 * the timeout passed without a response.
-		 */
-		if (!waiting_for_ping_response)
+		if (causal_reads_timeout != 0)
+			wakeup_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+													  causal_reads_timeout /
+													  CAUSAL_READS_KEEPALIVE_RATIO);
+		else
+		{
+			/*
+			 * At the latest stop sleeping once wal_sender_timeout has been
+			 * reached.
+			 */
 			wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-													  wal_sender_timeout / 2);
+													  wal_sender_timeout);
+
+			/*
+			 * If no ping has been sent yet, wakeup when it's time to do so.
+			 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
+			 * the timeout passed without a response.
+			 */
+			if (!waiting_for_ping_response)
+				wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+														  wal_sender_timeout / 2);
+		}
 
 		/* Compute relative time until wakeup. */
 		TimestampDifference(now, wakeup_time,
@@ -1765,15 +1956,28 @@ static void
 WalSndCheckTimeOut(TimestampTz now)
 {
 	TimestampTz timeout;
+	int allowed_time;
 
 	/* don't bail out if we're doing something that doesn't require timeouts */
 	if (last_reply_timestamp <= 0)
 		return;
 
+	/*
+	 * If a causal_reads_timeout is configured, it is used instead of
+	 * wal_sender_timeout.  Ideally we'd use causal_reads_timeout / 2 +
+	 * allowance for network latency, but since walreceiver can become quite
+	 * bogged down fsyncing WAL we allow more tolerance.  (This could be
+	 * tightened up once standbys hand writing off to the WAL writer).
+	 */
+	if (causal_reads_timeout != 0)
+		allowed_time = causal_reads_timeout;
+	else
+		allowed_time = wal_sender_timeout;
+
 	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
-										  wal_sender_timeout);
+										  allowed_time);
 
-	if (wal_sender_timeout > 0 && now >= timeout)
+	if (allowed_time > 0 && now >= timeout)
 	{
 		/*
 		 * Since typically expiration of replication timeout means
@@ -1806,6 +2010,9 @@ WalSndLoop(WalSndSendDataCallback send_data)
 	last_reply_timestamp = GetCurrentTimestamp();
 	waiting_for_ping_response = false;
 
+	/* Check if we are managing potential causal_reads standby. */
+	am_potential_causal_reads_standby = CausalReadsPotentialStandby();
+
 	/*
 	 * Loop until we reach the end of this timeline or the client requests to
 	 * stop streaming.
@@ -1966,6 +2173,7 @@ InitWalSenderSlot(void)
 			walsnd->flush = InvalidXLogRecPtr;
 			walsnd->apply = InvalidXLogRecPtr;
 			walsnd->state = WALSNDSTATE_STARTUP;
+			walsnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
 			walsnd->latch = &MyProc->procLatch;
 			SpinLockRelease(&walsnd->mutex);
 			/* don't need the lock anymore */
@@ -2735,6 +2943,24 @@ WalSndGetStateString(WalSndState state)
 	return "UNKNOWN";
 }
 
+/*
+ * Return a string constant representing the causal reads state. This is used
+ * in system views, and should *not* be translated.
+ */
+static const char *
+WalSndGetCausalReadsStateString(WalSndCausalReadsState causal_reads_state)
+{
+	switch (causal_reads_state)
+	{
+		case WALSNDCRSTATE_UNAVAILABLE:
+			return "unavailable";
+		case WALSNDCRSTATE_JOINING:
+			return "joining";
+		case WALSNDCRSTATE_AVAILABLE:
+			return "available";
+	}
+	return "UNKNOWN";
+}
 
 /*
  * Returns activity of walsenders, including pids and xlog locations sent to
@@ -2743,7 +2969,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	8
+#define PG_STAT_GET_WAL_SENDERS_COLS	10
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2791,8 +3017,10 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		XLogRecPtr	write;
 		XLogRecPtr	flush;
 		XLogRecPtr	apply;
+		int			applyLagMs;
 		int			priority;
 		WalSndState state;
+		WalSndCausalReadsState causalReadsState;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
 		bool		nulls[PG_STAT_GET_WAL_SENDERS_COLS];
 
@@ -2802,9 +3030,11 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		SpinLockAcquire(&walsnd->mutex);
 		sentPtr = walsnd->sentPtr;
 		state = walsnd->state;
+		causalReadsState = walsnd->causal_reads_state;
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
+		applyLagMs = walsnd->applyLagMs;
 		priority = walsnd->sync_standby_priority;
 		SpinLockRelease(&walsnd->mutex);
 
@@ -2839,6 +3069,23 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				nulls[5] = true;
 			values[5] = LSNGetDatum(apply);
 
+			if (applyLagMs < 0)
+				nulls[6] = true;
+			else
+			{
+				Interval *applyLagInterval = palloc(sizeof(Interval));
+
+				applyLagInterval->month = 0;
+				applyLagInterval->day = 0;
+#ifdef HAVE_INT64_TIMESTAMP
+				applyLagInterval->time = applyLagMs * 1000;
+#else
+				applyLagInterval->time = applyLagMs / 1000.0;
+#endif
+				nulls[6] = false;
+				values[6] = IntervalPGetDatum(applyLagInterval);
+			}
+
 			/*
 			 * Treat a standby such as a pg_basebackup background process
 			 * which always returns an invalid flush location, as an
@@ -2846,18 +3093,21 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 			 */
 			priority = XLogRecPtrIsInvalid(walsnd->flush) ? 0 : priority;
 
-			values[6] = Int32GetDatum(priority);
+			values[7] = Int32GetDatum(priority);
 
 			/*
 			 * More easily understood version of standby state. This is purely
 			 * informational, not different from priority.
 			 */
 			if (priority == 0)
-				values[7] = CStringGetTextDatum("async");
+				values[8] = CStringGetTextDatum("async");
 			else if (walsnd == sync_standby)
-				values[7] = CStringGetTextDatum("sync");
+				values[8] = CStringGetTextDatum("sync");
 			else
-				values[7] = CStringGetTextDatum("potential");
+				values[8] = CStringGetTextDatum("potential");
+
+			values[9] =
+				CStringGetTextDatum(WalSndGetCausalReadsStateString(causalReadsState));
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
@@ -2877,14 +3127,52 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 static void
 WalSndKeepalive(bool requestReply)
 {
+	TimestampTz now;
+	TimestampTz causal_reads_lease;
+
 	elog(DEBUG2, "sending replication keepalive");
 
+	/*
+	 * If the walsender currently deems the standby to be available for causal
+	 * reads, then it grants a causal reads lease.  The lease authorizes the
+	 * standby to consider itself available for causal reads until a short
+	 * time in the future.  The primary promises to uphold the causal reads
+	 * guarantee until that time, by stalling commits until the the lease has
+	 * expired if necessary.
+	 */
+	now = GetCurrentTimestamp();
+	if (MyWalSnd->causal_reads_state < WALSNDCRSTATE_AVAILABLE)
+		causal_reads_lease = 0; /* Not available, no lease granted. */
+	else
+	{
+		/*
+		 * Since this timestamp is being sent to the standby where it will be
+		 * compared against a time generated by the standby's system clock, we
+		 * must consider clock skew.  First, we decide on a maximum tolerable
+		 * difference between system clocks.  If the primary's clock is ahead
+		 * of the standby's by more than this, then all bets are off (the
+		 * standby could falsely believe it has a valid lease).  If the
+		 * primary's clock is behind the standby's by more than this, then the
+		 * standby will err the other way and generate spurious errors in
+		 * causal_reads mode.  Rather than having a separate GUC for this, we
+		 * derive it from causal_reads_timeout.
+		 */
+		int max_clock_skew = causal_reads_timeout / CAUSAL_READS_CLOCK_SKEW_RATIO;
+
+		/* Compute and remember the expiry time of the lease we're granting. */
+		causal_reads_last_lease = TimestampTzPlusMilliseconds(now, causal_reads_timeout);
+		/* The version we'll send to the standby is adjusted to tolerate clock skew. */
+		causal_reads_lease =
+			TimestampTzPlusMilliseconds(causal_reads_last_lease, -max_clock_skew);
+	}
+
 	/* construct the message... */
 	resetStringInfo(&output_message);
 	pq_sendbyte(&output_message, 'k');
 	pq_sendint64(&output_message, sentPtr);
-	pq_sendint64(&output_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(now));
 	pq_sendbyte(&output_message, requestReply ? 1 : 0);
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(causal_reads_lease));
 
 	/* ... and send it wrapped in CopyData */
 	pq_putmessage_noblock('d', output_message.data, output_message.len);
@@ -2902,23 +3190,32 @@ WalSndKeepaliveIfNecessary(TimestampTz now)
 	 * Don't send keepalive messages if timeouts are globally disabled or
 	 * we're doing something not partaking in timeouts.
 	 */
-	if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
+	if ((wal_sender_timeout <= 0 && causal_reads_timeout == 0) || last_reply_timestamp <= 0)
 		return;
 
-	if (waiting_for_ping_response)
+	if (waiting_for_ping_response && causal_reads_timeout == 0)
 		return;
 
 	/*
 	 * If half of wal_sender_timeout has lapsed without receiving any reply
 	 * from the standby, send a keep-alive message to the standby requesting
 	 * an immediate reply.
+	 *
+	 * If causal_reads_timeout has been configured, use it to control
+	 * keepalive intervals rather than wal_sender_timeout.
 	 */
-	ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-											wal_sender_timeout / 2);
+	if (causal_reads_timeout != 0)
+		ping_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+												causal_reads_timeout /
+												CAUSAL_READS_KEEPALIVE_RATIO);
+	else
+		ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+												wal_sender_timeout / 2);
 	if (now >= ping_time)
 	{
 		WalSndKeepalive(true);
 		waiting_for_ping_response = true;
+		last_keepalive_timestamp = now;
 
 		/* Try to flush pending output to the client */
 		if (pq_flush_if_writable() != 0)
diff --git a/src/backend/utils/adt/timestamp.c b/src/backend/utils/adt/timestamp.c
index 1525d2a..6ff111f 100644
--- a/src/backend/utils/adt/timestamp.c
+++ b/src/backend/utils/adt/timestamp.c
@@ -1611,6 +1611,20 @@ IntegerTimestampToTimestampTz(int64 timestamp)
 #endif
 
 /*
+ * TimestampTzToIntegerTimestamp -- convert a native timestamp to int64 format
+ *
+ * When compiled with --enable-integer-datetimes, this is implemented as a
+ * no-op macro.
+ */
+#ifndef HAVE_INT64_TIMESTAMP
+int64
+TimestampTzToIntegerTimestamp(TimestampTz timestamp)
+{
+	return timestamp * 1000000;
+}
+#endif
+
+/*
  * TimestampDifference -- convert the difference between two timestamps
  *		into integer seconds and microseconds
  *
diff --git a/src/backend/utils/errcodes.txt b/src/backend/utils/errcodes.txt
index 04c9c00..d4bf0c0 100644
--- a/src/backend/utils/errcodes.txt
+++ b/src/backend/utils/errcodes.txt
@@ -302,6 +302,7 @@ Section: Class 40 - Transaction Rollback
 40001    E    ERRCODE_T_R_SERIALIZATION_FAILURE                              serialization_failure
 40003    E    ERRCODE_T_R_STATEMENT_COMPLETION_UNKNOWN                       statement_completion_unknown
 40P01    E    ERRCODE_T_R_DEADLOCK_DETECTED                                  deadlock_detected
+40P02    E    ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE                         causal_reads_not_available
 
 Section: Class 42 - Syntax Error or Access Rule Violation
 
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index ea5a09a..fb91cad 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -345,12 +345,13 @@ static const struct config_enum_entry constraint_exclusion_options[] = {
 };
 
 /*
- * Although only "on", "off", "remote_write", and "local" are documented, we
- * accept all the likely variants of "on" and "off".
+ * Although only "on", "off", "remote_apply", "remote_write", and "local" are
+ * documented, we accept all the likely variants of "on" and "off".
  */
 static const struct config_enum_entry synchronous_commit_options[] = {
 	{"local", SYNCHRONOUS_COMMIT_LOCAL_FLUSH, false},
 	{"remote_write", SYNCHRONOUS_COMMIT_REMOTE_WRITE, false},
+	{"remote_apply", SYNCHRONOUS_COMMIT_REMOTE_APPLY, false},
 	{"on", SYNCHRONOUS_COMMIT_ON, false},
 	{"off", SYNCHRONOUS_COMMIT_OFF, false},
 	{"true", SYNCHRONOUS_COMMIT_ON, true},
@@ -1632,6 +1633,16 @@ static struct config_bool ConfigureNamesBool[] =
 		NULL, NULL, NULL
 	},
 
+	{
+		{"causal_reads", PGC_USERSET, REPLICATION_STANDBY,
+		 gettext_noop("Enables causal reads."),
+		 NULL
+		},
+		&causal_reads,
+		false,
+		NULL, NULL, NULL
+	},
+
 	/* End-of-list marker */
 	{
 		{NULL, 0, 0, NULL, NULL}, NULL, false, NULL, NULL, NULL
@@ -1790,6 +1801,17 @@ static struct config_int ConfigureNamesInt[] =
 	},
 
 	{
+		{"causal_reads_timeout", PGC_SIGHUP, REPLICATION_STANDBY,
+			gettext_noop("Sets the maximum apply lag before causal reads standbys are no longer available."),
+			NULL,
+			GUC_UNIT_MS
+		},
+		&causal_reads_timeout,
+		0, 0, INT_MAX,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"max_connections", PGC_POSTMASTER, CONN_AUTH_SETTINGS,
 			gettext_noop("Sets the maximum number of concurrent connections."),
 			NULL
@@ -3386,7 +3408,18 @@ static struct config_string ConfigureNamesString[] =
 		},
 		&SyncRepStandbyNames,
 		"",
-		check_synchronous_standby_names, NULL, NULL
+		check_standby_names, NULL, NULL
+	},
+
+	{
+		{"causal_reads_standby_names", PGC_SIGHUP, REPLICATION_MASTER,
+			gettext_noop("List of names of potential causal reads standbys."),
+			NULL,
+			GUC_LIST_INPUT
+		},
+		&causal_reads_standby_names,
+		"*",
+		check_standby_names, NULL, NULL
 	},
 
 	{
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index 63e908d..b1455e1 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -46,8 +46,11 @@
 
 #include "access/transam.h"
 #include "access/xact.h"
+#include "access/xlog.h"
 #include "lib/pairingheap.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
+#include "replication/walreceiver.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
 #include "storage/procarray.h"
@@ -209,6 +212,18 @@ GetTransactionSnapshot(void)
 				 "cannot take query snapshot during a parallel operation");
 
 		/*
+		 * In causal_reads mode on a standby, check if we have definitely
+		 * applied WAL for any COMMIT that returned successfully on the
+		 * primary.
+		 *
+		 * TODO: Machine readable error code?
+		 */
+		if (causal_reads && RecoveryInProgress() && !WalRcvCausalReadsAvailable())
+			ereport(ERROR,
+					(errcode(ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE),
+					 errmsg("standby is not available for causal reads")));
+
+		/*
 		 * In transaction-snapshot mode, the first snapshot must live until
 		 * end of xact regardless of what the caller does with it, so we must
 		 * make a copy of it rather than returning CurrentSnapshotData
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index ebeb582..4037dc6 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -60,7 +60,11 @@ typedef enum
 	SYNCHRONOUS_COMMIT_LOCAL_FLUSH,		/* wait for local flush only */
 	SYNCHRONOUS_COMMIT_REMOTE_WRITE,	/* wait for local flush and remote
 										 * write */
-	SYNCHRONOUS_COMMIT_REMOTE_FLUSH		/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_FLUSH,	/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_APPLY,	/* wait for local flush and remote
+										 * apply */
+	SYNCHRONOUS_COMMIT_CONSISTENT_APPLY /* wait for local flusha and remote
+										   apply with causal consistency */
 }	SyncCommitLevel;
 
 /* Define the default setting for synchonous_commit */
@@ -144,10 +148,13 @@ typedef void (*SubXactCallback) (SubXactEvent event, SubTransactionId mySubid,
  * EOXact... routines which run at the end of the original transaction
  * completion.
  */
+#define XACT_COMPLETION_SYNC_APPLY_FEEDBACK		(1U << 29)
 #define XACT_COMPLETION_UPDATE_RELCACHE_FILE	(1U << 30)
 #define XACT_COMPLETION_FORCE_SYNC_COMMIT		(1U << 31)
 
 /* Access macros for above flags */
+#define XactCompletionSyncApplyFeedback(xinfo) \
+	(!!(xinfo & XACT_COMPLETION_SYNC_APPLY_FEEDBACK))
 #define XactCompletionRelcacheInitFileInval(xinfo) \
 	(!!(xinfo & XACT_COMPLETION_UPDATE_RELCACHE_FILE))
 #define XactCompletionForceSyncCommit(xinfo) \
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index ecd30ce..efb9719 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -236,6 +236,9 @@ extern void GetXLogReceiptTime(TimestampTz *rtime, bool *fromStream);
 extern XLogRecPtr GetXLogReplayRecPtr(TimeLineID *replayTLI);
 extern XLogRecPtr GetXLogInsertRecPtr(void);
 extern XLogRecPtr GetXLogWriteRecPtr(void);
+extern void SetXLogReplayTimestamp(TimestampTz timestamp);
+extern void SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn);
+extern TimestampTz GetXLogReplayTimestamp(XLogRecPtr *lsn);
 extern bool RecoveryIsPaused(void);
 extern void SetRecoveryPause(bool recoveryPause);
 extern TimestampTz GetLatestXTime(void);
@@ -268,6 +271,8 @@ extern bool CheckPromoteSignal(void);
 extern void WakeupRecovery(void);
 extern void SetWalWriterSleeping(bool sleeping);
 
+extern void XLogRequestWalReceiverReply(void);
+
 extern void assign_max_wal_size(int newval, void *extra);
 extern void assign_checkpoint_completion_target(double newval, void *extra);
 
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 62b9125..fa9b184 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2710,7 +2710,7 @@ DATA(insert OID = 1936 (  pg_stat_get_backend_idset		PGNSP PGUID 12 1 100 0 0 f
 DESCR("statistics: currently active backend IDs");
 DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f f f t s r 1 0 2249 "23" "{23,26,23,26,25,25,25,16,1184,1184,1184,1184,869,25,23,28,28,16,25,25,23,16,25}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{pid,datid,pid,usesysid,application_name,state,query,waiting,xact_start,query_start,backend_start,state_change,client_addr,client_hostname,client_port,backend_xid,backend_xmin,ssl,sslversion,sslcipher,sslbits,sslcompression,sslclientdn}" _null_ _null_ pg_stat_get_activity _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active backends");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,23,25}" "{o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25,25}" "{o,o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state,causal_reads_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index 96e059b..76a4ee9 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -23,14 +23,34 @@
 #define SYNC_REP_NO_WAIT		-1
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
+#define SYNC_REP_WAIT_APPLY		2
+#define SYNC_REP_WAIT_CAUSAL_READS_APPLY 3
 
-#define NUM_SYNC_REP_WAIT_MODE	2
+#define NUM_SYNC_REP_WAIT_MODE	4
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
 #define SYNC_REP_WAITING			1
 #define SYNC_REP_WAIT_COMPLETE		2
 
+/*
+ * ratio of causal_read_timeout to max_clock_skew (4 means than the maximum
+ * tolerated clock difference between primary and standbys using causal_reads
+ * is 1/4 of causal_reads_timeout)
+ */
+#define CAUSAL_READS_CLOCK_SKEW_RATIO 4
+
+/*
+ * ratio of causal_reads_timeout to keepalive time (2 means that the effective
+ * keepalive time is 1/2 of the causal_reads_timeout GUC when it is non-zero)
+ */
+#define CAUSAL_READS_KEEPALIVE_RATIO 2
+
+/* GUC variables */
+extern int causal_reads_timeout;
+extern bool causal_reads;
+extern char *causal_reads_standby_names;
+
 /* user-settable parameters for synchronous replication */
 extern char *SyncRepStandbyNames;
 
@@ -42,16 +62,23 @@ extern void SyncRepCleanupAtProcExit(void);
 
 /* called by wal sender */
 extern void SyncRepInitConfig(void);
-extern void SyncRepReleaseWaiters(void);
+extern void SyncRepReleaseWaiters(bool walsender_cr_available_or_joining);
 
 /* called by checkpointer */
 extern void SyncRepUpdateSyncStandbysDefined(void);
 
+/* called by user backend (xact.c) */
+extern void CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN);
+
+/* called by wal sender */
+extern void CausalReadsBeginStall(TimestampTz lease_expiry_time);
+extern bool CausalReadsPotentialStandby(void);
+
 /* forward declaration to avoid pulling in walsender_private.h */
 struct WalSnd;
 extern struct WalSnd *SyncRepGetSynchronousStandby(void);
 
-extern bool check_synchronous_standby_names(char **newval, void **extra, GucSource source);
+extern bool check_standby_names(char **newval, void **extra, GucSource source);
 extern void assign_synchronous_commit(int newval, void *extra);
 
 #endif   /* _SYNCREP_H */
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 6eacb09..7f83934 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -80,6 +80,13 @@ typedef struct
 	TimeLineID	receivedTLI;
 
 	/*
+	 * causalReadsLease is the time until which the primary has authorized
+	 * this standby to consider itself available for causal_reads mode, or 0
+	 * for not authorized.
+	 */
+	TimestampTz causalReadsLease;
+
+	/*
 	 * latestChunkStart is the starting byte position of the current "batch"
 	 * of received WAL.  It's actually the same as the previous value of
 	 * receivedUpto before the last flush to disk.  Startup process can use
@@ -162,5 +169,8 @@ extern void RequestXLogStreaming(TimeLineID tli, XLogRecPtr recptr,
 extern XLogRecPtr GetWalRcvWriteRecPtr(XLogRecPtr *latestChunkStart, TimeLineID *receiveTLI);
 extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
+extern void WalRcvWakeup(void);
+
+extern bool WalRcvCausalReadsAvailable(void);
 
 #endif   /* _WALRECEIVER_H */
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 7794aa5..81a2776 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -27,6 +27,13 @@ typedef enum WalSndState
 	WALSNDSTATE_STREAMING
 } WalSndState;
 
+typedef enum WalSndCausalReadsState
+{
+	WALSNDCRSTATE_UNAVAILABLE = 0,
+	WALSNDCRSTATE_JOINING,
+	WALSNDCRSTATE_AVAILABLE
+} WalSndCausalReadsState;
+
 /*
  * Each walsender has a WalSnd struct in shared memory.
  */
@@ -34,6 +41,7 @@ typedef struct WalSnd
 {
 	pid_t		pid;			/* this walsender's process id, or 0 */
 	WalSndState state;			/* this walsender's state */
+	WalSndCausalReadsState causal_reads_state; /* the walsender's causal reads state */
 	XLogRecPtr	sentPtr;		/* WAL has been sent up to this point */
 	bool		needreload;		/* does currently-open file need to be
 								 * reloaded? */
@@ -46,6 +54,7 @@ typedef struct WalSnd
 	XLogRecPtr	write;
 	XLogRecPtr	flush;
 	XLogRecPtr	apply;
+	int			applyLagMs;
 
 	/* Protects shared variables shown above. */
 	slock_t		mutex;
@@ -88,6 +97,12 @@ typedef struct
 	 */
 	bool		sync_standbys_defined;
 
+	/*
+	 * Until when must commits in causal_reads stall?  This is used to wait
+	 * for causal reads leases to expire.
+	 */
+	TimestampTz	stall_causal_reads_until;
+
 	WalSnd		walsnds[FLEXIBLE_ARRAY_MEMBER];
 } WalSndCtlData;
 
diff --git a/src/include/utils/timestamp.h b/src/include/utils/timestamp.h
index fbead3a..297e151 100644
--- a/src/include/utils/timestamp.h
+++ b/src/include/utils/timestamp.h
@@ -227,9 +227,11 @@ extern bool TimestampDifferenceExceeds(TimestampTz start_time,
 #ifndef HAVE_INT64_TIMESTAMP
 extern int64 GetCurrentIntegerTimestamp(void);
 extern TimestampTz IntegerTimestampToTimestampTz(int64 timestamp);
+extern int64 TimestampTzToIntegerTimestamp(TimestampTz timestamp);
 #else
 #define GetCurrentIntegerTimestamp()	GetCurrentTimestamp()
 #define IntegerTimestampToTimestampTz(timestamp) (timestamp)
+#define TimestampTzToIntegerTimestamp(timestamp) (timestamp)
 #endif
 
 extern TimestampTz time_t_to_timestamptz(pg_time_t tm);
#41Michael Paquier
michael.paquier@gmail.com
In reply to: Thomas Munro (#40)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Tue, Mar 1, 2016 at 11:53 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Tue, Mar 1, 2016 at 2:46 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

Hi,

On 2016/02/29 18:05, Thomas Munro wrote:

On Mon, Feb 29, 2016 at 9:05 PM, Amit Langote wrote:

+         servers.  A transaction that is run with
<varname>causal_reads</> set
+         to <literal>on</> is guaranteed either to see the effects of all
+         completed transactions run on the primary with the setting on, or to
+         receive an error "standby is not available for causal reads".

"A transaction that is run" means "A transaction that is run on a
standby", right?

Well, it could be any server, standby or primary. Of course standbys
are the interesting case since it it was already true that if you run
two sequential transactions run on the primary, the second can see the
effect of the first, but I like the idea of a general rule that
applies anywhere, allowing you not to care which server it is.

I meant actually in context of that sentence only.

Ok, here's a new version that includes that change, fixes a conflict
with recent commit 10b48522 and removes an accidental duplicate copy
of the README file.

Finally I got my eyes on this patch.

To put it short, this patch introduces two different concepts:
- addition of a new value, remote_apply, for synchronous_commit, which
is actually where things overlap a bit with the N-sync patch, because
by combining the addition of remote_apply with the N-sync patch, it is
possible to ensure that an LSN is applied to multiple targets instead
of one now. In this case, as the master will wait for the LSN to be
applied on the sync standby, there is no need for the application to
have error handling in case a read transaction is happening on the
standby as the change is already visible on the standby when
committing it on master. However the cost here is that if the standby
node lags behind, it puts some extra wait as well on the master side.
- casual reads, which makes the master able to drop standbys that are
lagging too much behind and let callers of standbys know that it is
lagging to much behind and cannot satisfy causal reads. In this case
error handling is needed by the application in case a standby is
lagging to much behind.

That's one of my concerns about this patch now: it is trying to do too
much. I think that there is definitely a need for both things:
applications may be fine to pay the lagging price when remote_apply is
used by not having an extra error handling layer, or they cannot
accept a perhaps large of lag and are ready to have an extra error
handling layer to manage read failures on standbys. So I would
recommend a split to begin with:
1) Addition of remote_apply in synchronous_commit, which would be
quite a simple patch, and I am convinced that there are benefits in
having that. Compared to the previous patch sent, a standby message is
sent back to the master once COMMIT has been applied, accelerating
things a bit.
2) Patch for causal reads, with all its extra parametrization logic
and stuff to select standby candidates.

Here is as well a more detailed review...

Regression tests are failing, rules.sql is generating diffs because
pg_stat_replication is changed.

CausalReadsWaitForLSN() should be called for 2PC I think, for PREPARE,
COMMIT PREPARED and ROLLBACK PREPARED. WAL replay for 2PC should also
call XLogRequestWalReceiverReply() when needed.

The new parameters are missing from postgresql.conf.sample.

+static bool
+SyncRepCheckEarlyExit(void)
+{
Doing this refactoring would actually make sense as a separate patch I
think, and that would simplify the core patch for causal reads.
+For this reason we say that the causal reads guarantee only holds as
+long as the absolute difference between the system clocks of the
+machines is no more than max_clock_skew.  The theory is that NTP makes
+it possible to reason about the maximum possible clock difference
+between machines and choose a value that allows for a much larger
+difference.  However, we do make a best effort attempt to detect
+misconfigured systems as described above, to catch the case of servers
+not running ntp a correctly configured ntp daemon, or with a clock so
+far out of whack that ntp refuses to fix it.
Just wondering how this reacts when standby and master are on
different timezones. I know of two ways to measure WAL lag: one is
what you are doing, by using a timestamp and rely on the two servers
to be in sync at clock level. The second is in bytes with a WAL
quantity, though it is less user-friendly to set up, say max_wal_lag
or similar, symbolized by the number of WAL segments the standby is
lagging behind, the concerns regarding clock sync across nodes go
away. To put it short, I find the timestamp approach easy to set up
and understand for the user, but not really solid as it depends much
on the state dependency between different hosts, while a difference
between flush and apply LSN positions is a quite independent concept.
So basically what would be used as a base comparison is not the
timestamp of the transaction commit but the flush LSN at the moment
commit has been replayed.
+   /*
+    * If a causal_reads_timeout is configured, it is used instead of
+    * wal_sender_timeout.  Ideally we'd use causal_reads_timeout / 2 +
+    * allowance for network latency, but since walreceiver can become quite
+    * bogged down fsyncing WAL we allow more tolerance.  (This could be
+    * tightened up once standbys hand writing off to the WAL writer).
+    */
+   if (causal_reads_timeout != 0)
+       allowed_time = causal_reads_timeout;
+   else
+       allowed_time = wal_sender_timeout;
I find that surprising, for two reasons:
1) it seems to me that causal_read_timeout has in concept no relation
with WAL sender process control.
2) A standby should still be able to receive WAL even if it cannot
satisfy causal reads to give it a chance to catch up faster the amount
it is late.
-   SYNCHRONOUS_COMMIT_REMOTE_FLUSH     /* wait for local and remote flush */
+   SYNCHRONOUS_COMMIT_REMOTE_FLUSH,    /* wait for local and remote flush */
+   SYNCHRONOUS_COMMIT_REMOTE_APPLY,    /* wait for local flush and remote
+                                        * apply */
+   SYNCHRONOUS_COMMIT_CONSISTENT_APPLY /* wait for local flusha and remote
+                                          apply with causal consistency */
SYNCHRONOUS_COMMIT_CONSISTENT_APPLY is used nowhere, and there is a
typo s/flusha/flush a/.

I am still digging into the patch, the available/joining/unavailable
logic being quite interesting.
--
Michael

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#42Michael Paquier
michael.paquier@gmail.com
In reply to: Michael Paquier (#41)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Thu, Mar 3, 2016 at 3:34 PM, Michael Paquier
<michael.paquier@gmail.com> wrote:

That's one of my concerns about this patch now: it is trying to do too
much. I think that there is definitely a need for both things:
applications may be fine to pay the lagging price when remote_apply is
used by not having an extra error handling layer, or they cannot
accept a perhaps large of lag and are ready to have an extra error
handling layer to manage read failures on standbys. So I would
recommend a split to begin with:
1) Addition of remote_apply in synchronous_commit, which would be
quite a simple patch, and I am convinced that there are benefits in
having that. Compared to the previous patch sent, a standby message is
sent back to the master once COMMIT has been applied, accelerating
things a bit.

Hm. Looking now at
/messages/by-id/CANP8+j+jCpNoOjc-KQLtt4PDyOX2Sq6wYWqCSy6aaHWkvNa0hw@mail.gmail.com
it would be nice to get a clear solution for it first, though the use
of signals to wake up the WAL receiver and enforce it to send a new
LSN apply position back to the master to unlock it asap does not look
very appealing. Seeing that no patch has been sent for 9.6 regarding
that, it would be better to simply drop this code from the causal-read
patch perhaps...
--
Michael

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#43Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Michael Paquier (#41)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Thu, Mar 3, 2016 at 7:34 PM, Michael Paquier
<michael.paquier@gmail.com> wrote:

On Tue, Mar 1, 2016 at 11:53 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Tue, Mar 1, 2016 at 2:46 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

Hi,

On 2016/02/29 18:05, Thomas Munro wrote:

On Mon, Feb 29, 2016 at 9:05 PM, Amit Langote wrote:

+         servers.  A transaction that is run with
<varname>causal_reads</> set
+         to <literal>on</> is guaranteed either to see the effects of all
+         completed transactions run on the primary with the setting on, or to
+         receive an error "standby is not available for causal reads".

"A transaction that is run" means "A transaction that is run on a
standby", right?

Well, it could be any server, standby or primary. Of course standbys
are the interesting case since it it was already true that if you run
two sequential transactions run on the primary, the second can see the
effect of the first, but I like the idea of a general rule that
applies anywhere, allowing you not to care which server it is.

I meant actually in context of that sentence only.

Ok, here's a new version that includes that change, fixes a conflict
with recent commit 10b48522 and removes an accidental duplicate copy
of the README file.

Finally I got my eyes on this patch.

To put it short, this patch introduces two different concepts:
- addition of a new value, remote_apply, for synchronous_commit, which
is actually where things overlap a bit with the N-sync patch, because
by combining the addition of remote_apply with the N-sync patch, it is
possible to ensure that an LSN is applied to multiple targets instead
of one now. In this case, as the master will wait for the LSN to be
applied on the sync standby, there is no need for the application to
have error handling in case a read transaction is happening on the
standby as the change is already visible on the standby when
committing it on master. However the cost here is that if the standby
node lags behind, it puts some extra wait as well on the master side.
- casual reads, which makes the master able to drop standbys that are
lagging too much behind and let callers of standbys know that it is
lagging to much behind and cannot satisfy causal reads. In this case
error handling is needed by the application in case a standby is
lagging to much behind.

That's one of my concerns about this patch now: it is trying to do too
much. I think that there is definitely a need for both things:
applications may be fine to pay the lagging price when remote_apply is
used by not having an extra error handling layer, or they cannot
accept a perhaps large of lag and are ready to have an extra error
handling layer to manage read failures on standbys. So I would
recommend a split to begin with:
1) Addition of remote_apply in synchronous_commit, which would be
quite a simple patch, and I am convinced that there are benefits in
having that. Compared to the previous patch sent, a standby message is
sent back to the master once COMMIT has been applied, accelerating
things a bit.
2) Patch for causal reads, with all its extra parametrization logic
and stuff to select standby candidates.

Thanks. Yes, this makes a lot of sense. I have done some work on
splitting this up and will post the result soon, along with my
responses to your other feedback.

Here is as well a more detailed review...

Regression tests are failing, rules.sql is generating diffs because
pg_stat_replication is changed.

CausalReadsWaitForLSN() should be called for 2PC I think, for PREPARE,
COMMIT PREPARED and ROLLBACK PREPARED. WAL replay for 2PC should also
call XLogRequestWalReceiverReply() when needed.

The new parameters are missing from postgresql.conf.sample.

+static bool
+SyncRepCheckEarlyExit(void)
+{
Doing this refactoring would actually make sense as a separate patch I
think, and that would simplify the core patch for causal reads.
+For this reason we say that the causal reads guarantee only holds as
+long as the absolute difference between the system clocks of the
+machines is no more than max_clock_skew.  The theory is that NTP makes
+it possible to reason about the maximum possible clock difference
+between machines and choose a value that allows for a much larger
+difference.  However, we do make a best effort attempt to detect
+misconfigured systems as described above, to catch the case of servers
+not running ntp a correctly configured ntp daemon, or with a clock so
+far out of whack that ntp refuses to fix it.
Just wondering how this reacts when standby and master are on
different timezones. I know of two ways to measure WAL lag: one is
what you are doing, by using a timestamp and rely on the two servers
to be in sync at clock level. The second is in bytes with a WAL
quantity, though it is less user-friendly to set up, say max_wal_lag
or similar, symbolized by the number of WAL segments the standby is
lagging behind, the concerns regarding clock sync across nodes go
away. To put it short, I find the timestamp approach easy to set up
and understand for the user, but not really solid as it depends much
on the state dependency between different hosts, while a difference
between flush and apply LSN positions is a quite independent concept.
So basically what would be used as a base comparison is not the
timestamp of the transaction commit but the flush LSN at the moment
commit has been replayed.
+   /*
+    * If a causal_reads_timeout is configured, it is used instead of
+    * wal_sender_timeout.  Ideally we'd use causal_reads_timeout / 2 +
+    * allowance for network latency, but since walreceiver can become quite
+    * bogged down fsyncing WAL we allow more tolerance.  (This could be
+    * tightened up once standbys hand writing off to the WAL writer).
+    */
+   if (causal_reads_timeout != 0)
+       allowed_time = causal_reads_timeout;
+   else
+       allowed_time = wal_sender_timeout;
I find that surprising, for two reasons:
1) it seems to me that causal_read_timeout has in concept no relation
with WAL sender process control.
2) A standby should still be able to receive WAL even if it cannot
satisfy causal reads to give it a chance to catch up faster the amount
it is late.
-   SYNCHRONOUS_COMMIT_REMOTE_FLUSH     /* wait for local and remote flush */
+   SYNCHRONOUS_COMMIT_REMOTE_FLUSH,    /* wait for local and remote flush */
+   SYNCHRONOUS_COMMIT_REMOTE_APPLY,    /* wait for local flush and remote
+                                        * apply */
+   SYNCHRONOUS_COMMIT_CONSISTENT_APPLY /* wait for local flusha and remote
+                                          apply with causal consistency */
SYNCHRONOUS_COMMIT_CONSISTENT_APPLY is used nowhere, and there is a
typo s/flusha/flush a/.

I am still digging into the patch, the available/joining/unavailable
logic being quite interesting.
--
Michael

--
Thomas Munro
http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#44Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Michael Paquier (#41)
4 attachment(s)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Thu, Mar 3, 2016 at 7:34 PM, Michael Paquier
<michael.paquier@gmail.com> wrote:

Finally I got my eyes on this patch.

Thanks, I really appreciate your time and interest. Your feedback
gave me plenty to chew on. See below for replies to both your recent
emails.

To put it short, this patch introduces two different concepts:
- addition of a new value, remote_apply, for synchronous_commit, which
is actually where things overlap a bit with the N-sync patch, because
by combining the addition of remote_apply with the N-sync patch, it is
possible to ensure that an LSN is applied to multiple targets instead
of one now. In this case, as the master will wait for the LSN to be
applied on the sync standby, there is no need for the application to
have error handling in case a read transaction is happening on the
standby as the change is already visible on the standby when
committing it on master. However the cost here is that if the standby
node lags behind, it puts some extra wait as well on the master side.
- casual reads, which makes the master able to drop standbys that are
lagging too much behind and let callers of standbys know that it is
lagging to much behind and cannot satisfy causal reads. In this case
error handling is needed by the application in case a standby is
lagging to much behind.

That's one of my concerns about this patch now: it is trying to do too
much. I think that there is definitely a need for both things:
applications may be fine to pay the lagging price when remote_apply is
used by not having an extra error handling layer, or they cannot
accept a perhaps large of lag and are ready to have an extra error
handling layer to manage read failures on standbys. So I would
recommend a split to begin with:
1) Addition of remote_apply in synchronous_commit, which would be
quite a simple patch, and I am convinced that there are benefits in
having that. Compared to the previous patch sent, a standby message is
sent back to the master once COMMIT has been applied, accelerating
things a bit.
2) Patch for causal reads, with all its extra parametrization logic
and stuff to select standby candidates.

Agreed. I have split this work up into four patches which apply on
top of each other, and provide something (hopefully) useful at each
stage. Namely:

0001-remote-apply.patch: This adds synchronous_commit = remote_apply.
It works by setting a new bit in commit records to say that the
primary requests feedback when the record is applied. When the
recovery process sees this bit, it wakes the walreceiver process and
asks it to send a reply to the primary ASAP.

0002-replay-lag.patch: This adds a time-based estimate of the current
apply lag on each standby. The value is visible in
pg_stat_replication. It uses a combination of commit timestamps and
periodic LSN->time samples from keepalive messages, so that replay lag
is computed even when there are no commits, for example during bulk
data loads. This builds on the previous patch, because it makes use
of the mechanism whereby the recovery process can ask the walreceiver
to send replies whenever it applies WAL records for which it has
timestamps.

0003-refactor-syncrep-exit.patch: This moves the code to handle
syncrep wait cancellation into a function, as you suggested.
Conceptually independent of the previous two patches, but presented as
a patch on top of the previous patches and separately from
causal-reads.patch for ease of review.

0004-causal-reads.patch: This adds the causal reads feature, building
on top of the other three patches. From the remote-apply patch it
uses the mechanism for asking for apply feedback. From the replay-lag
patch it uses the mechanism for tracking apply lag. It also uses
syncrep wait cancellation code.

Here is as well a more detailed review...

Regression tests are failing, rules.sql is generating diffs because
pg_stat_replication is changed.

Fixed, thanks.

CausalReadsWaitForLSN() should be called for 2PC I think, for PREPARE,
COMMIT PREPARED and ROLLBACK PREPARED. WAL replay for 2PC should also
call XLogRequestWalReceiverReply() when needed.

Ah yes, agreed for COMMIT PREPARED. Fixed.

But why for PREPARE TRANSACTION or ROLLBACK PREPARED? Those don't
generate visible effects that others might expect to see on a standby,
so I don't think that is necessary.

The new parameters are missing from postgresql.conf.sample.

Fixed, thanks.

+static bool
+SyncRepCheckEarlyExit(void)
+{
Doing this refactoring would actually make sense as a separate patch I
think, and that would simplify the core patch for causal reads.

Agreed -- see 0003-refactor-syncrep-exit.patch.

+For this reason we say that the causal reads guarantee only holds as
+long as the absolute difference between the system clocks of the
+machines is no more than max_clock_skew.  The theory is that NTP makes
+it possible to reason about the maximum possible clock difference
+between machines and choose a value that allows for a much larger
+difference.  However, we do make a best effort attempt to detect
+misconfigured systems as described above, to catch the case of servers
+not running ntp a correctly configured ntp daemon, or with a clock so
+far out of whack that ntp refuses to fix it.

Just wondering how this reacts when standby and master are on
different timezones. I know of two ways to measure WAL lag: one is
what you are doing, by using a timestamp and rely on the two servers
to be in sync at clock level. The second is in bytes with a WAL
quantity, though it is less user-friendly to set up, say max_wal_lag
or similar, symbolized by the number of WAL segments the standby is
lagging behind, the concerns regarding clock sync across nodes go
away. To put it short, I find the timestamp approach easy to set up
and understand for the user, but not really solid as it depends much
on the state dependency between different hosts, while a difference
between flush and apply LSN positions is a quite independent concept.
So basically what would be used as a base comparison is not the
timestamp of the transaction commit but the flush LSN at the moment
commit has been replayed.

I'm not actually using either of those approaches. Postgres already
periodically captures (time, WAL end) pairs on the primary, and feeds
them to the standby. With this patch set, the standby records these
samples in a circular buffer, and when the recovery process eventually
applies sampled WAL positions, it feeds the associated timestamps back
to the primary in unsolicited keepalive reply messages. Then the
primary can judge how long it took the standby to apply the sampled
WAL position. It doesn't suffer from clock skew problems because it's
comparing two timestamps obtained from the system clock on the same
box. On the other hand, the time reported includes an extra network
latency term as the reply message has to reach the primary. That
might be a good thing, depending on your purpose, as it reflects the
time you'd be waiting for syncrep/causal reads on that standby, which
includes such latency.

I had to implement this because, in an early prototype of the causal
reads feature, the system would choke on large data load transactions
even though the standbys were able to keep up. Lag could only be
measured when commit records came along. That meant that a very big
data load was indistinguishable from a system not applying fast
enough, and the standby would drop to causal reads unavailable state
spuriously. I considered interpolating WAL positions in between
commit records, but couldn't see how to make it work. I considered
injecting extra timestamps into the WAL periodically, but I didn't
need anything persistent. I just needed a way to collect occasional
(time, LSN) samples in memory and feed the times back at the
appropriate moments as the WAL is replayed.

+   /*
+    * If a causal_reads_timeout is configured, it is used instead of
+    * wal_sender_timeout.  Ideally we'd use causal_reads_timeout / 2 +
+    * allowance for network latency, but since walreceiver can become quite
+    * bogged down fsyncing WAL we allow more tolerance.  (This could be
+    * tightened up once standbys hand writing off to the WAL writer).
+    */
+   if (causal_reads_timeout != 0)
+       allowed_time = causal_reads_timeout;
+   else
+       allowed_time = wal_sender_timeout;
I find that surprising, for two reasons:
1) it seems to me that causal_read_timeout has in concept no relation
with WAL sender process control.
2) A standby should still be able to receive WAL even if it cannot
satisfy causal reads to give it a chance to catch up faster the amount
it is late.

This is only used to detect dead/disconnected/unreachable standbys,
not to handle standbys that are merely not keeping up with replication
traffic. To get disconnected this way, a standby has to be not
responding to keepalive pings. When dealing with a causal reads
standby, we want to cap the time we're prepared to hang around waiting
for a zombie standby before nuking it. If we didn't have that change,
then we would stop waiting for slow standbys (not applying fast
enough) in a timely fashion, but crashed/disconnected standbys would
make our commits hang for the potentially much longer
wal_sender_timeout, which seems silly. Zombie servers shouldn't be
able to interfere with your commits for longer periods of time than
lagging servers.

-   SYNCHRONOUS_COMMIT_REMOTE_FLUSH     /* wait for local and remote flush */
+   SYNCHRONOUS_COMMIT_REMOTE_FLUSH,    /* wait for local and remote flush */
+   SYNCHRONOUS_COMMIT_REMOTE_APPLY,    /* wait for local flush and remote
+                                        * apply */
+   SYNCHRONOUS_COMMIT_CONSISTENT_APPLY /* wait for local flusha and remote
+                                          apply with causal consistency */
SYNCHRONOUS_COMMIT_CONSISTENT_APPLY is used nowhere, and there is a
typo s/flusha/flush a/.

I am still digging into the patch, the available/joining/unavailable
logic being quite interesting.

I hope the new patch set makes that job easier...

On Thu, Mar 3, 2016 at 8:00 PM, Michael Paquier
<michael.paquier@gmail.com> wrote:

Hm. Looking now at
/messages/by-id/CANP8+j+jCpNoOjc-KQLtt4PDyOX2Sq6wYWqCSy6aaHWkvNa0hw@mail.gmail.com
it would be nice to get a clear solution for it first, though the use
of signals to wake up the WAL receiver and enforce it to send a new
LSN apply position back to the master to unlock it asap does not look
very appealing. Seeing that no patch has been sent for 9.6 regarding
that, it would be better to simply drop this code from the causal-read
patch perhaps...

The signalling mechanism is still present. Since that discussion I
had the idea of using a different (non-overloaded) signal and
unblocking it only while actually waiting for the network, so there
should be no signal delivery interrupting blocking disk IO operations.
But I certainly agree that the disk IO should be moved out of the
walreceiver process and in to the WAL writer process, for several
reasons. This patch series doesn't need that to work though.

Here's a recap of how the signal is used, in the context of the
communication introduced by each patch:

0001-remote-apply.patch
* the primary sets a new bit XACT_COMPLETION_SYNC_APPLY_FEEDBACK in
commit records' xinfo when synchronous_commit = remote_apply
* the WAL record is transported walsender->walreceiver as usual
* the recovery process eventually applies the commit record, and, when
it sees that bit, it wakes the walreceiver with SIGUSR2
* when walreceiver receives the signal it generates a keepalive reply
to report the latest apply position
* the walsender uses that information to release syncrep waiters (just
as it always did)

0002-replay-lag.patch
* the walreceiver records (time, lsn) pairs into a circular buffer as
they arrive
* the recovery process, when it discovers that it has replayed an LSN
associated with a time in the circular buffer, uses the same signal as
used in 0001-remote-apply.patch to ask walreceiver to report this
progress
* the walreceiver includes the last replay timestamp in a new field
added to the reply message

0004-causal-reads.patch
* reuses the above to implement causal_reads mode:
XACT_COMPLETION_SYNC_APPLY_FEEDBACK is now also set if causal_reads is
on, because it too needs to know when commit records are applied
* the rest of the patch implements the new state machine, lease
control and waiting machinery

Some alternatives to that signal could include: a pipe between
recovery and walreceiver that could be multiplexed with network IO
(which may be tricky to implement cross-platform, I don't know, but I
don't see anything similar in Postgres), or a Postgres latch that
could be multiplexed with network IO using some future primitive as
suggested elsewhere (but would ultimately still involve a signal).
There would be plenty of scope for evolution, for example when
integrating with the standby WAL writer work, but the signal approach
seemed simple and effective for now and certainly isn't the first case
of backends signalling each other.

Thanks,

--
Thomas Munro
http://www.enterprisedb.com

Attachments:

0001-remote-apply.patchapplication/octet-stream; name=0001-remote-apply.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index a09ceb2..b21b4c0 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -2091,7 +2091,7 @@ include_dir 'conf.d'
         Specifies whether transaction commit will wait for WAL records
         to be written to disk before the command returns a <quote>success</>
         indication to the client.  Valid values are <literal>on</>,
-        <literal>remote_write</>, <literal>local</>, and <literal>off</>.
+        <literal>remote_write</>, <literal> remote_apply</>, <literal>local</>, and <literal>off</>.
         The default, and safe, setting
         is <literal>on</>.  When <literal>off</>, there can be a delay between
         when success is reported to the client and when the transaction is
@@ -2125,6 +2125,10 @@ include_dir 'conf.d'
         ensure data preservation even if the standby instance of
         <productname>PostgreSQL</> were to crash, but not if the standby
         suffers an operating-system-level crash.
+        When set to <literal>remote_apply</>, commits will wait until a reply
+        from the current synchronous stanbyindicates it has received the
+        commit record of the transaction and applied it, so that it has become
+        visible to queries.
        </para>
        <para>
         When synchronous
diff --git a/doc/src/sgml/high-availability.sgml b/doc/src/sgml/high-availability.sgml
index 6cb690c..2600fba 100644
--- a/doc/src/sgml/high-availability.sgml
+++ b/doc/src/sgml/high-availability.sgml
@@ -1081,6 +1081,9 @@ primary_slot_name = 'node_a_slot'
     WAL record is then sent to the standby. The standby sends reply
     messages each time a new batch of WAL data is written to disk, unless
     <varname>wal_receiver_status_interval</> is set to zero on the standby.
+    In the case that <varname>synchronous_commit</> is set to
+    <literal>remote_apply</>, the standby sends reply messages when the commit
+    record is replayed, making the transaction visible.
     If the standby is the first matching standby, as specified in
     <varname>synchronous_standby_names</> on the primary, the reply
     messages from that standby will be used to wake users waiting for
@@ -1107,6 +1110,14 @@ primary_slot_name = 'node_a_slot'
    </para>
 
    <para>
+    Setting <varname>synchronous_commit</> to <literal>remote_apply</> will
+    cause each commit to wait until the current synchronous standby reports
+    that it has replayed the transaction, making it visible to user queries.
+    In simple cases, this allows for load balancing with causal consistency
+    on a single hot standby.
+   </para>
+
+   <para>
     Users will stop waiting if a fast shutdown is requested.  However, as
     when using asynchronous replication, the server will not fully
     shutdown until all outstanding WAL records are transferred to the currently
@@ -1160,9 +1171,10 @@ primary_slot_name = 'node_a_slot'
     <title>Planning for High Availability</title>
 
    <para>
-    Commits made when <varname>synchronous_commit</> is set to <literal>on</>
-    or <literal>remote_write</> will wait until the synchronous standby responds. The response
-    may never occur if the last, or only, standby should crash.
+    Commits made when <varname>synchronous_commit</> is set to <literal>on</>,
+    <literal>remote_write</> or <literal>remote_apply</> will wait until the
+    synchronous standby responds. The response may never occur if the last, or
+    only, standby should crash.
    </para>
 
    <para>
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index b0d5440..59066e6 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -5117,6 +5117,13 @@ XactLogCommitRecord(TimestampTz commit_time,
 		xl_xinfo.xinfo |= XACT_COMPLETION_FORCE_SYNC_COMMIT;
 
 	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this commit is applied.
+	*/
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
+	/*
 	 * Relcache invalidations requires information about the current database
 	 * and so does logical decoding.
 	 */
@@ -5452,6 +5459,13 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 	if (XactCompletionForceSyncCommit(parsed->xinfo))
 		XLogFlush(lsn);
 
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit = remote_apply), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 /*
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 00f139a..0519357 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -346,6 +346,12 @@ static XLogRecPtr RedoRecPtr;
 static bool doPageWrites;
 
 /*
+ * doRequestWalReceiverReply is used by recovery code to ask the main recovery
+ * loop to trigger a walreceiver reply.
+ */
+static bool doRequestWalReceiverReply;
+
+/*
  * RedoStartLSN points to the checkpoint's REDO location which is specified
  * in a backup label file, backup history file or control file. In standby
  * mode, XLOG streaming usually starts from the position where an invalid
@@ -6909,6 +6915,19 @@ StartupXLOG(void)
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
 				SpinLockRelease(&XLogCtl->info_lck);
 
+				/*
+				 * If rm_redo reported that it applied a commit record that
+				 * the master is waiting for by calling
+				 * XLogRequestWalReceiverReply, then we wake up the receiver
+				 * so that it notices the updated lastReplayedEndRecPtr and
+				 * sends a reply to the master.
+				 */
+				if (doRequestWalReceiverReply)
+				{
+					doRequestWalReceiverReply = false;
+					WalRcvWakeup();
+				}
+
 				/* Remember this record as the last-applied one */
 				LastRec = ReadRecPtr;
 
@@ -11630,3 +11649,13 @@ SetWalWriterSleeping(bool sleeping)
 	XLogCtl->WalWriterSleeping = sleeping;
 	SpinLockRelease(&XLogCtl->info_lck);
 }
+
+/*
+ * Called by redo code to indicate that the xlog replay loop should wake up
+ * the walreceiver process so that a reply can be sent to the primary.
+ */
+void
+XLogRequestWalReceiverReply(void)
+{
+	doRequestWalReceiverReply = true;
+}
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 92faf4e..4565348 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -416,6 +416,7 @@ SyncRepReleaseWaiters(void)
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
+	int			numapply = 0;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
@@ -462,12 +463,18 @@ SyncRepReleaseWaiters(void)
 		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
 		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
 	}
+	if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+	{
+		walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+		numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY);
+	}
 
 	LWLockRelease(SyncRepLock);
 
-	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X",
+	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%x",
 		 numwrite, (uint32) (MyWalSnd->write >> 32), (uint32) MyWalSnd->write,
-	   numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush);
+		 numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush,
+		 numapply, (uint32) (MyWalSnd->apply >> 32), (uint32) MyWalSnd->apply);
 
 	/*
 	 * If we are managing the highest priority standby, though we weren't
@@ -728,6 +735,9 @@ assign_synchronous_commit(int newval, void *extra)
 		case SYNCHRONOUS_COMMIT_REMOTE_FLUSH:
 			SyncRepWaitMode = SYNC_REP_WAIT_FLUSH;
 			break;
+		case SYNCHRONOUS_COMMIT_REMOTE_APPLY:
+			SyncRepWaitMode = SYNC_REP_WAIT_APPLY;
+			break;
 		default:
 			SyncRepWaitMode = SYNC_REP_NO_WAIT;
 			break;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 7b36e02..c19842e 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -101,6 +101,7 @@ static uint32 recvOff = 0;
  */
 static volatile sig_atomic_t got_SIGHUP = false;
 static volatile sig_atomic_t got_SIGTERM = false;
+static volatile sig_atomic_t got_SIGUSR2 = false;
 
 /*
  * LogstreamResult indicates the byte positions that we have already
@@ -150,9 +151,27 @@ static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
 /* Signal handlers */
 static void WalRcvSigHupHandler(SIGNAL_ARGS);
 static void WalRcvSigUsr1Handler(SIGNAL_ARGS);
+static void WalRcvSigUsr2Handler(SIGNAL_ARGS);
 static void WalRcvShutdownHandler(SIGNAL_ARGS);
 static void WalRcvQuickDieHandler(SIGNAL_ARGS);
 
+static void WalRcvBlockSigUsr2(void)
+{
+	sigset_t mask;
+
+	sigemptyset(&mask);
+	sigaddset(&mask, SIGUSR2);
+	sigprocmask(SIG_BLOCK, &mask, NULL);
+}
+
+static void WalRcvUnblockSigUsr2(void)
+{
+	sigset_t mask;
+
+	sigemptyset(&mask);
+	sigaddset(&mask, SIGUSR2);
+	sigprocmask(SIG_UNBLOCK, &mask, NULL);
+}
 
 static void
 ProcessWalRcvInterrupts(void)
@@ -200,6 +219,7 @@ WalReceiverMain(void)
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz last_recv_timestamp;
 	bool		ping_sent;
+	bool		forceReply;
 
 	/*
 	 * WalRcv should be set up already (if we are a backend, we inherit this
@@ -268,7 +288,7 @@ WalReceiverMain(void)
 	pqsignal(SIGALRM, SIG_IGN);
 	pqsignal(SIGPIPE, SIG_IGN);
 	pqsignal(SIGUSR1, WalRcvSigUsr1Handler);
-	pqsignal(SIGUSR2, SIG_IGN);
+	pqsignal(SIGUSR2, WalRcvSigUsr2Handler);
 
 	/* Reset some signals that are accepted by postmaster but not here */
 	pqsignal(SIGCHLD, SIG_DFL);
@@ -299,6 +319,10 @@ WalReceiverMain(void)
 	/* Unblock signals (they were blocked when the postmaster forked us) */
 	PG_SETMASK(&UnBlockSig);
 
+	/* Block SIGUSR2 (we unblock it only during network waits). */
+	WalRcvBlockSigUsr2();
+	got_SIGUSR2 = false;
+
 	/* Establish the connection to the primary for XLOG streaming */
 	EnableWalRcvImmediateExit();
 	walrcv_connect(conninfo);
@@ -408,7 +432,9 @@ WalReceiverMain(void)
 				}
 
 				/* Wait a while for data to arrive */
+				WalRcvUnblockSigUsr2();
 				len = walrcv_receive(NAPTIME_PER_CYCLE, &buf);
+				WalRcvBlockSigUsr2();
 				if (len != 0)
 				{
 					/*
@@ -439,11 +465,21 @@ WalReceiverMain(void)
 							endofwal = true;
 							break;
 						}
+						WalRcvUnblockSigUsr2();
 						len = walrcv_receive(0, &buf);
+						WalRcvBlockSigUsr2();
+					}
+
+					if (got_SIGUSR2)
+					{
+						/* The recovery process asked us to force a reply. */
+						got_SIGUSR2 = false;
+						forceReply = true;
 					}
 
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(false, false);
+					XLogWalRcvSendReply(forceReply, false);
+					forceReply = false;
 
 					/*
 					 * If we've written some records, flush them to disk and
@@ -498,7 +534,14 @@ WalReceiverMain(void)
 						}
 					}
 
-					XLogWalRcvSendReply(requestReply, requestReply);
+					if (got_SIGUSR2)
+					{
+						/* The recovery process asked us to force a reply. */
+						got_SIGUSR2 = false;
+						forceReply = true;
+					}
+					XLogWalRcvSendReply(requestReply || forceReply, requestReply);
+					forceReply = false;
 					XLogWalRcvSendHSFeedback(false);
 				}
 			}
@@ -740,6 +783,13 @@ WalRcvSigUsr1Handler(SIGNAL_ARGS)
 	errno = save_errno;
 }
 
+/* SIGUSR2: used to receive wakeups from recovery */
+static void
+WalRcvSigUsr2Handler(SIGNAL_ARGS)
+{
+	got_SIGUSR2 = true;
+}
+
 /* SIGTERM: set flag for main loop, or shutdown immediately if safe */
 static void
 WalRcvShutdownHandler(SIGNAL_ARGS)
@@ -1222,6 +1272,22 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 }
 
 /*
+ * Wake up the walreceiver if it happens to be blocked in walrcv_receive,
+ * and tell it that a commit record has been applied.
+ *
+ * This is called by the startup process whenever interesting xlog records
+ * are applied, so that walreceiver can check if it needs to send an apply
+ * notification back to the master which may be waiting in a COMMIT with
+ * synchronous_commit = remote_apply.
+ */
+void
+WalRcvWakeup(void)
+{
+	if (WalRcv->pid != 0)
+		kill(WalRcv->pid, SIGUSR2);
+}
+
+/*
  * Return a string constant representing the state. This is used
  * in system functions and views, and should *not* be translated.
  */
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index ea5a09a..a8eaa5f 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -345,12 +345,13 @@ static const struct config_enum_entry constraint_exclusion_options[] = {
 };
 
 /*
- * Although only "on", "off", "remote_write", and "local" are documented, we
- * accept all the likely variants of "on" and "off".
+ * Although only "on", "off", "remote_apply", "remote_write", and "local" are
+ * documented, we accept all the likely variants of "on" and "off".
  */
 static const struct config_enum_entry synchronous_commit_options[] = {
 	{"local", SYNCHRONOUS_COMMIT_LOCAL_FLUSH, false},
 	{"remote_write", SYNCHRONOUS_COMMIT_REMOTE_WRITE, false},
+	{"remote_apply", SYNCHRONOUS_COMMIT_REMOTE_APPLY, false},
 	{"on", SYNCHRONOUS_COMMIT_ON, false},
 	{"off", SYNCHRONOUS_COMMIT_OFF, false},
 	{"true", SYNCHRONOUS_COMMIT_ON, true},
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index ee3d378..085099c 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -177,7 +177,7 @@
 					# (change requires restart)
 #fsync = on				# turns forced synchronization on or off
 #synchronous_commit = on		# synchronization level;
-					# off, local, remote_write, or on
+					# off, local, remote_write, remote_apply, or on
 #wal_sync_method = fsync		# the default is the first option
 					# supported by the operating system:
 					#   open_datasync
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index ebeb582..ed8d22c 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -60,7 +60,9 @@ typedef enum
 	SYNCHRONOUS_COMMIT_LOCAL_FLUSH,		/* wait for local flush only */
 	SYNCHRONOUS_COMMIT_REMOTE_WRITE,	/* wait for local flush and remote
 										 * write */
-	SYNCHRONOUS_COMMIT_REMOTE_FLUSH		/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_FLUSH,	/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_APPLY		/* wait for local flush and remote
+										 * apply */
 }	SyncCommitLevel;
 
 /* Define the default setting for synchonous_commit */
@@ -144,10 +146,13 @@ typedef void (*SubXactCallback) (SubXactEvent event, SubTransactionId mySubid,
  * EOXact... routines which run at the end of the original transaction
  * completion.
  */
+#define XACT_COMPLETION_SYNC_APPLY_FEEDBACK		(1U << 29)
 #define XACT_COMPLETION_UPDATE_RELCACHE_FILE	(1U << 30)
 #define XACT_COMPLETION_FORCE_SYNC_COMMIT		(1U << 31)
 
 /* Access macros for above flags */
+#define XactCompletionSyncApplyFeedback(xinfo) \
+	(!!(xinfo & XACT_COMPLETION_SYNC_APPLY_FEEDBACK))
 #define XactCompletionRelcacheInitFileInval(xinfo) \
 	(!!(xinfo & XACT_COMPLETION_UPDATE_RELCACHE_FILE))
 #define XactCompletionForceSyncCommit(xinfo) \
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index ecd30ce..68e20e4 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -268,6 +268,8 @@ extern bool CheckPromoteSignal(void);
 extern void WakeupRecovery(void);
 extern void SetWalWriterSleeping(bool sleeping);
 
+extern void XLogRequestWalReceiverReply(void);
+
 extern void assign_max_wal_size(int newval, void *extra);
 extern void assign_checkpoint_completion_target(double newval, void *extra);
 
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index 96e059b..28b68f6 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -23,8 +23,9 @@
 #define SYNC_REP_NO_WAIT		-1
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
+#define SYNC_REP_WAIT_APPLY		2
 
-#define NUM_SYNC_REP_WAIT_MODE	2
+#define NUM_SYNC_REP_WAIT_MODE	3
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 6eacb09..3294df9 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -162,5 +162,6 @@ extern void RequestXLogStreaming(TimeLineID tli, XLogRecPtr recptr,
 extern XLogRecPtr GetWalRcvWriteRecPtr(XLogRecPtr *latestChunkStart, TimeLineID *receiveTLI);
 extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
+extern void WalRcvWakeup(void);
 
 #endif   /* _WALRECEIVER_H */
0002-replay-lag.patchapplication/octet-stream; name=0002-replay-lag.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 85459d0..818b952 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -820,6 +820,12 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       standby server</entry>
     </row>
     <row>
+     <entry><structfield>replay_lag</></entry>
+     <entry><type>interval</></entry>
+     <entry>Estimated time taken for recent WAL records to be replayed on this
+      standby server</entry>
+    </row>
+    <row>
      <entry><structfield>sync_priority</></entry>
      <entry><type>integer</></entry>
      <entry>Priority of this standby server for being chosen as the
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 59066e6..019a1af 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -5460,6 +5460,12 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 		XLogFlush(lsn);
 
 	/*
+	 * Record the primary's timestamp for the commit record, so it can be used
+	 * for tracking replay lag.
+	 */
+	SetXLogReplayTimestamp(parsed->xact_time);
+
+	/*
 	 * If asked by the primary (because someone is waiting for a synchronous
 	 * commit = remote_apply), we will need to ask walreceiver to send a
 	 * reply immediately.
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 0519357..64e05bf 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -81,6 +81,8 @@ extern uint32 bootstrap_data_checksum_version;
 #define PROMOTE_SIGNAL_FILE		"promote"
 #define FALLBACK_PROMOTE_SIGNAL_FILE "fallback_promote"
 
+/* Size of the circular buffer of timestamped LSNs. */
+#define MAX_TIMESTAMPED_LSNS 8192
 
 /* User-settable parameters */
 int			max_wal_size = 64;	/* 1 GB */
@@ -363,6 +365,13 @@ static bool doRequestWalReceiverReply;
  */
 static XLogRecPtr RedoStartLSN = InvalidXLogRecPtr;
 
+/*
+ * LastReplayedTimestamp can be set by redo handlers when they apply a record
+ * that carries a timestamp, by calling SetXLogReplayedTimestamp.  The xlog
+ * apply loop can then update the value in shared memory.
+ */
+static TimestampTz LastReplayedTimestamp = 0;
+
 /*----------
  * Shared-memory data structures for XLOG control
  *
@@ -637,6 +646,21 @@ typedef struct XLogCtlData
 	/* current effective recovery target timeline */
 	TimeLineID	RecoveryTargetTLI;
 
+	/* timestamp from the most recently applied record associated with a timestamp. */
+	TimestampTz lastReplayedTimestamp;
+
+	/*
+	 * We maintain a circular buffer of LSNs and associated timestamps.
+	 * Walreceiver writes into it using information from timestamps, and the
+	 * startup recovery process reads from it and notifies walreceiver when
+	 * LSNs are replayed so that the timestamps can eventually be fed back to
+	 * the upstream server, to track lag.
+	 */
+	Index			timestampedLsnRead;
+	Index			timestampedLsnWrite;
+	XLogRecPtr		timestampedLsn[MAX_TIMESTAMPED_LSNS];
+	TimestampTz		timestampedLsnTime[MAX_TIMESTAMPED_LSNS];
+
 	/*
 	 * timestamp of when we started replaying the current chunk of WAL data,
 	 * only relevant for replication or archive recovery
@@ -6907,20 +6931,51 @@ StartupXLOG(void)
 				error_context_stack = errcallback.previous;
 
 				/*
-				 * Update lastReplayedEndRecPtr after this record has been
-				 * successfully replayed.
+				 * Update lastReplayedEndRecPtr and lastReplayedTimestamp
+				 * after this record has been successfully replayed.
 				 */
 				SpinLockAcquire(&XLogCtl->info_lck);
 				XLogCtl->lastReplayedEndRecPtr = EndRecPtr;
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
+				if (LastReplayedTimestamp != 0)
+				{
+					/* If replaying a record produced a timestamp, use that. */
+					XLogCtl->lastReplayedTimestamp = LastReplayedTimestamp;
+					LastReplayedTimestamp = 0;
+				}
+				else
+				{
+					/*
+					 * If we have applied LSNs associated with timestamps
+					 * received by walreceiver, then use the recorded
+					 * timestamp.  We consume from the read end of the
+					 * circular buffer.
+					 */
+					while (XLogCtl->timestampedLsnRead !=
+						   XLogCtl->timestampedLsnWrite &&
+						   XLogCtl->timestampedLsn[XLogCtl->timestampedLsnRead]
+						   <= EndRecPtr)
+					{
+						if (XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead] >
+							XLogCtl->lastReplayedTimestamp)
+						{
+							XLogCtl->lastReplayedTimestamp =
+								XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead];
+							doRequestWalReceiverReply = true;
+						}
+						XLogCtl->timestampedLsnRead =
+							(XLogCtl->timestampedLsnRead + 1) % MAX_TIMESTAMPED_LSNS;
+					}
+				}
 				SpinLockRelease(&XLogCtl->info_lck);
 
 				/*
 				 * If rm_redo reported that it applied a commit record that
 				 * the master is waiting for by calling
-				 * XLogRequestWalReceiverReply, then we wake up the receiver
-				 * so that it notices the updated lastReplayedEndRecPtr and
-				 * sends a reply to the master.
+				 * XLogRequestWalReceiverReply, or we encountered a WAL
+				 * location that was associated with a timestamp above, then
+				 * we wake up the receiver so that it notices the updated
+				 * lastReplayedEndRecPtr and sends a reply to the master.
 				 */
 				if (doRequestWalReceiverReply)
 				{
@@ -11659,3 +11714,91 @@ XLogRequestWalReceiverReply(void)
 {
 	doRequestWalReceiverReply = true;
 }
+
+/*
+ * Record the timestamp that is associated with a WAL position.
+ *
+ * This is called by walreceiver on standby servers when keepalive messages
+ * arrive, using timestamps generated on the primary server.  The timestamp
+ * will be sent back to the primary server when the standby had applied this
+ * WAL position.  The primary can use the elapsed time to estimate the replay
+ * lag.
+ */
+void
+SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn)
+{
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn == XLogCtl->lastReplayedEndRecPtr)
+	{
+		/*
+		 * That is the last replayed LSN: we are fully replayed, so we can
+		 * update the replay timestamp immediately.
+		 */
+		XLogCtl->lastReplayedTimestamp = timestamp;
+	}
+	else
+	{
+		/*
+		 * There is WAL still to be applied.  We will associate the timestamp
+		 * with this WAL position and wait for it to be replayed.  We add it
+		 * at the 'write' end of the circular buffer of LSN/timestamp
+		 * mappings, which the replay loop will eventually read.
+		 */
+		Index w = XLogCtl->timestampedLsnWrite;
+		Index r = XLogCtl->timestampedLsnRead;
+
+		XLogCtl->timestampedLsn[w] = lsn;
+		XLogCtl->timestampedLsnTime[w] = timestamp;
+
+		/* Advance the write point. */
+		w = (w + 1) % MAX_TIMESTAMPED_LSNS;
+		XLogCtl->timestampedLsnWrite = w;
+		if (w == r)
+		{
+			/*
+			 * The buffer is full.  Advance the read point (throwing away
+			 * oldest values; we will begin to overestimate replay lag, until
+			 * lag decreases to a size our buffer can manage, or the next
+			 * commit record is replayed).
+			 */
+			r = (r + 1) % MAX_TIMESTAMPED_LSNS;
+			XLogCtl->timestampedLsnRead = r;
+		}
+	}
+	SpinLockRelease(&XLogCtl->info_lck);
+}
+
+/*
+ * Set the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary.  This can be called by redo handlers that have
+ * an appropriate timestamp (currently only commit records).  Updating the
+ * shared memory value is deferred until after the redo handler returns.
+ */
+void
+SetXLogReplayTimestamp(TimestampTz timestamp)
+{
+	LastReplayedTimestamp = timestamp;
+}
+
+/*
+ * Get the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary, and also the most recently applied LSN.  (Note
+ * that the timestamp and the LSN don't necessarily relate to the same
+ * record.)
+ *
+ * This is similar to GetLatestXTime, except that it is not only advanced by
+ * commit records (see SetXLogReplayTimestampAtLsn).
+ */
+TimestampTz
+GetXLogReplayTimestamp(XLogRecPtr *lsn)
+{
+	TimestampTz result;
+
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn)
+		*lsn = XLogCtl->lastReplayedEndRecPtr;
+	result = XLogCtl->lastReplayedTimestamp;
+	SpinLockRelease(&XLogCtl->info_lck);
+
+	return result;
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index abf9a70..f5a7c5c 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -661,6 +661,7 @@ CREATE VIEW pg_stat_replication AS
             W.write_location,
             W.flush_location,
             W.replay_location,
+	    W.replay_lag,
             W.sync_priority,
             W.sync_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index c19842e..d061dfe 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -85,6 +85,8 @@ walrcv_disconnect_type walrcv_disconnect = NULL;
 
 #define NAPTIME_PER_CYCLE 100	/* max sleep time between cycles (100ms) */
 
+#define MIN_TIME_BETWEEN_TIMESTAMPED_LSNS 1000 /* 1s */
+
 /*
  * These variables are used similarly to openLogFile/SegNo/Off,
  * but for walreceiver to write the XLOG. recvFileTLI is the TimeLineID
@@ -103,6 +105,8 @@ static volatile sig_atomic_t got_SIGHUP = false;
 static volatile sig_atomic_t got_SIGTERM = false;
 static volatile sig_atomic_t got_SIGUSR2 = false;
 
+static bool reply_sent_on_recovery_activity = false;
+
 /*
  * LogstreamResult indicates the byte positions that we have already
  * written/fsynced.
@@ -144,7 +148,7 @@ static void WalRcvDie(int code, Datum arg);
 static void XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len);
 static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
-static void XLogWalRcvSendReply(bool force, bool requestReply);
+static void XLogWalRcvSendReply(bool force, bool requestReply, bool includeApplyTimestamp);
 static void XLogWalRcvSendHSFeedback(bool immed);
 static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
 
@@ -219,7 +223,7 @@ WalReceiverMain(void)
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz last_recv_timestamp;
 	bool		ping_sent;
-	bool		forceReply;
+	bool		timestampedWalApplied;
 
 	/*
 	 * WalRcv should be set up already (if we are a backend, we inherit this
@@ -472,14 +476,15 @@ WalReceiverMain(void)
 
 					if (got_SIGUSR2)
 					{
-						/* The recovery process asked us to force a reply. */
+						/* The recovery process asked us to report an applied timestamp. */
 						got_SIGUSR2 = false;
-						forceReply = true;
+						timestampedWalApplied = true;
+						reply_sent_on_recovery_activity = true;
 					}
 
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(forceReply, false);
-					forceReply = false;
+					XLogWalRcvSendReply(timestampedWalApplied, false, timestampedWalApplied);
+					timestampedWalApplied = false;
 
 					/*
 					 * If we've written some records, flush them to disk and
@@ -536,12 +541,14 @@ WalReceiverMain(void)
 
 					if (got_SIGUSR2)
 					{
-						/* The recovery process asked us to force a reply. */
+						/* The recovery process asked us to report an apply timestamp. */
 						got_SIGUSR2 = false;
-						forceReply = true;
+						timestampedWalApplied = true;
+						reply_sent_on_recovery_activity = true;
 					}
-					XLogWalRcvSendReply(requestReply || forceReply, requestReply);
-					forceReply = false;
+					XLogWalRcvSendReply(requestReply || timestampedWalApplied, requestReply,
+										timestampedWalApplied);
+					timestampedWalApplied = false;
 					XLogWalRcvSendHSFeedback(false);
 				}
 			}
@@ -879,6 +886,8 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 			}
 		case 'k':				/* Keepalive */
 			{
+				bool reportApplyTimestamp = false;
+
 				/* copy message to StringInfo */
 				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
 				if (len != hdrlen)
@@ -895,9 +904,22 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 
 				ProcessWalSndrMessage(walEnd, sendTime);
 
-				/* If the primary requested a reply, send one immediately */
-				if (replyRequested)
-					XLogWalRcvSendReply(true, false);
+				/*
+				 * If no apply timestamps have been sent at the request of the
+				 * recovery process since we last received a keepalive, then
+				 * we will send one now.  This allows us to feed back
+				 * timestamps in response to pings if we are idle or if the
+				 * recovery process is somehow blocked, but we don't want to
+				 * do that if it's actively applying and periodically waking
+				 * us up.
+				 */
+				if (!reply_sent_on_recovery_activity)
+					reportApplyTimestamp = true;
+				reply_sent_on_recovery_activity = false;
+
+				/* If the primary requested a reply, send one immediately. */
+				if (replyRequested || reportApplyTimestamp)
+					XLogWalRcvSendReply(true, false, reportApplyTimestamp);
 				break;
 			}
 		default:
@@ -1060,7 +1082,7 @@ XLogWalRcvFlush(bool dying)
 		/* Also let the master know that we made some progress */
 		if (!dying)
 		{
-			XLogWalRcvSendReply(false, false);
+			XLogWalRcvSendReply(false, false, false);
 			XLogWalRcvSendHSFeedback(false);
 		}
 	}
@@ -1078,15 +1100,20 @@ XLogWalRcvFlush(bool dying)
  * If 'requestReply' is true, requests the server to reply immediately upon
  * receiving this message. This is used for heartbearts, when approaching
  * wal_receiver_timeout.
+ *
+ * If 'reportApplyTimestamp' is true, the latest apply timestamp is included.
+ * This is set to true only when this function is called after the recovery
+ * process has replayed a record with an associated timestamp.
  */
 static void
-XLogWalRcvSendReply(bool force, bool requestReply)
+XLogWalRcvSendReply(bool force, bool requestReply, bool reportApplyTimestamp)
 {
 	static XLogRecPtr writePtr = 0;
 	static XLogRecPtr flushPtr = 0;
 	XLogRecPtr	applyPtr;
 	static TimestampTz sendTime = 0;
 	TimestampTz now;
+	TimestampTz applyTimestamp = 0;
 
 	/*
 	 * If the user doesn't want status to be reported to the master, be sure
@@ -1102,10 +1129,8 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	 * We can compare the write and flush positions to the last message we
 	 * sent without taking any lock, but the apply position requires a spin
 	 * lock, so we don't check that unless something else has changed or 10
-	 * seconds have passed.  This means that the apply log position will
-	 * appear, from the master's point of view, to lag slightly, but since
-	 * this is only for reporting purposes and only on idle systems, that's
-	 * probably OK.
+	 * seconds have passed, or the force flag has been set (which happens when
+	 * apply feedback has been requested by the primary).
 	 */
 	if (!force
 		&& writePtr == LogstreamResult.Write
@@ -1118,7 +1143,10 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	/* Construct a new message */
 	writePtr = LogstreamResult.Write;
 	flushPtr = LogstreamResult.Flush;
-	applyPtr = GetXLogReplayRecPtr(NULL);
+	if (reportApplyTimestamp)
+		applyTimestamp = GetXLogReplayTimestamp(&applyPtr);
+	else
+		applyPtr = GetXLogReplayRecPtr(NULL);
 
 	resetStringInfo(&reply_message);
 	pq_sendbyte(&reply_message, 'r');
@@ -1126,6 +1154,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	pq_sendint64(&reply_message, flushPtr);
 	pq_sendint64(&reply_message, applyPtr);
 	pq_sendint64(&reply_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&reply_message, TimestampTzToIntegerTimestamp(applyTimestamp));
 	pq_sendbyte(&reply_message, requestReply ? 1 : 0);
 
 	/* Send it */
@@ -1230,8 +1259,8 @@ static void
 ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 {
 	WalRcvData *walrcv = WalRcv;
-
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
+	static TimestampTz lastRecordedTimestamp = 0;
 
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
@@ -1242,6 +1271,18 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
 	SpinLockRelease(&walrcv->mutex);
 
+	/*
+	 * Remember primary's timestamp at this WAL location.  We throw away
+	 * samples if they are coming too fast because we don't want to fill up
+	 * the finite circular buffer and have to throw away older samples.
+	 */
+	if (lastRecordedTimestamp < TimestampTzPlusMilliseconds(sendTime,
+															-MIN_TIME_BETWEEN_TIMESTAMPED_LSNS))
+	{
+		SetXLogReplayTimestampAtLsn(sendTime, walEnd);
+		lastRecordedTimestamp = sendTime;
+	}
+
 	if (log_min_messages <= DEBUG2)
 	{
 		char	   *sendtime;
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index c03e045..5bb142d 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1543,15 +1543,29 @@ ProcessStandbyReplyMessage(void)
 	XLogRecPtr	writePtr,
 				flushPtr,
 				applyPtr;
+	int			applyLagMs;
 	bool		replyRequested;
+	TimestampTz now = GetCurrentTimestamp();
+	TimestampTz applyTimestamp;
 
 	/* the caller already consumed the msgtype byte */
 	writePtr = pq_getmsgint64(&reply_message);
 	flushPtr = pq_getmsgint64(&reply_message);
 	applyPtr = pq_getmsgint64(&reply_message);
 	(void) pq_getmsgint64(&reply_message);		/* sendTime; not used ATM */
+	applyTimestamp = IntegerTimestampToTimestampTz(pq_getmsgint64(&reply_message));
 	replyRequested = pq_getmsgbyte(&reply_message);
 
+	/* Compute the apply lag in milliseconds. */
+	if (applyTimestamp == 0)
+		applyLagMs = -1;
+	else
+#ifdef HAVE_INT64_TIMESTAMP
+		applyLagMs = (now - applyTimestamp) / 1000;
+#else
+		applyLagMs = (now - applyTimestamp) * 1000.0;
+#endif
+
 	elog(DEBUG2, "write %X/%X flush %X/%X apply %X/%X%s",
 		 (uint32) (writePtr >> 32), (uint32) writePtr,
 		 (uint32) (flushPtr >> 32), (uint32) flushPtr,
@@ -1573,6 +1587,8 @@ ProcessStandbyReplyMessage(void)
 		walsnd->write = writePtr;
 		walsnd->flush = flushPtr;
 		walsnd->apply = applyPtr;
+		if (applyLagMs >= 0)
+			walsnd->applyLagMs = applyLagMs;
 		SpinLockRelease(&walsnd->mutex);
 	}
 
@@ -2743,7 +2759,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	8
+#define PG_STAT_GET_WAL_SENDERS_COLS	9
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2791,6 +2807,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		XLogRecPtr	write;
 		XLogRecPtr	flush;
 		XLogRecPtr	apply;
+		int			applyLagMs;
 		int			priority;
 		WalSndState state;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
@@ -2805,6 +2822,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
+		applyLagMs = walsnd->applyLagMs;
 		priority = walsnd->sync_standby_priority;
 		SpinLockRelease(&walsnd->mutex);
 
@@ -2839,6 +2857,23 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				nulls[5] = true;
 			values[5] = LSNGetDatum(apply);
 
+			if (applyLagMs < 0)
+				nulls[6] = true;
+			else
+			{
+				Interval *applyLagInterval = palloc(sizeof(Interval));
+
+				applyLagInterval->month = 0;
+				applyLagInterval->day = 0;
+#ifdef HAVE_INT64_TIMESTAMP
+				applyLagInterval->time = applyLagMs * 1000;
+#else
+				applyLagInterval->time = applyLagMs / 1000.0;
+#endif
+				nulls[6] = false;
+				values[6] = IntervalPGetDatum(applyLagInterval);
+			}
+
 			/*
 			 * Treat a standby such as a pg_basebackup background process
 			 * which always returns an invalid flush location, as an
@@ -2846,18 +2881,18 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 			 */
 			priority = XLogRecPtrIsInvalid(walsnd->flush) ? 0 : priority;
 
-			values[6] = Int32GetDatum(priority);
+			values[7] = Int32GetDatum(priority);
 
 			/*
 			 * More easily understood version of standby state. This is purely
 			 * informational, not different from priority.
 			 */
 			if (priority == 0)
-				values[7] = CStringGetTextDatum("async");
+				values[8] = CStringGetTextDatum("async");
 			else if (walsnd == sync_standby)
-				values[7] = CStringGetTextDatum("sync");
+				values[8] = CStringGetTextDatum("sync");
 			else
-				values[7] = CStringGetTextDatum("potential");
+				values[8] = CStringGetTextDatum("potential");
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
diff --git a/src/backend/utils/adt/timestamp.c b/src/backend/utils/adt/timestamp.c
index c4f556a..2032f61 100644
--- a/src/backend/utils/adt/timestamp.c
+++ b/src/backend/utils/adt/timestamp.c
@@ -1611,6 +1611,20 @@ IntegerTimestampToTimestampTz(int64 timestamp)
 #endif
 
 /*
+ * TimestampTzToIntegerTimestamp -- convert a native timestamp to int64 format
+ *
+ * When compiled with --enable-integer-datetimes, this is implemented as a
+ * no-op macro.
+ */
+#ifndef HAVE_INT64_TIMESTAMP
+int64
+TimestampTzToIntegerTimestamp(TimestampTz timestamp)
+{
+	return timestamp * 1000000;
+}
+#endif
+
+/*
  * TimestampDifference -- convert the difference between two timestamps
  *		into integer seconds and microseconds
  *
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 68e20e4..efb9719 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -236,6 +236,9 @@ extern void GetXLogReceiptTime(TimestampTz *rtime, bool *fromStream);
 extern XLogRecPtr GetXLogReplayRecPtr(TimeLineID *replayTLI);
 extern XLogRecPtr GetXLogInsertRecPtr(void);
 extern XLogRecPtr GetXLogWriteRecPtr(void);
+extern void SetXLogReplayTimestamp(TimestampTz timestamp);
+extern void SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn);
+extern TimestampTz GetXLogReplayTimestamp(XLogRecPtr *lsn);
 extern bool RecoveryIsPaused(void);
 extern void SetRecoveryPause(bool recoveryPause);
 extern TimestampTz GetLatestXTime(void);
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index cbbb883..03a5f22 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2710,7 +2710,7 @@ DATA(insert OID = 1936 (  pg_stat_get_backend_idset		PGNSP PGUID 12 1 100 0 0 f
 DESCR("statistics: currently active backend IDs");
 DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f f f t s r 1 0 2249 "23" "{23,26,23,26,25,25,25,16,1184,1184,1184,1184,869,25,23,28,28,16,25,25,23,16,25}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{pid,datid,pid,usesysid,application_name,state,query,waiting,xact_start,query_start,backend_start,state_change,client_addr,client_hostname,client_port,backend_xid,backend_xmin,ssl,sslversion,sslcipher,sslbits,sslcompression,sslclientdn}" _null_ _null_ pg_stat_get_activity _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active backends");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,23,25}" "{o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25}" "{o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 7794aa5..88ccdf0 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -46,6 +46,7 @@ typedef struct WalSnd
 	XLogRecPtr	write;
 	XLogRecPtr	flush;
 	XLogRecPtr	apply;
+	int			applyLagMs;
 
 	/* Protects shared variables shown above. */
 	slock_t		mutex;
diff --git a/src/include/utils/timestamp.h b/src/include/utils/timestamp.h
index fbead3a..297e151 100644
--- a/src/include/utils/timestamp.h
+++ b/src/include/utils/timestamp.h
@@ -227,9 +227,11 @@ extern bool TimestampDifferenceExceeds(TimestampTz start_time,
 #ifndef HAVE_INT64_TIMESTAMP
 extern int64 GetCurrentIntegerTimestamp(void);
 extern TimestampTz IntegerTimestampToTimestampTz(int64 timestamp);
+extern int64 TimestampTzToIntegerTimestamp(TimestampTz timestamp);
 #else
 #define GetCurrentIntegerTimestamp()	GetCurrentTimestamp()
 #define IntegerTimestampToTimestampTz(timestamp) (timestamp)
+#define TimestampTzToIntegerTimestamp(timestamp) (timestamp)
 #endif
 
 extern TimestampTz time_t_to_timestamptz(pg_time_t tm);
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 81bc5c9..6f80548 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1760,11 +1760,12 @@ pg_stat_replication| SELECT s.pid,
     w.write_location,
     w.flush_location,
     w.replay_location,
+    w.replay_lag,
     w.sync_priority,
     w.sync_state
    FROM pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, waiting, xact_start, query_start, backend_start, state_change, client_addr, client_hostname, client_port, backend_xid, backend_xmin, ssl, sslversion, sslcipher, sslbits, sslcompression, sslclientdn),
     pg_authid u,
-    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, sync_priority, sync_state)
+    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state)
   WHERE ((s.usesysid = u.oid) AND (s.pid = w.pid));
 pg_stat_ssl| SELECT s.pid,
     s.ssl,
0003-refactory-syncrep-exit.patchapplication/octet-stream; name=0003-refactory-syncrep-exit.patchDownload
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 4565348..bbc28a7 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -83,6 +83,64 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * ===========================================================
  */
 
+static bool
+SyncRepCheckEarlyExit(void)
+{
+	/*
+	 * If a wait for synchronous replication is pending, we can neither
+	 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
+	 * lead the client to believe that the transaction aborted, which
+	 * is not true: it's already committed locally. The former is no good
+	 * either: the client has requested synchronous replication, and is
+	 * entitled to assume that an acknowledged commit is also replicated,
+	 * which might not be true. So in this case we issue a WARNING (which
+	 * some clients may be able to interpret) and shut off further output.
+	 * We do NOT reset ProcDiePending, so that the process will die after
+	 * the commit is cleaned up.
+	 */
+	if (ProcDiePending)
+	{
+		ereport(WARNING,
+				(errcode(ERRCODE_ADMIN_SHUTDOWN),
+				 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * It's unclear what to do if a query cancel interrupt arrives.  We
+	 * can't actually abort at this point, but ignoring the interrupt
+	 * altogether is not helpful, so we just terminate the wait with a
+	 * suitable warning.
+	 */
+	if (QueryCancelPending)
+	{
+		QueryCancelPending = false;
+		ereport(WARNING,
+				(errmsg("canceling wait for synchronous replication due to user request"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * If the postmaster dies, we'll probably never get an
+	 * acknowledgement, because all the wal sender processes will exit. So
+	 * just bail out.
+	 */
+	if (!PostmasterIsAlive())
+	{
+		ProcDiePending = true;
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	return false;
+}
+
 /*
  * Wait for synchronous replication, if requested by user.
  *
@@ -180,57 +238,9 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 		if (syncRepState == SYNC_REP_WAIT_COMPLETE)
 			break;
 
-		/*
-		 * If a wait for synchronous replication is pending, we can neither
-		 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
-		 * lead the client to believe that the transaction aborted, which
-		 * is not true: it's already committed locally. The former is no good
-		 * either: the client has requested synchronous replication, and is
-		 * entitled to assume that an acknowledged commit is also replicated,
-		 * which might not be true. So in this case we issue a WARNING (which
-		 * some clients may be able to interpret) and shut off further output.
-		 * We do NOT reset ProcDiePending, so that the process will die after
-		 * the commit is cleaned up.
-		 */
-		if (ProcDiePending)
-		{
-			ereport(WARNING,
-					(errcode(ERRCODE_ADMIN_SHUTDOWN),
-					 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit())
 			break;
-		}
-
-		/*
-		 * It's unclear what to do if a query cancel interrupt arrives.  We
-		 * can't actually abort at this point, but ignoring the interrupt
-		 * altogether is not helpful, so we just terminate the wait with a
-		 * suitable warning.
-		 */
-		if (QueryCancelPending)
-		{
-			QueryCancelPending = false;
-			ereport(WARNING,
-					(errmsg("canceling wait for synchronous replication due to user request"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * If the postmaster dies, we'll probably never get an
-		 * acknowledgement, because all the wal sender processes will exit. So
-		 * just bail out.
-		 */
-		if (!PostmasterIsAlive())
-		{
-			ProcDiePending = true;
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
-			break;
-		}
 
 		/*
 		 * Wait on latch.  Any condition that should wake us up will set the
0004-causal-reads.patchapplication/octet-stream; name=0004-causal-reads.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index b21b4c0..de24f1c 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -2668,6 +2668,35 @@ include_dir 'conf.d'
      across the cluster without problems if that is required.
     </para>
 
+    <sect2 id="runtime-config-replication-all">
+     <title>All Servers</title>
+     <para>
+      These parameters can be set on the primary or any standby.
+     </para>
+     <variablelist>
+      <varlistentry id="guc-causal-reads" xreflabel="causal_reads">
+       <term><varname>causal_reads</varname> (<type>boolean</type>)
+       <indexterm>
+        <primary><varname>causal_reads</> configuration parameter</primary>
+       </indexterm>
+       </term>
+       <listitem>
+        <para>
+         Enables causal consistency between transactions run on different
+         servers.  A transaction that is run on a standby
+         with <varname>causal_reads</> set to <literal>on</> is guaranteed
+         either to see the effects of all completed transactions run on the
+         primary with the setting on, or to receive an error "standby is not
+         available for causal reads".  Note that both transactions involved in
+         a causal dependency (a write on the primary followed by a read on any
+         server which must see the write) must be run with the setting on.
+         See <xref linkend="causal-reads"> for more details.
+        </para>
+       </listitem>
+      </varlistentry>
+     </variablelist>     
+    </sect2>
+
     <sect2 id="runtime-config-replication-sender">
      <title>Sending Server(s)</title>
 
@@ -2899,6 +2928,48 @@ include_dir 'conf.d'
       </listitem>
      </varlistentry>
 
+     <varlistentry>
+      <term><varname>causal_reads_timeout</varname> (<type>integer</type>)
+       <indexterm>
+        <primary><varname>causal_reads_timeout</> configuration parameter</primary>
+       </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies the maximum replay lag the primary will tolerate from a
+        standby before dropping it from the set of standbys available for
+        causal reads.
+       </para>
+       <para>
+        This setting is also used to control the <firstterm>leases</> used to
+        maintain the causal reads guarantee.  It must be set to a value which
+        is at least 4 times the maximum possible difference in system clocks
+        between the primary and standby servers, as described
+        in <xref linkend="causal-reads">.
+       </para>
+      </listitem>
+     </varlistentry>
+
+     <varlistentry id="guc-causal-reads-standby-names" xreflabel="causal-reads-standby-names">
+      <term><varname>causal_reads_standby_names</varname> (<type>string</type>)
+      <indexterm>
+       <primary><varname>causal_reads_standby_names</> configuration parameter</primary>
+      </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies a comma-separated list of standby names that can support
+        <firstterm>causal reads</>, as described in
+        <xref linkend="causal-reads">.  Follows the same convention
+        as <link linkend="guc-synchronous-standby-names"><literal>synchronous_standby_name</></>.
+        The default is <literal>*</>, matching all standbys.
+       </para>
+       <para>
+        This setting has no effect if <varname>causal_reads_timeout</> is not set.
+       </para>
+      </listitem>
+     </varlistentry>
+
      </variablelist>
     </sect2>
 
diff --git a/doc/src/sgml/high-availability.sgml b/doc/src/sgml/high-availability.sgml
index 2600fba..a01df0d 100644
--- a/doc/src/sgml/high-availability.sgml
+++ b/doc/src/sgml/high-availability.sgml
@@ -1115,6 +1115,9 @@ primary_slot_name = 'node_a_slot'
     that it has replayed the transaction, making it visible to user queries.
     In simple cases, this allows for load balancing with causal consistency
     on a single hot standby.
+    (See also
+    <xref linkend="causal-reads"> which deals with multiple standbys and
+    standby failure.)
    </para>
 
    <para>
@@ -1233,6 +1236,119 @@ primary_slot_name = 'node_a_slot'
    </sect3>
   </sect2>
 
+  <sect2 id="causal-reads">
+   <title>Causal reads</title>
+   <indexterm>
+    <primary>causal reads</primary>
+    <secondary>in standby</secondary>
+   </indexterm>
+
+   <para>
+    The causal reads feature allows read-only queries to run on hot standby
+    servers without exposing stale data to the client, providing a form of
+    causal consistency.  Transactions can run on any standby with the
+    following guarantee about the visibility of preceding transactions: If you
+    set <varname>causal_reads</> to <literal>on</> in any pair of consecutive
+    transactions tx1, tx2 where tx2 begins after tx1 successfully returns,
+    then tx2 will either see tx1 or fail with a new error "standby is not
+    available for causal reads", no matter which server it runs on.  Although
+    the guarantee is expressed in terms of two individual transactions, the
+    GUC can also be set at session, role or system level to make the guarantee
+    generally, allowing for load balancing of applications that were not
+    designed with load balancing in mind.
+   </para>
+
+   <para>
+    In order to enable the feature, <varname>causal_reads_timeout</> must be
+    set to a non-zero value on the primary server.  The
+    GUC <varname>causal_reads_standby_names</> can be used to limit the set of
+    standbys that can join the dynamic set of causal reads standbys by
+    providing a comma-separated list of application names.  By default, all
+    standbys are candidates, if the feature is enabled.
+   </para>
+
+   <para>
+    The current set of servers that the primary considers to be available for
+    causal reads can be seen in
+    the <link linkend="monitoring-stats-views-table"> <literal>pg_stat_replication</></>
+    view.  Administrators, applications and load balancing middleware can use
+    this view to discover standbys that can currently handle causal reads
+    transactions without raising the error.  Since that information is only an
+    instantantaneous snapshot, clients should still be prepared for the error
+    to be raised at any time, and consider redirecting transactions to another
+    standby.
+   </para>
+
+   <para>
+    The advantages of the causal reads feature over simply
+    setting <varname>synchronous_commit</> to <literal>remote_apply</> are:
+    <orderedlist>
+      <listitem>
+       <para>
+        It allows the primary to wait for multiple standbys to replay
+        transactions.
+       </para>
+      </listitem>
+      <listitem>
+       <para>
+        It places a configurable limit on how much replay lag (and therefore
+        delay at commit time) the primary tolerates from standbys before it
+        drops them from the dynamic set of standbys it waits for.
+       </para>   
+      </listitem>
+      <listitem>
+       <para>
+        It upholds the causal reads guarantee during the transitions that
+        occur when new standbys are added or removed from the set of standbys,
+        including scenarios where contact has been lost between the primary
+        and standbys but the standby is still alive and running client
+        queries.
+       </para>
+      </listitem>
+    </orderedlist>
+   </para>
+
+   <para>
+    The protocol used to uphold the guarantee even in the case of network
+    failure depends on the system clocks of the primary and standby servers
+    being synchronized, with an allowance for a difference up to one quarter
+    of <varname>causal_reads_timeout</>.  For example,
+    if <varname>causal_reads_timeout</> is set to <literal>4s</>, then the
+    clocks must not be further than 1 second apart for the guarantee to be
+    upheld reliably during transitions.  The ubiquity of the Network Time
+    Protocol (NTP) on modern operating systems and availability of high
+    quality time servers makes it possible to choose a tolerance significantly
+    higher than the maximum expected clock difference.  An effort is
+    nevertheless made to detect and report misconfigured and faulty systems
+    with clock differences greater than the configured tolerance.
+   </para>
+
+   <note>
+    <para>
+     Current hardware clocks, NTP implementations and public time servers are
+     unlikely to allow the system clocks to differ more than tens or hundreds
+     of milliseconds, and systems synchronized with dedicated local time
+     servers may be considerably more accurate, but you should only consider
+     setting <varname>causal_reads_timeout</> below 4 seconds (allowing up to
+     1 second of clock difference) after researching your time synchronization
+     infrastructure thoroughly.
+    </para>  
+   </note>
+
+   <note>
+    <para>
+      While similar to synchronous replication in the sense that both involve
+      the primary server waiting for responses from standby servers, the
+      causal reads feature is not concerned with avoiding data loss.  A
+      primary configured for causal reads will drop all standbys that stop
+      responding or replay too slowly from the dynamic set that it waits for,
+      so you should consider configuring both synchronous replication and
+      causal reads if you need data loss avoidance guarantees and causal
+      consistency guarantees for load balancing.
+    </para>
+   </note>
+  </sect2>
+
   <sect2 id="continuous-archiving-in-standby">
    <title>Continuous archiving in standby</title>
 
@@ -1581,7 +1697,16 @@ if (!triggered)
     so there will be a measurable delay between primary and standby. Running the
     same query nearly simultaneously on both primary and standby might therefore
     return differing results. We say that data on the standby is
-    <firstterm>eventually consistent</firstterm> with the primary.  Once the
+    <firstterm>eventually consistent</firstterm> with the primary by default.
+    The data visible to a transaction running on a standby can be
+    made <firstterm>causally consistent</> with respect to a transaction that
+    has completed on the primary by setting <varname>causal_reads</>
+    to <literal>on</> in both transactions.  For more details,
+    see <xref linkend="causal-reads">.
+   </para>
+
+   <para>
+    Once the    
     commit record for a transaction is replayed on the standby, the changes
     made by that transaction will be visible to any new snapshots taken on
     the standby.  Snapshots may be taken at the start of each query or at the
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 818b952..5a87f37 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -836,6 +836,17 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
      <entry><type>text</></entry>
      <entry>Synchronous state of this standby server</entry>
     </row>
+    <row>
+     <entry><structfield>causal_reads_state</></entry>
+     <entry><type>text</></entry>
+     <entry>Causal reads state of this standby server.  This field will be
+     non-null only if <varname>cause_reads_timeout</> is set.  If a standby is
+     in <literal>available</> state, then it can currently serve causal reads
+     queries.  If it is not replaying fast enough or not responding to
+     keepalive messages, it will be in <literal>unavailable</> state, and if
+     it is currently transitioning to availability it will be
+     in <literal>joining</> state for a short time.</entry>
+    </row>
    </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 8a22836..1837c1b 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -2090,11 +2090,12 @@ RecordTransactionCommitPrepared(TransactionId xid,
 	END_CRIT_SECTION();
 
 	/*
-	 * Wait for synchronous replication, if required.
+	 * Wait for causal reads and synchronous replication, if required.
 	 *
 	 * Note that at this stage we have marked clog, but still show as running
 	 * in the procarray and continue to hold locks.
 	 */
+	CausalReadsWaitForLSN(recptr);
 	SyncRepWaitForLSN(recptr);
 }
 
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 019a1af..acca499 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,10 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
+	{
+		CausalReadsWaitForLSN(XactLastRecEnd);
 		SyncRepWaitForLSN(XactLastRecEnd);
+	}
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5120,7 +5123,7 @@ XactLogCommitRecord(TimestampTz commit_time,
 	 * Check if the caller would like to ask standbys for immediate feedback
 	 * once this commit is applied.
 	*/
-	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY || causal_reads)
 		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
 
 	/*
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index f5a7c5c..1fb9812 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -663,7 +663,8 @@ CREATE VIEW pg_stat_replication AS
             W.replay_location,
 	    W.replay_lag,
             W.sync_priority,
-            W.sync_state
+            W.sync_state,
+	    W.causal_reads_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
             pg_stat_get_wal_senders() AS W
     WHERE S.usesysid = U.oid AND
diff --git a/src/backend/replication/README.causal_reads b/src/backend/replication/README.causal_reads
new file mode 100644
index 0000000..1fddd62
--- /dev/null
+++ b/src/backend/replication/README.causal_reads
@@ -0,0 +1,193 @@
+The causal reads guarantee says: If you run any two consecutive
+transactions tx1, tx2 where tx1 completes before tx2 begins, with
+causal_reads set to "on" in both transactions, tx2 will see tx1 or
+raise an error to complain that it can't guarantee causal consistency,
+no matter which servers (primary or any standby) you run each
+transaction on.
+
+When both transactions run on the primary, the guarantee is trivially
+upheld.
+
+To deal with read-only physical streaming standbys, the primary keeps
+track of a set of standbys that it considers to be currently
+"available" for causal reads, and sends a stream of "leases" to those
+standbys granting them the right to handle causal reads transactions
+for a short time without any further communication with the primary.
+
+In general, the primary provides the guarantee by waiting for all of
+the "available" standbys to report that they have applied a
+transaction.  However, the set of available standbys is dynamic, and
+things get more complicated during state transitions.  There are two
+types of transitions to consider:
+
+1.  unavailable->joining->available
+
+Standbys start out as "unavailable".  If a standby is unavailable and
+is applying fast enough and matches causal_reads_standby_names, the
+primary transitions it to "available", but first it sets it to
+"joining" until it is sure that any transaction committed while it was
+unavailable has definitely been applied on the standby.  This closes a
+race that would otherwise exist if we moved directly to available
+state: tx1 might not wait for a given standby because it's
+unavailable, then a lease might be granted, and then tx2 might run a
+causal reads transaction without error but see stale data.  The
+joining state acts as an airlock: while in joining state, the primary
+waits for that standby to replay causal reads transactions in
+anticipation of the move to available, but it doesn't progress to
+available state and grant a lease to the standby until everything
+preceding joining state has also been applied.
+
+2.  available->unavailable
+
+If a standby is not applying fast enough or not responding to
+keepalive messages, then the primary kicks that standby out of the
+dynamic set of available standbys, that is, marks it as "unavailable".
+In order to make sure that the standby has started rejecting causal
+reads transactions, it needs to revoke the lease it most recently
+granted.  It does that by waiting for the lease to expire before
+allowing any causal reads commits to return.  (In future there could
+be a fast-path revocation message which waits for a serial-numbered
+acknowledgement to reduce waiting in the case where the standby is
+lagging but still reachable and responding).
+
+The rest of this document illustrates how clock skew affects the
+available->unavailable transition.
+
+The following 4 variables are derived from a single GUC, and these
+values will be used in the following illustrations:
+
+causal_reads_timeout = 4s
+lease_time           = 4s (= causal_reads_timeout)
+keepalive_time       = 2s (= lease_time / 2)
+max_clock_skew       = 1s (= lease_time / 4)
+
+Every keepalive_time, the primary transmits a lease that expires at
+local_clock_time + lease_time - max_clock_skew, shown in the following
+diagram as 't' for transmission time and '|' for expiry time.  If
+contact is lost with a standby, the primary will wait until sent_time
++ lease_time for the most recently granted lease to expire, shown on
+the following diagram 'x', to be sure that the standby's clock has
+reached the expiry time even if its clock differs by up to
+max_clock_skew.  In other words, the primary tells the standby that
+the expiry time is at one time, but it trusts that the standby will
+surely agree if it gives it some extra time.  The extra time is
+max_clock_skew.  If the clocks differ by more than max_clock_skew, all
+bets are off (but see below for attempt to detect obvious cases).
+
+0     1     2     3     4     5     6     7     8     9
+t-----------------|-----x
+            t-----------------|-----x
+                        t-----------------|-----x
+                                    t-----------------|...
+                                                t------...
+
+A standby whose clock is 2 seconds ahead of the primary's clock
+perceives gaps in the stream of leases, and will reject causal_reads
+transactions in those intervals.  The causal reads guarantee is
+upheld, but spurious errors are raised between leases, as a
+consequence of the clock skew being greater than max_clock_skew.  In
+the following diagram 'r' shows reception time, and the timeline along
+the top shows the standby's local clock time.
+
+2     3     4     5     6     7     8     9    10    11
+r-----|
+            r-----|
+                        r-----|
+                                    r-----|
+                                                r-----|
+
+If there were no network latency, a standby whose clock is exactly 1
+second ahead of the primary's clock would perceive the stream of
+leases as being replaced just in time, so there is no gap.  Since in
+reality the time of receipt is some time after the time of
+transmission due to network latency, if the standby's clock is exactly
+1 second behind, then there will be small network-latency-sized gaps
+before the next lease arrives, but still no correctness problem with
+respect to the causal reads guarantee.
+
+1     2     3     4     5     6     7     8     9    10
+r-----------|
+            r-----------|
+                        r-----------|
+                                    r-----------|
+                                                r------...
+
+A standby whose clock is perfectly in sync with the primary's
+perceives the stream of leases overlapping (this matches the primary's
+perception of the leases it sent):
+
+0     1     2     3     4     5     6     7     8     9
+r-----------------|
+            r-----------------|
+                        r-----------------|
+                                    r-----------------|
+                                                r------...
+
+A standby whose clock is exactly 1 second behind the primary's
+perceives the stream of leases as overlapping even more, but the time
+of expiry as judged by the standby is no later than the time the
+primary will wait for if required ('x').  That is, if contact is lost
+with the standby, the primary can still reliably hold up causal reads
+commits until the standby has started raising the error in
+causal_reads transactions.
+
+-1    0     1     2     3     4     5     6     7     8
+r-----------------------|
+            r-----------------------|
+                        r-----------------------|
+                                    r------------------...
+                                                r------...
+
+
+A standby whose clock is 2 seconds behind the primary's would perceive
+the stream of leases overlapping even more, and the primary would no
+longer be able to wait for a lease to expire if it wanted to revoke
+it.  But because the expiry time is after local_clock_time +
+lease_time, the standby can immediately see that its own clock must be
+more than 1 second behind the primary's, so it ignores the lease and
+logs a clock skew warning.  In the following diagram a lease expiry
+time that is obviously generated by a primary with a clock set too far
+in the future compared to the local clock is shown with a '!'.
+
+-2    -1    0     1     2     3     4     5     6     7
+r-----------------------------!
+            r-----------------------------!
+                        r-----------------------------!
+                                    r------------------...
+                                                r------...
+
+A danger window exists when the standby's clock is more than
+max_clock_skew behind the primary's clock, but not more than
+max_clock_skew + network latency time behind.  If the clock difference
+is in that range, then the algorithm presented above which is based on
+time of receipt cannot detect that the local clock is too far behind.
+The consequence of this problem could be as follows:
+
+1.  The standby loses contact with the primary due to a network fault.
+
+2.  The primary decides to drop the standby from the set of available
+    causal reads standbys due to lack of keepalive responses or
+    excessive lag, which necessitates holding up commits of causal
+    reads transactions until the most recently sent lease expires, in
+    the belief that the standby will definitely have started raising
+    the 'causal reads unavailable' error in causal reads transactions
+    by that time, if it is still alive and servicing requests.
+
+3.  The standby still has clients connected and running queries.
+
+4.  Due to clock skew in the problematic range, in the standby's
+    opinion the lease lasts slightly longer than the primary waits.
+
+5.  For a short window at most the duration of the network latency
+    time, clients running causal reads transactions are allowed to see
+    potentially stale data.
+
+For this reason we say that the causal reads guarantee only holds as
+long as the absolute difference between the system clocks of the
+machines is no more than max_clock_skew.  The theory is that NTP makes
+it possible to reason about the maximum possible clock difference
+between machines and choose a value that allows for a much larger
+difference.  However, we do make a best effort attempt to detect
+wildly divergent systems as described above, to catch the case of
+servers not running a correctly configured ntp daemon, or with a clock
+so far out of whack that ntp refuses to fix it.
\ No newline at end of file
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index bbc28a7..4f6b1be 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -57,6 +57,11 @@
 #include "utils/builtins.h"
 #include "utils/ps_status.h"
 
+/* GUC variables */
+int causal_reads_timeout;
+bool causal_reads;
+char *causal_reads_standby_names;
+
 /* User-settable parameters for sync rep */
 char	   *SyncRepStandbyNames;
 
@@ -69,7 +74,7 @@ static int	SyncRepWaitMode = SYNC_REP_NO_WAIT;
 
 static void SyncRepQueueInsert(int mode);
 static void SyncRepCancelWait(void);
-static int	SyncRepWakeQueue(bool all, int mode);
+static int	SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn);
 
 static int	SyncRepGetStandbyPriority(void);
 
@@ -142,6 +147,198 @@ SyncRepCheckEarlyExit(void)
 }
 
 /*
+ * Check if we can stop waiting for causal consistency.  We can stop waiting
+ * when the following conditions are met:
+ *
+ * 1.  All walsenders currently in 'joining' or 'available' state have
+ * applied the target LSN.
+ *
+ * 2.  Any stall periods caused by standbys dropping out of 'available' state
+ * have passed, so that we can be sure that their leases have expired and they
+ * have started rejecting causal reads transactions.
+ *
+ * The output parameter 'waitingFor' is set to the number of nodes we are
+ * currently waiting for.  The output parameters 'stallTimeMillis' is set to
+ * the number of milliseconds we need to wait for to observe any current
+ * commit stall.
+ *
+ * Returns true if commit can return control, because every standby has either
+ * applied the LSN or started rejecting causal_reads transactions.
+ */
+static bool
+CausalReadsCommitCanReturn(XLogRecPtr XactCommitLSN,
+						   int *waitingFor,
+						   long *stallTimeMillis)
+{
+	int i;
+	TimestampTz now;
+
+	/* Count how many joining/available nodes we are waiting for. */
+	*waitingFor = 0;
+	for (i = 0; i < max_wal_senders; ++i)
+	{
+		WalSnd *walsnd = &WalSndCtl->walsnds[i];
+
+		/*
+		 * Assuming atomic read of pid_t, we can check walsnd->pid without
+		 * acquiring the spinlock to avoid memory synchronization costs for
+		 * unused walsender slots.  We see a value that existed sometime at
+		 * least as recently as the last memory barrier.
+		 */
+		if (walsnd->pid != 0)
+		{
+			/*
+			 * We need to hold the spinlock to read LSNs, because we can't be
+			 * sure they can be read atomically.
+			 */
+			SpinLockAcquire(&walsnd->mutex);
+			if (walsnd->pid != 0 && walsnd->causal_reads_state >= WALSNDCRSTATE_JOINING)
+			{
+				if (walsnd->apply < XactCommitLSN)
+					++*waitingFor;
+			}
+			SpinLockRelease(&walsnd->mutex);
+		}
+	}
+
+	/* Check if there is a stall in progress that we need to observe. */
+	now = GetCurrentTimestamp();
+	LWLockAcquire(SyncRepLock, LW_SHARED);
+	if (WalSndCtl->stall_causal_reads_until > now)
+	{
+		long seconds;
+		int usecs;
+
+		/* Compute how long we have to wait, rounded up to nearest ms. */
+		TimestampDifference(now, WalSndCtl->stall_causal_reads_until,
+							&seconds, &usecs);
+		*stallTimeMillis = seconds * 1000 + (usecs + 999) / 1000;
+	}
+	else
+		*stallTimeMillis = 0;
+	LWLockRelease(SyncRepLock);
+
+	/* We are done if we are not waiting for any nodes or stalls. */
+	return *waitingFor == 0 && *stallTimeMillis == 0;
+}
+
+/*
+ * Wait for causal consistency in causal_reads mode, if requested by user.
+ */
+void
+CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN)
+{
+	long stallTimeMillis;
+	int waitingFor;
+	char *ps_display_buffer = NULL;
+
+	/* Leave if we aren't in causal_reads mode. */
+	if (!causal_reads)
+		return;
+
+	for (;;)
+	{
+		/* Reset latch before checking state. */
+		ResetLatch(MyLatch);
+
+		/*
+		 * Join the queue to be woken up if any causal reads joining/available
+		 * standby applies XactCommitLSN or the set of causal reads standbys
+		 * changes (if we aren't already in the queue).  We don't actually know
+		 * if we need to wait for any peers to reach the target LSN yet, but
+		 * we have to register just in case before checking the walsenders'
+		 * state to avoid a race condition that could occur if we did it after
+		 * calling CausalReadsCommitCanReturn.  (SyncRepWaitForLSN doesn't
+		 * have to do this because it can check the highest-seen LSN in
+		 * walsndctl->lsn[mode] which is protected by SyncRepLock, the same
+		 * lock as the queues.  We can't do that here, because there is no
+		 * single highest-seen LSN that is useful.  We must check
+		 * walsnd->apply for all relevant walsenders.  Therefore we must
+		 * register for notifications first, so that we can be notified via
+		 * our latch of any standby applying the LSN we're interested in after
+		 * we check but before we start waiting, or we could wait forever for
+		 * something that has already happened.)
+		 */
+		LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+		if (MyProc->syncRepState != SYNC_REP_WAITING)
+		{
+			MyProc->waitLSN = XactCommitLSN;
+			MyProc->syncRepState = SYNC_REP_WAITING;
+			SyncRepQueueInsert(SYNC_REP_WAIT_CAUSAL_READS);
+			Assert(SyncRepQueueIsOrderedByLSN(SYNC_REP_WAIT_CAUSAL_READS));
+		}
+		LWLockRelease(SyncRepLock);
+
+		/* Check if we're done. */
+		if (CausalReadsCommitCanReturn(XactCommitLSN, &waitingFor, &stallTimeMillis))
+		{
+			SyncRepCancelWait();
+			break;
+		}
+
+		Assert(waitingFor > 0 || stallTimeMillis > 0);
+
+		/* If we aren't actually waiting for any standbys, leave the queue. */
+		if (waitingFor == 0)
+			SyncRepCancelWait();
+
+		/* Update the ps title. */
+		if (update_process_title)
+		{
+			char buffer[80];
+
+			/* Remember the old value if this is our first update. */
+			if (ps_display_buffer == NULL)
+			{
+				int len;
+				const char *ps_display = get_ps_display(&len);
+
+				ps_display_buffer = palloc(len + 1);
+				memcpy(ps_display_buffer, ps_display, len);
+				ps_display_buffer[len] = '\0';
+			}
+
+			snprintf(buffer, sizeof(buffer),
+					 "waiting for %d peer(s) to apply %X/%X%s",
+					 waitingFor,
+					 (uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN,
+					 stallTimeMillis > 0 ? " (stalling)" : "");
+			set_ps_display(buffer, false);
+		}
+
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit()) /* Calls SyncRepCancelWait() if true. */
+			break;
+
+		/*
+		 * If are still waiting for peers, then we wait for any joining or
+		 * available peer to reach the LSN (or possibly stop being in one of
+		 * those states or go away).
+		 *
+		 * If not, there must be a non-zero stall time, so we wait for that to
+		 * elapse.
+		 */
+		if (waitingFor > 0)
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH, -1);
+		else
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH | WL_TIMEOUT,
+					  stallTimeMillis);
+	}
+
+	/* There is no way out of the loop that could leave us in the queue. */
+	Assert(SHMQueueIsDetached(&(MyProc->syncRepLinks)));
+	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
+	MyProc->waitLSN = 0;
+
+	/* Restore the ps display. */
+	if (ps_display_buffer != NULL)
+	{
+		set_ps_display(ps_display_buffer, false);
+		pfree(ps_display_buffer);
+	}
+}
+
+/*
  * Wait for synchronous replication, if requested by user.
  *
  * Initially backends start in state SYNC_REP_NOT_WAITING and then
@@ -413,6 +610,53 @@ SyncRepGetSynchronousStandby(void)
 }
 
 /*
+ * Check if the current WALSender process's application_name matches a name in
+ * causal_reads_standby_names (including '*' for wildcard).
+ */
+bool
+CausalReadsPotentialStandby(void)
+{
+	char *rawstring;
+	List	   *elemlist;
+	ListCell   *l;
+	bool		found = false;
+
+	/* If the feature is disable, then no. */
+	if (causal_reads_timeout == 0)
+		return false;
+
+	/* Need a modifiable copy of string */
+	rawstring = pstrdup(causal_reads_standby_names);
+
+	/* Parse string into list of identifiers */
+	if (!SplitIdentifierString(rawstring, ',', &elemlist))
+	{
+		/* syntax error in list */
+		pfree(rawstring);
+		list_free(elemlist);
+		/* GUC machinery will have already complained - no need to do again */
+		return 0;
+	}
+
+	foreach(l, elemlist)
+	{
+		char	   *standby_name = (char *) lfirst(l);
+
+		if (pg_strcasecmp(standby_name, application_name) == 0 ||
+			pg_strcasecmp(standby_name, "*") == 0)
+		{
+			found = true;
+			break;
+		}
+	}
+
+	pfree(rawstring);
+	list_free(elemlist);
+
+	return found;
+}
+
+/*
  * Update the LSNs on each queue based upon our latest state. This
  * implements a simple policy of first-valid-standby-releases-waiter.
  *
@@ -420,13 +664,15 @@ SyncRepGetSynchronousStandby(void)
  * perhaps also which information we store as well.
  */
 void
-SyncRepReleaseWaiters(void)
+SyncRepReleaseWaiters(bool walsender_cr_available_or_joining)
 {
 	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
 	int			numapply = 0;
+	int			numcausalreadsapply = 0;
+	bool		is_highest_priority_sync_standby;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
@@ -446,13 +692,19 @@ SyncRepReleaseWaiters(void)
 	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
 	syncWalSnd = SyncRepGetSynchronousStandby();
 
-	/* We should have found ourselves at least */
-	Assert(syncWalSnd != NULL);
+	/*
+	 * If we aren't managing the highest priority standby then make a note of
+	 * that so we can announce a takeover in the log if we ever get that job.
+	 */
+	is_highest_priority_sync_standby = syncWalSnd == MyWalSnd;
+	if (!is_highest_priority_sync_standby)
+		announce_next_takeover = true;
 
 	/*
-	 * If we aren't managing the highest priority standby then just leave.
+	 * If we aren't managing the highest priority standby or a standby in
+	 * causal reads 'joining' or 'available' state, then just leave.
 	 */
-	if (syncWalSnd != MyWalSnd)
+	if (!is_highest_priority_sync_standby && !walsender_cr_available_or_joining)
 	{
 		LWLockRelease(SyncRepLock);
 		announce_next_takeover = true;
@@ -461,24 +713,45 @@ SyncRepReleaseWaiters(void)
 
 	/*
 	 * Set the lsn first so that when we wake backends they will release up to
-	 * this location.
+	 * this location.  For the single-standby synchronous commit levels, we
+	 * only do this if we are the current synchronous standby and we are
+	 * advancing the LSN further than it has been advanced before, so that
+	 * SyncRepWaitForLSN can skip waiting in some cases.
 	 */
-	if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
-		numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
-		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+	if (is_highest_priority_sync_standby)
 	{
-		walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
-		numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY);
+		if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
+			numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE,
+										MyWalSnd->write);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
+			numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH,
+										MyWalSnd->flush);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+			numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY,
+										MyWalSnd->apply);
+		}
 	}
 
+	/*
+	 * For causal_reads, all walsenders currently in available or joining
+	 * state must reach the LSN on their own, and standbys will reach LSNs in
+	 * any order.  It doesn't make sense to keep the highest seen LSN in a
+	 * single walsndctl->lsn element.  (CausalReadsWaitForLSN has handling for
+	 * LSNs that have already been reached).
+	 */
+	if (walsender_cr_available_or_joining)
+		numcausalreadsapply =
+			SyncRepWakeQueue(false, SYNC_REP_WAIT_CAUSAL_READS,
+							 MyWalSnd->apply);
+
 	LWLockRelease(SyncRepLock);
 
 	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%x",
@@ -490,7 +763,7 @@ SyncRepReleaseWaiters(void)
 	 * If we are managing the highest priority standby, though we weren't
 	 * prior to this, then announce we are now the sync standby.
 	 */
-	if (announce_next_takeover)
+	if (is_highest_priority_sync_standby && announce_next_takeover)
 	{
 		announce_next_takeover = false;
 		ereport(LOG,
@@ -565,9 +838,8 @@ SyncRepGetStandbyPriority(void)
  * Must hold SyncRepLock.
  */
 static int
-SyncRepWakeQueue(bool all, int mode)
+SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn)
 {
-	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	PGPROC	   *proc = NULL;
 	PGPROC	   *thisproc = NULL;
 	int			numprocs = 0;
@@ -584,7 +856,7 @@ SyncRepWakeQueue(bool all, int mode)
 		/*
 		 * Assume the queue is ordered by LSN
 		 */
-		if (!all && walsndctl->lsn[mode] < proc->waitLSN)
+		if (!all && lsn < proc->waitLSN)
 			return numprocs;
 
 		/*
@@ -644,7 +916,7 @@ SyncRepUpdateSyncStandbysDefined(void)
 			int			i;
 
 			for (i = 0; i < NUM_SYNC_REP_WAIT_MODE; i++)
-				SyncRepWakeQueue(true, i);
+				SyncRepWakeQueue(true, i, InvalidXLogRecPtr);
 		}
 
 		/*
@@ -696,13 +968,31 @@ SyncRepQueueIsOrderedByLSN(int mode)
 #endif
 
 /*
+ * Make sure that CausalReadsWaitForLSN can't return until after the given
+ * lease expiry time has been reached.  In other words, revoke the lease.
+ *
+ * Wake up all backends waiting in CausalReadsWaitForLSN, because the set of
+ * available/joining peers has changed, and there is a new stall time they
+ * need to observe.
+ */
+void
+CausalReadsBeginStall(TimestampTz lease_expiry_time)
+{
+	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+	WalSndCtl->stall_causal_reads_until =
+		Max(WalSndCtl->stall_causal_reads_until, lease_expiry_time);
+	SyncRepWakeQueue(true, SYNC_REP_WAIT_CAUSAL_READS, InvalidXLogRecPtr);
+	LWLockRelease(SyncRepLock);
+}
+
+/*
  * ===========================================================
  * Synchronous Replication functions executed by any process
  * ===========================================================
  */
 
 bool
-check_synchronous_standby_names(char **newval, void **extra, GucSource source)
+check_standby_names(char **newval, void **extra, GucSource source)
 {
 	char	   *rawstring;
 	List	   *elemlist;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index d061dfe..e68384c 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -55,6 +55,7 @@
 #include "libpq/pqformat.h"
 #include "libpq/pqsignal.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "replication/walsender.h"
 #include "storage/ipc.h"
@@ -150,7 +151,8 @@ static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
 static void XLogWalRcvSendReply(bool force, bool requestReply, bool includeApplyTimestamp);
 static void XLogWalRcvSendHSFeedback(bool immed);
-static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
+static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+								  TimestampTz *causalReadsUntil);
 
 /* Signal handlers */
 static void WalRcvSigHupHandler(SIGNAL_ARGS);
@@ -857,6 +859,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 	XLogRecPtr	walEnd;
 	TimestampTz sendTime;
 	bool		replyRequested;
+	TimestampTz causalReadsLease;
 
 	resetStringInfo(&incoming_message);
 
@@ -877,7 +880,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				walEnd = pq_getmsgint64(&incoming_message);
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, NULL);
 
 				buf += hdrlen;
 				len -= hdrlen;
@@ -889,7 +892,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				bool reportApplyTimestamp = false;
 
 				/* copy message to StringInfo */
-				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
+				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char) + sizeof(int64);
 				if (len != hdrlen)
 					ereport(ERROR,
 							(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -901,8 +904,10 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
 				replyRequested = pq_getmsgbyte(&incoming_message);
+				causalReadsLease = IntegerTimestampToTimestampTz(
+					pq_getmsgint64(&incoming_message));
 
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, &causalReadsLease);
 
 				/*
 				 * If no apply timestamps have been sent at the request of the
@@ -1253,15 +1258,52 @@ XLogWalRcvSendHSFeedback(bool immed)
  * Update shared memory status upon receiving a message from primary.
  *
  * 'walEnd' and 'sendTime' are the end-of-WAL and timestamp of the latest
- * message, reported by primary.
+ * message, reported by primary.  'causalReadsLease' is a pointer to
+ * the time the primary promises that this standby can safely claim to be
+ * causally consistent, to 0 if it cannot, or a NULL pointer for no change.
  */
 static void
-ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
+ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+					  TimestampTz *causalReadsLease)
 {
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
 	static TimestampTz lastRecordedTimestamp = 0;
 
+	/* Sanity check for the causalReadsLease time. */
+	if (causalReadsLease != NULL && *causalReadsLease != 0)
+	{
+		/* Deduce max_clock_skew from the causalReadsLease and sendTime. */
+#ifdef HAVE_INT64_TIMESTAMP
+		int64 diffMillis = (*causalReadsLease - sendTime) / 1000;
+#else
+		int64 diffMillis = (*causalReadsLease - sendTime) * 1000;
+#endif
+		int64 max_clock_skew = diffMillis / (CAUSAL_READS_CLOCK_SKEW_RATIO - 1);
+
+		if (sendTime > TimestampTzPlusMilliseconds(lastMsgReceiptTime, max_clock_skew))
+		{
+			/*
+			 * The primary's clock is more than max_clock_skew + network
+			 * latency ahead of the standby's clock.  (If the primary's clock
+			 * is more than max_clock_skew ahead of the standby's clock, but
+			 * by less than the network latency, then there isn't much we can
+			 * do to detect that; but it still seems useful to have this basic
+			 * sanity check for wildly misconfigured servers.)
+			 */
+			elog(LOG, "the primary server's clock time is too far ahead");
+			causalReadsLease = NULL;
+		}
+		/*
+		 * We could also try to detect cases where sendTime is more than
+		 * max_clock_skew in the past according to the standby's clock, but
+		 * that is indistinguishable from network latency/buffering, so we
+		 * could produce misleading error messages; if we do nothing, the
+		 * consequence is 'standby is not available for causal reads' errors
+		 * which should cause the user to investigate.
+		 */
+	}
+
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
 	if (walrcv->latestWalEnd < walEnd)
@@ -1269,6 +1311,8 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->latestWalEnd = walEnd;
 	walrcv->lastMsgSendTime = sendTime;
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
+	if (causalReadsLease != NULL)
+		walrcv->causalReadsLease = *causalReadsLease;
 	SpinLockRelease(&walrcv->mutex);
 
 	/*
diff --git a/src/backend/replication/walreceiverfuncs.c b/src/backend/replication/walreceiverfuncs.c
index 5f6e423..e502f74 100644
--- a/src/backend/replication/walreceiverfuncs.c
+++ b/src/backend/replication/walreceiverfuncs.c
@@ -28,6 +28,7 @@
 #include "replication/walreceiver.h"
 #include "storage/pmsignal.h"
 #include "storage/shmem.h"
+#include "utils/guc.h"
 #include "utils/timestamp.h"
 
 WalRcvData *WalRcv = NULL;
@@ -374,3 +375,21 @@ GetReplicationTransferLatency(void)
 
 	return ms;
 }
+
+/*
+ * Used by snapmgr to check if this standby has a valid lease, granting it the
+ * right to consider itself available for causal reads.
+ */
+bool
+WalRcvCausalReadsAvailable(void)
+{
+	WalRcvData *walrcv = WalRcv;
+	TimestampTz now = GetCurrentTimestamp();
+	bool result;
+
+	SpinLockAcquire(&walrcv->mutex);
+	result = walrcv->causalReadsLease != 0 && now <= walrcv->causalReadsLease;
+	SpinLockRelease(&walrcv->mutex);
+
+	return result;
+}
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 5bb142d..adb8e3e 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -153,9 +153,20 @@ static StringInfoData tmpbuf;
  */
 static TimestampTz last_reply_timestamp = 0;
 
+static TimestampTz last_keepalive_timestamp = 0;
+
 /* Have we sent a heartbeat message asking for reply, since last reply? */
 static bool waiting_for_ping_response = false;
 
+/* How long do need to stay in JOINING state? */
+static XLogRecPtr causal_reads_joining_until = 0;
+
+/* The last causal reads lease sent to the standby. */
+static TimestampTz causal_reads_last_lease = 0;
+
+/* Is this WALSender listed in causal_reads_standby_names? */
+static bool am_potential_causal_reads_standby = false;
+
 /*
  * While streaming WAL in Copy mode, streamingDoneSending is set to true
  * after we have sent CopyDone. We should not send any more CopyData messages
@@ -241,6 +252,57 @@ InitWalSender(void)
 	SendPostmasterSignal(PMSIGNAL_ADVANCE_STATE_MACHINE);
 }
 
+ /*
+ * If we are exiting unexpectedly, we may need to communicate with concurrent
+ * causal_reads commits to maintain the causal consistency guarantee.
+ */
+static void
+PrepareUncleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * We've lost contact with the standby, but it may still be alive.  We
+		 * can't let any causal_reads transactions return until we've stalled
+		 * for long enough for a zombie standby to start raising errors
+		 * because its lease has expired.
+		 */
+		elog(LOG, "standby \"%s\" is lost (no longer available for causal reads)", application_name);
+		CausalReadsBeginStall(causal_reads_last_lease);
+
+		/*
+		 * We set the state to a lower level _after_ beginning the stall,
+		 * otherwise there would be a tiny window where commits could return
+		 * without observing the stall.
+		 */
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
+ * We are shutting down because we received a goodbye message from the
+ * walreceiver.
+ */
+static void
+PrepareCleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * The standby is shutting down, so it won't be running any more
+		 * transactions.  It is therefore safe to stop waiting for it, and no
+		 * stall is necessary.
+		 */
+		elog(LOG, "standby \"%s\" is leaving (no longer available for causal reads)", application_name);
+
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
 /*
  * Clean up after an error.
  *
@@ -264,7 +326,10 @@ WalSndErrorCleanup(void)
 
 	replication_active = false;
 	if (walsender_ready_to_stop)
+	{
+		PrepareUncleanExit();
 		proc_exit(0);
+	}
 
 	/* Revert back to startup state */
 	WalSndSetState(WALSNDSTATE_STARTUP);
@@ -276,6 +341,8 @@ WalSndErrorCleanup(void)
 static void
 WalSndShutdown(void)
 {
+	PrepareUncleanExit();
+
 	/*
 	 * Reset whereToSendOutput to prevent ereport from attempting to send any
 	 * more messages to the standby.
@@ -1386,6 +1453,7 @@ ProcessRepliesIfAny(void)
 		if (r < 0)
 		{
 			/* unexpected error or EOF */
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1402,6 +1470,7 @@ ProcessRepliesIfAny(void)
 		resetStringInfo(&reply_message);
 		if (pq_getmessage(&reply_message, 0))
 		{
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1451,6 +1520,7 @@ ProcessRepliesIfAny(void)
 				 * 'X' means that the standby is closing down the socket.
 				 */
 			case 'X':
+				PrepareCleanExit();
 				proc_exit(0);
 
 			default:
@@ -1582,6 +1652,83 @@ ProcessStandbyReplyMessage(void)
 	 */
 	{
 		WalSnd *walsnd = MyWalSnd;
+		WalSndCausalReadsState causal_reads_state = walsnd->causal_reads_state;
+		bool causal_reads_state_changed = false;
+		bool causal_reads_set_joining_until = false;
+
+		/*
+		 * Handle causal reads state transitions, if a causal_reads_timeout is
+		 * configured, this standby is listed in causal_reads_standby_names,
+		 * and we are a primary database (not a cascading standby).
+		 */
+		if (am_potential_causal_reads_standby &&
+			!am_cascading_walsender &&
+			applyLagMs >= 0)
+		{
+			if (applyLagMs < causal_reads_timeout)
+			{
+				if (causal_reads_state == WALSNDCRSTATE_UNAVAILABLE)
+				{
+					/*
+					 * The standby is applying fast enough.  We can't grant a
+					 * lease yet though, we need to wait for everything that
+					 * was committed while this standby was unavailable to be
+					 * applied first.  We move to joining state while we wait
+					 * for the standby to catch up.
+					 */
+					causal_reads_state = WALSNDCRSTATE_JOINING;
+					causal_reads_set_joining_until = true;
+					causal_reads_state_changed = true;
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING &&
+						 applyPtr >= causal_reads_joining_until)
+				{
+					/*
+					 * The standby has applied everything committed before we
+					 * reached joining state, and has been waiting for remote
+					 * apply on this standby while it's been in joining state,
+					 * so it is safe to move to available state and send a
+					 * lease.
+					 */
+					causal_reads_state = WALSNDCRSTATE_AVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+			else
+			{
+				if (causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+					/*
+					 * We are dropping a causal reads available standby, so we
+					 * mustn't let any commit command that is waiting in
+					 * CausalReadsWaitForLSN return until we are sure that the
+					 * standby definitely knows that it's not available and
+					 * starts raising errors for causal_reads transactions.
+					 * TODO: We could just wait until the standby acks that
+					 * its lease has been cancelled, and start numbering
+					 * keepalives and sending the number back in replies, so
+					 * we know it's acking the right message; then lagging
+					 * standbys would be less disruptive, but for now we just
+					 * wait for the lease to expire, as we do when we lose
+					 * contact with a standby, for the sake of simplicity.
+					 */
+					CausalReadsBeginStall(causal_reads_last_lease);
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING)
+				{
+					/*
+					 * Dropping a joining standby doesn't require a stall,
+					 * because the standby doesn't think it's available, so
+					 * it's already raising the error for causal_reads
+					 * transactions.
+					 */
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+		}
 
 		SpinLockAcquire(&walsnd->mutex);
 		walsnd->write = writePtr;
@@ -1589,11 +1736,33 @@ ProcessStandbyReplyMessage(void)
 		walsnd->apply = applyPtr;
 		if (applyLagMs >= 0)
 			walsnd->applyLagMs = applyLagMs;
+		walsnd->causal_reads_state = causal_reads_state;
 		SpinLockRelease(&walsnd->mutex);
+
+		if (causal_reads_set_joining_until)
+		{
+			/*
+			 * Record the end of the primary's WAL at some arbitrary point
+			 * observed _after_ we moved to joining state (so that causal
+			 * reads commits start waiting, closing a race).  The standby
+			 * won't become available until it has replayed up to here.
+			 */
+			causal_reads_joining_until = GetFlushRecPtr();
+		}
+
+		if (causal_reads_state_changed)
+		{
+			WalSndKeepalive(true);
+			elog(LOG, "standby \"%s\" is %s", application_name,
+				 causal_reads_state == WALSNDCRSTATE_UNAVAILABLE ? "unavailable for causal reads" :
+				 causal_reads_state == WALSNDCRSTATE_JOINING ? "joining as a causal reads standby..." :
+				 causal_reads_state == WALSNDCRSTATE_AVAILABLE ? "available for causal reads" :
+				 "UNKNOWN");
+		}
 	}
 
 	if (!am_cascading_walsender)
-		SyncRepReleaseWaiters();
+		SyncRepReleaseWaiters(MyWalSnd->causal_reads_state >= WALSNDCRSTATE_JOINING);
 
 	/*
 	 * Advance our local xmin horizon when the client confirmed a flush.
@@ -1734,33 +1903,53 @@ ProcessStandbyHSFeedbackMessage(void)
  * If wal_sender_timeout is enabled we want to wake up in time to send
  * keepalives and to abort the connection if wal_sender_timeout has been
  * reached.
+ *
+ * But if causal_reads_timeout is enabled, we override that and send
+ * keepalives at a constant rate to replace expiring leases.
  */
 static long
 WalSndComputeSleeptime(TimestampTz now)
 {
 	long		sleeptime = 10000;		/* 10 s */
 
-	if (wal_sender_timeout > 0 && last_reply_timestamp > 0)
+	if ((wal_sender_timeout > 0 && last_reply_timestamp > 0) ||
+		am_potential_causal_reads_standby)
 	{
 		TimestampTz wakeup_time;
 		long		sec_to_timeout;
 		int			microsec_to_timeout;
 
-		/*
-		 * At the latest stop sleeping once wal_sender_timeout has been
-		 * reached.
-		 */
-		wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-												  wal_sender_timeout);
-
-		/*
-		 * If no ping has been sent yet, wakeup when it's time to do so.
-		 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
-		 * the timeout passed without a response.
-		 */
-		if (!waiting_for_ping_response)
+		if (am_potential_causal_reads_standby)
+		{
+			/*
+			 * Leases last for a period of between 50% and 100% of
+			 * causal_reads_timeout, depending on clock skew, assuming clock
+			 * skew is under the 25% of causal_reads_timeout.  We send new
+			 * leases every half a lease, so that there are no gaps between
+			 * leases.
+			 */
+			wakeup_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+													  causal_reads_timeout /
+													  CAUSAL_READS_KEEPALIVE_RATIO);
+		}
+		else
+		{
+			/*
+			 * At the latest stop sleeping once wal_sender_timeout has been
+			 * reached.
+			 */
 			wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-													  wal_sender_timeout / 2);
+													  wal_sender_timeout);
+
+			/*
+			 * If no ping has been sent yet, wakeup when it's time to do so.
+			 * WalSndKeepaliveIfNecessary() wants to send a keepalive once
+			 * half of the timeout passed without a response.
+			 */
+			if (!waiting_for_ping_response)
+				wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+														  wal_sender_timeout / 2);
+		}
 
 		/* Compute relative time until wakeup. */
 		TimestampDifference(now, wakeup_time,
@@ -1776,20 +1965,33 @@ WalSndComputeSleeptime(TimestampTz now)
 /*
  * Check whether there have been responses by the client within
  * wal_sender_timeout and shutdown if not.
+ *
+ * If causal_reads_timeout is configured we override that, so that
+ * unresponsive standbys are detected sooner.
  */
 static void
 WalSndCheckTimeOut(TimestampTz now)
 {
 	TimestampTz timeout;
+	int allowed_time;
 
 	/* don't bail out if we're doing something that doesn't require timeouts */
 	if (last_reply_timestamp <= 0)
 		return;
 
-	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
-										  wal_sender_timeout);
+	/*
+	 * If a causal_reads_timeout is configured, it is used instead of
+	 * wal_sender_timeout, to limit the time before an unresponsive causal
+	 * reads standby is dropped.
+	 */
+	if (am_potential_causal_reads_standby)
+		allowed_time = causal_reads_timeout;
+	else
+		allowed_time = wal_sender_timeout;
 
-	if (wal_sender_timeout > 0 && now >= timeout)
+	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
+										  allowed_time);
+	if (allowed_time > 0 && now >= timeout)
 	{
 		/*
 		 * Since typically expiration of replication timeout means
@@ -1822,6 +2024,10 @@ WalSndLoop(WalSndSendDataCallback send_data)
 	last_reply_timestamp = GetCurrentTimestamp();
 	waiting_for_ping_response = false;
 
+	/* Check if we are managing potential causal_reads standby. */
+	am_potential_causal_reads_standby = CausalReadsPotentialStandby();
+	elog(WARNING, "am_potential_causal_reads_standby = %d", am_potential_causal_reads_standby);
+
 	/*
 	 * Loop until we reach the end of this timeline or the client requests to
 	 * stop streaming.
@@ -1982,6 +2188,7 @@ InitWalSenderSlot(void)
 			walsnd->flush = InvalidXLogRecPtr;
 			walsnd->apply = InvalidXLogRecPtr;
 			walsnd->state = WALSNDSTATE_STARTUP;
+			walsnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
 			walsnd->latch = &MyProc->procLatch;
 			SpinLockRelease(&walsnd->mutex);
 			/* don't need the lock anymore */
@@ -2751,6 +2958,24 @@ WalSndGetStateString(WalSndState state)
 	return "UNKNOWN";
 }
 
+/*
+ * Return a string constant representing the causal reads state. This is used
+ * in system views, and should *not* be translated.
+ */
+static const char *
+WalSndGetCausalReadsStateString(WalSndCausalReadsState causal_reads_state)
+{
+	switch (causal_reads_state)
+	{
+		case WALSNDCRSTATE_UNAVAILABLE:
+			return "unavailable";
+		case WALSNDCRSTATE_JOINING:
+			return "joining";
+		case WALSNDCRSTATE_AVAILABLE:
+			return "available";
+	}
+	return "UNKNOWN";
+}
 
 /*
  * Returns activity of walsenders, including pids and xlog locations sent to
@@ -2759,7 +2984,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	9
+#define PG_STAT_GET_WAL_SENDERS_COLS	10
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2810,6 +3035,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		int			applyLagMs;
 		int			priority;
 		WalSndState state;
+		WalSndCausalReadsState causalReadsState;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
 		bool		nulls[PG_STAT_GET_WAL_SENDERS_COLS];
 
@@ -2819,6 +3045,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		SpinLockAcquire(&walsnd->mutex);
 		sentPtr = walsnd->sentPtr;
 		state = walsnd->state;
+		causalReadsState = walsnd->causal_reads_state;
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
@@ -2893,6 +3120,9 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				values[8] = CStringGetTextDatum("sync");
 			else
 				values[8] = CStringGetTextDatum("potential");
+
+			values[9] =
+				CStringGetTextDatum(WalSndGetCausalReadsStateString(causalReadsState));
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
@@ -2912,14 +3142,52 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 static void
 WalSndKeepalive(bool requestReply)
 {
+	TimestampTz now;
+	TimestampTz causal_reads_lease;
+
 	elog(DEBUG2, "sending replication keepalive");
 
+	/*
+	 * If the walsender currently deems the standby to be available for causal
+	 * reads, then it grants a causal reads lease.  The lease authorizes the
+	 * standby to consider itself available for causal reads until a short
+	 * time in the future.  The primary promises to uphold the causal reads
+	 * guarantee until that time, by stalling commits until the the lease has
+	 * expired if necessary.
+	 */
+	now = GetCurrentTimestamp();
+	if (MyWalSnd->causal_reads_state < WALSNDCRSTATE_AVAILABLE)
+		causal_reads_lease = 0; /* Not available, no lease granted. */
+	else
+	{
+		/*
+		 * Since this timestamp is being sent to the standby where it will be
+		 * compared against a time generated by the standby's system clock, we
+		 * must consider clock skew.  First, we decide on a maximum tolerable
+		 * difference between system clocks.  If the primary's clock is ahead
+		 * of the standby's by more than this, then all bets are off (the
+		 * standby could falsely believe it has a valid lease).  If the
+		 * primary's clock is behind the standby's by more than this, then the
+		 * standby will err the other way and generate spurious errors in
+		 * causal_reads mode.  Rather than having a separate GUC for this, we
+		 * derive it from causal_reads_timeout.
+		 */
+		int max_clock_skew = causal_reads_timeout / CAUSAL_READS_CLOCK_SKEW_RATIO;
+
+		/* Compute and remember the expiry time of the lease we're granting. */
+		causal_reads_last_lease = TimestampTzPlusMilliseconds(now, causal_reads_timeout);
+		/* The version we'll send to the standby is adjusted to tolerate clock skew. */
+		causal_reads_lease =
+			TimestampTzPlusMilliseconds(causal_reads_last_lease, -max_clock_skew);
+	}
+
 	/* construct the message... */
 	resetStringInfo(&output_message);
 	pq_sendbyte(&output_message, 'k');
 	pq_sendint64(&output_message, sentPtr);
-	pq_sendint64(&output_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(now));
 	pq_sendbyte(&output_message, requestReply ? 1 : 0);
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(causal_reads_lease));
 
 	/* ... and send it wrapped in CopyData */
 	pq_putmessage_noblock('d', output_message.data, output_message.len);
@@ -2937,23 +3205,35 @@ WalSndKeepaliveIfNecessary(TimestampTz now)
 	 * Don't send keepalive messages if timeouts are globally disabled or
 	 * we're doing something not partaking in timeouts.
 	 */
-	if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
-		return;
-
-	if (waiting_for_ping_response)
-		return;
+	if (!am_potential_causal_reads_standby)
+	{
+		if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
+			return;
+		if (waiting_for_ping_response)
+			return;
+	}
 
 	/*
 	 * If half of wal_sender_timeout has lapsed without receiving any reply
 	 * from the standby, send a keep-alive message to the standby requesting
 	 * an immediate reply.
+	 *
+	 * If causal_reads_timeout has been configured, use it to control
+	 * keepalive intervals rather than wal_sender_timeout, so that we can keep
+	 * replacing leases at the right frequency.
 	 */
-	ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-											wal_sender_timeout / 2);
+	if (am_potential_causal_reads_standby)
+		ping_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+												causal_reads_timeout /
+												CAUSAL_READS_KEEPALIVE_RATIO);
+	else
+		ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+												wal_sender_timeout / 2);
 	if (now >= ping_time)
 	{
 		WalSndKeepalive(true);
 		waiting_for_ping_response = true;
+		last_keepalive_timestamp = now;
 
 		/* Try to flush pending output to the client */
 		if (pq_flush_if_writable() != 0)
diff --git a/src/backend/utils/errcodes.txt b/src/backend/utils/errcodes.txt
index 04c9c00..d4bf0c0 100644
--- a/src/backend/utils/errcodes.txt
+++ b/src/backend/utils/errcodes.txt
@@ -302,6 +302,7 @@ Section: Class 40 - Transaction Rollback
 40001    E    ERRCODE_T_R_SERIALIZATION_FAILURE                              serialization_failure
 40003    E    ERRCODE_T_R_STATEMENT_COMPLETION_UNKNOWN                       statement_completion_unknown
 40P01    E    ERRCODE_T_R_DEADLOCK_DETECTED                                  deadlock_detected
+40P02    E    ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE                         causal_reads_not_available
 
 Section: Class 42 - Syntax Error or Access Rule Violation
 
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index a8eaa5f..cd69889 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -1633,6 +1633,16 @@ static struct config_bool ConfigureNamesBool[] =
 		NULL, NULL, NULL
 	},
 
+	{
+		{"causal_reads", PGC_USERSET, REPLICATION_STANDBY,
+		 gettext_noop("Enables causal reads."),
+		 NULL
+		},
+		&causal_reads,
+		false,
+		NULL, NULL, NULL
+	},
+
 	/* End-of-list marker */
 	{
 		{NULL, 0, 0, NULL, NULL}, NULL, false, NULL, NULL, NULL
@@ -1791,6 +1801,17 @@ static struct config_int ConfigureNamesInt[] =
 	},
 
 	{
+		{"causal_reads_timeout", PGC_SIGHUP, REPLICATION_STANDBY,
+			gettext_noop("Sets the maximum apply lag before causal reads standbys are no longer available."),
+			NULL,
+			GUC_UNIT_MS
+		},
+		&causal_reads_timeout,
+		0, 0, INT_MAX,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"max_connections", PGC_POSTMASTER, CONN_AUTH_SETTINGS,
 			gettext_noop("Sets the maximum number of concurrent connections."),
 			NULL
@@ -3387,10 +3408,21 @@ static struct config_string ConfigureNamesString[] =
 		},
 		&SyncRepStandbyNames,
 		"",
-		check_synchronous_standby_names, NULL, NULL
+		check_standby_names, NULL, NULL
 	},
 
 	{
+		{"causal_reads_standby_names", PGC_SIGHUP, REPLICATION_MASTER,
+			gettext_noop("List of names of potential causal reads standbys."),
+			NULL,
+			GUC_LIST_INPUT
+		},
+		&causal_reads_standby_names,
+		"*",
+		check_standby_names, NULL, NULL
+ 	},
+
+	{
 		{"default_text_search_config", PGC_USERSET, CLIENT_CONN_LOCALE,
 			gettext_noop("Sets default text search configuration."),
 			NULL
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index 085099c..733b9e5 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -244,6 +244,15 @@
 				# from standby(s); '*' = all
 #vacuum_defer_cleanup_age = 0	# number of xacts by which cleanup is delayed
 
+#causal_reads_timeout = 0s      # maximum replication delay to tolerate from
+                                # standbys before dropping them from the set of
+				# available causal reads peers; 0 to disable
+				# causal reads
+
+#causal_reads_standy_names = '*'
+                                # standby servers that can potentially become
+				# available for causal reads; '*' = all
+
 # - Standby Servers -
 
 # These settings are ignored on a master server.
@@ -266,6 +275,14 @@
 #wal_retrieve_retry_interval = 5s	# time to wait before retrying to
 					# retrieve WAL after a failed attempt
 
+# - All Servers -
+
+#causal_reads = off                     # "on" in any pair of consecutive
+                                        # transactions guarantees that the second
+					# can see the first (even if the second
+					# is run on a standby), or will raise an
+					# error to report that the standby is
+					# unavailable for causal reads
 
 #------------------------------------------------------------------------------
 # QUERY TUNING
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index b88e012..6336240 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -46,8 +46,11 @@
 
 #include "access/transam.h"
 #include "access/xact.h"
+#include "access/xlog.h"
 #include "lib/pairingheap.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
+#include "replication/walreceiver.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
 #include "storage/procarray.h"
@@ -209,6 +212,16 @@ GetTransactionSnapshot(void)
 				 "cannot take query snapshot during a parallel operation");
 
 		/*
+		 * In causal_reads mode on a standby, check if we have definitely
+		 * applied WAL for any COMMIT that returned successfully on the
+		 * primary.
+		 */
+		if (causal_reads && RecoveryInProgress() && !WalRcvCausalReadsAvailable())
+			ereport(ERROR,
+					(errcode(ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE),
+					 errmsg("standby is not available for causal reads")));
+
+		/*
 		 * In transaction-snapshot mode, the first snapshot must live until
 		 * end of xact regardless of what the caller does with it, so we must
 		 * make a copy of it rather than returning CurrentSnapshotData
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 03a5f22..157b50e 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2710,7 +2710,7 @@ DATA(insert OID = 1936 (  pg_stat_get_backend_idset		PGNSP PGUID 12 1 100 0 0 f
 DESCR("statistics: currently active backend IDs");
 DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f f f t s r 1 0 2249 "23" "{23,26,23,26,25,25,25,16,1184,1184,1184,1184,869,25,23,28,28,16,25,25,23,16,25}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{pid,datid,pid,usesysid,application_name,state,query,waiting,xact_start,query_start,backend_start,state_change,client_addr,client_hostname,client_port,backend_xid,backend_xmin,ssl,sslversion,sslcipher,sslbits,sslcompression,sslclientdn}" _null_ _null_ pg_stat_get_activity _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active backends");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25}" "{o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25,25}" "{o,o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state,causal_reads_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index 28b68f6..253f831 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -24,14 +24,33 @@
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
 #define SYNC_REP_WAIT_APPLY		2
+#define SYNC_REP_WAIT_CAUSAL_READS 3
 
-#define NUM_SYNC_REP_WAIT_MODE	3
+#define NUM_SYNC_REP_WAIT_MODE	4
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
 #define SYNC_REP_WAITING			1
 #define SYNC_REP_WAIT_COMPLETE		2
 
+/*
+ * ratio of causal_read_timeout to max_clock_skew (4 means than the maximum
+ * tolerated clock difference between primary and standbys using causal_reads
+ * is 1/4 of causal_reads_timeout)
+ */
+#define CAUSAL_READS_CLOCK_SKEW_RATIO 4
+
+/*
+ * ratio of causal_reads_timeout to keepalive time (2 means that the effective
+ * keepalive time is 1/2 of the causal_reads_timeout GUC when it is non-zero)
+ */
+#define CAUSAL_READS_KEEPALIVE_RATIO 2
+
+/* GUC variables */
+extern int causal_reads_timeout;
+extern bool causal_reads;
+extern char *causal_reads_standby_names;
+
 /* user-settable parameters for synchronous replication */
 extern char *SyncRepStandbyNames;
 
@@ -43,16 +62,23 @@ extern void SyncRepCleanupAtProcExit(void);
 
 /* called by wal sender */
 extern void SyncRepInitConfig(void);
-extern void SyncRepReleaseWaiters(void);
+extern void SyncRepReleaseWaiters(bool walsender_cr_available_or_joining);
 
 /* called by checkpointer */
 extern void SyncRepUpdateSyncStandbysDefined(void);
 
+/* called by user backend (xact.c) */
+extern void CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN);
+
+/* called by wal sender */
+extern void CausalReadsBeginStall(TimestampTz lease_expiry_time);
+extern bool CausalReadsPotentialStandby(void);
+
 /* forward declaration to avoid pulling in walsender_private.h */
 struct WalSnd;
 extern struct WalSnd *SyncRepGetSynchronousStandby(void);
 
-extern bool check_synchronous_standby_names(char **newval, void **extra, GucSource source);
+extern bool check_standby_names(char **newval, void **extra, GucSource source);
 extern void assign_synchronous_commit(int newval, void *extra);
 
 #endif   /* _SYNCREP_H */
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 3294df9..7f83934 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -80,6 +80,13 @@ typedef struct
 	TimeLineID	receivedTLI;
 
 	/*
+	 * causalReadsLease is the time until which the primary has authorized
+	 * this standby to consider itself available for causal_reads mode, or 0
+	 * for not authorized.
+	 */
+	TimestampTz causalReadsLease;
+
+	/*
 	 * latestChunkStart is the starting byte position of the current "batch"
 	 * of received WAL.  It's actually the same as the previous value of
 	 * receivedUpto before the last flush to disk.  Startup process can use
@@ -164,4 +171,6 @@ extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
 extern void WalRcvWakeup(void);
 
+extern bool WalRcvCausalReadsAvailable(void);
+
 #endif   /* _WALRECEIVER_H */
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 88ccdf0..81a2776 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -27,6 +27,13 @@ typedef enum WalSndState
 	WALSNDSTATE_STREAMING
 } WalSndState;
 
+typedef enum WalSndCausalReadsState
+{
+	WALSNDCRSTATE_UNAVAILABLE = 0,
+	WALSNDCRSTATE_JOINING,
+	WALSNDCRSTATE_AVAILABLE
+} WalSndCausalReadsState;
+
 /*
  * Each walsender has a WalSnd struct in shared memory.
  */
@@ -34,6 +41,7 @@ typedef struct WalSnd
 {
 	pid_t		pid;			/* this walsender's process id, or 0 */
 	WalSndState state;			/* this walsender's state */
+	WalSndCausalReadsState causal_reads_state; /* the walsender's causal reads state */
 	XLogRecPtr	sentPtr;		/* WAL has been sent up to this point */
 	bool		needreload;		/* does currently-open file need to be
 								 * reloaded? */
@@ -89,6 +97,12 @@ typedef struct
 	 */
 	bool		sync_standbys_defined;
 
+	/*
+	 * Until when must commits in causal_reads stall?  This is used to wait
+	 * for causal reads leases to expire.
+	 */
+	TimestampTz	stall_causal_reads_until;
+
 	WalSnd		walsnds[FLEXIBLE_ARRAY_MEMBER];
 } WalSndCtlData;
 
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 6f80548..2727533 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1762,10 +1762,11 @@ pg_stat_replication| SELECT s.pid,
     w.replay_location,
     w.replay_lag,
     w.sync_priority,
-    w.sync_state
+    w.sync_state,
+    w.causal_reads_state
    FROM pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, waiting, xact_start, query_start, backend_start, state_change, client_addr, client_hostname, client_port, backend_xid, backend_xmin, ssl, sslversion, sslcipher, sslbits, sslcompression, sslclientdn),
     pg_authid u,
-    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state)
+    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state, causal_reads_state)
   WHERE ((s.usesysid = u.oid) AND (s.pid = w.pid));
 pg_stat_ssl| SELECT s.pid,
     s.ssl,
#45Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Michael Paquier (#41)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Thu, Mar 3, 2016 at 7:34 PM, Michael Paquier
<michael.paquier@gmail.com> wrote:

WAL replay for 2PC should also
call XLogRequestWalReceiverReply() when needed.

Ah yes, I missed this important sentence. I will address that in the
next version after some testing.

--
Thomas Munro
http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#46Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Thomas Munro (#44)
5 attachment(s)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Wed, Mar 9, 2016 at 6:07 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

Agreed. I have split this work up into four patches which apply on
top of each other, and provide something (hopefully) useful at each
stage.

Yesterday's patch set doesn't apply after commit
b6fb6471f6afaf649e52f38269fd8c5c60647669 which added a neighbouring
line in pg_proc.h, so here's a new set that does.

I looked into COMMIT PREPARED replay feedback and realised that it
doesn't need any special handling beyond what is already in
xact_redo_commit. However, I see now that I *do* need to do something
when replaying PREPARE TRANSACTION, as you said. Not for causal reads
though -- it doesn't care about an operation with no visible effect --
but for synchronous_commit = remote_apply. I am thinking about how to
fix that. (Have PREPARE TRANSACTION wait only for flush even though
you asked for remote_apply? Add a 'feedback please' bit to
XLOG_XACT_PREPARE records? Always send feedback when replaying
XLOG_XACT_PREPARE records?)

The following rough ballpark numbers (generated with the attached test
client) aren't very scientific or in any way indicative of real
conditions (it's a bunch of clusters running on my laptop), but they
demonstrate that two-phase commit apply feedback is being reported to
the primary straight away in causal reads mode (otherwise the 2PC
causal reads number wouldn't be so high).

Sequential UPDATE in simple transaction:

single node: ~2700 TPS
sync rep remote flush: ~2500 TPS
sync rep remote apply: ~2000 TPS
causal reads (4 standbys): ~1600 TPS

Sequential UPDATE in two phase commit transaction:

single node: ~900 TPS
sync rep remote flush: ~900 TPS
sync rep remote apply: (hangs)
causal reads (4 standbys): ~900 TPS

(The actual numbers are pretty noisy. I've taken medians of 3 and
rounded to the nearest 100, and I guess the replication overheads are
not magnified as much in the case of the slower 2PC workload and then
get lost in the noise. With --check you can verify that the 2PC
transaction is not always visible on the standby it connects to until
you enable --causal-reads, so I don't think it's just broken!)

--
Thomas Munro
http://www.enterprisedb.com

Attachments:

0001-remote-apply-v2.patchapplication/octet-stream; name=0001-remote-apply-v2.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index a09ceb2..b21b4c0 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -2091,7 +2091,7 @@ include_dir 'conf.d'
         Specifies whether transaction commit will wait for WAL records
         to be written to disk before the command returns a <quote>success</>
         indication to the client.  Valid values are <literal>on</>,
-        <literal>remote_write</>, <literal>local</>, and <literal>off</>.
+        <literal>remote_write</>, <literal> remote_apply</>, <literal>local</>, and <literal>off</>.
         The default, and safe, setting
         is <literal>on</>.  When <literal>off</>, there can be a delay between
         when success is reported to the client and when the transaction is
@@ -2125,6 +2125,10 @@ include_dir 'conf.d'
         ensure data preservation even if the standby instance of
         <productname>PostgreSQL</> were to crash, but not if the standby
         suffers an operating-system-level crash.
+        When set to <literal>remote_apply</>, commits will wait until a reply
+        from the current synchronous stanbyindicates it has received the
+        commit record of the transaction and applied it, so that it has become
+        visible to queries.
        </para>
        <para>
         When synchronous
diff --git a/doc/src/sgml/high-availability.sgml b/doc/src/sgml/high-availability.sgml
index 6cb690c..2600fba 100644
--- a/doc/src/sgml/high-availability.sgml
+++ b/doc/src/sgml/high-availability.sgml
@@ -1081,6 +1081,9 @@ primary_slot_name = 'node_a_slot'
     WAL record is then sent to the standby. The standby sends reply
     messages each time a new batch of WAL data is written to disk, unless
     <varname>wal_receiver_status_interval</> is set to zero on the standby.
+    In the case that <varname>synchronous_commit</> is set to
+    <literal>remote_apply</>, the standby sends reply messages when the commit
+    record is replayed, making the transaction visible.
     If the standby is the first matching standby, as specified in
     <varname>synchronous_standby_names</> on the primary, the reply
     messages from that standby will be used to wake users waiting for
@@ -1107,6 +1110,14 @@ primary_slot_name = 'node_a_slot'
    </para>
 
    <para>
+    Setting <varname>synchronous_commit</> to <literal>remote_apply</> will
+    cause each commit to wait until the current synchronous standby reports
+    that it has replayed the transaction, making it visible to user queries.
+    In simple cases, this allows for load balancing with causal consistency
+    on a single hot standby.
+   </para>
+
+   <para>
     Users will stop waiting if a fast shutdown is requested.  However, as
     when using asynchronous replication, the server will not fully
     shutdown until all outstanding WAL records are transferred to the currently
@@ -1160,9 +1171,10 @@ primary_slot_name = 'node_a_slot'
     <title>Planning for High Availability</title>
 
    <para>
-    Commits made when <varname>synchronous_commit</> is set to <literal>on</>
-    or <literal>remote_write</> will wait until the synchronous standby responds. The response
-    may never occur if the last, or only, standby should crash.
+    Commits made when <varname>synchronous_commit</> is set to <literal>on</>,
+    <literal>remote_write</> or <literal>remote_apply</> will wait until the
+    synchronous standby responds. The response may never occur if the last, or
+    only, standby should crash.
    </para>
 
    <para>
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index b491735..5e8cc3d 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -5121,6 +5121,13 @@ XactLogCommitRecord(TimestampTz commit_time,
 		xl_xinfo.xinfo |= XACT_COMPLETION_FORCE_SYNC_COMMIT;
 
 	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this commit is applied.
+	*/
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
+	/*
 	 * Relcache invalidations requires information about the current database
 	 * and so does logical decoding.
 	 */
@@ -5456,6 +5463,13 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 	if (XactCompletionForceSyncCommit(parsed->xinfo))
 		XLogFlush(lsn);
 
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit = remote_apply), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 /*
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 00f139a..0519357 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -346,6 +346,12 @@ static XLogRecPtr RedoRecPtr;
 static bool doPageWrites;
 
 /*
+ * doRequestWalReceiverReply is used by recovery code to ask the main recovery
+ * loop to trigger a walreceiver reply.
+ */
+static bool doRequestWalReceiverReply;
+
+/*
  * RedoStartLSN points to the checkpoint's REDO location which is specified
  * in a backup label file, backup history file or control file. In standby
  * mode, XLOG streaming usually starts from the position where an invalid
@@ -6909,6 +6915,19 @@ StartupXLOG(void)
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
 				SpinLockRelease(&XLogCtl->info_lck);
 
+				/*
+				 * If rm_redo reported that it applied a commit record that
+				 * the master is waiting for by calling
+				 * XLogRequestWalReceiverReply, then we wake up the receiver
+				 * so that it notices the updated lastReplayedEndRecPtr and
+				 * sends a reply to the master.
+				 */
+				if (doRequestWalReceiverReply)
+				{
+					doRequestWalReceiverReply = false;
+					WalRcvWakeup();
+				}
+
 				/* Remember this record as the last-applied one */
 				LastRec = ReadRecPtr;
 
@@ -11630,3 +11649,13 @@ SetWalWriterSleeping(bool sleeping)
 	XLogCtl->WalWriterSleeping = sleeping;
 	SpinLockRelease(&XLogCtl->info_lck);
 }
+
+/*
+ * Called by redo code to indicate that the xlog replay loop should wake up
+ * the walreceiver process so that a reply can be sent to the primary.
+ */
+void
+XLogRequestWalReceiverReply(void)
+{
+	doRequestWalReceiverReply = true;
+}
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 92faf4e..4565348 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -416,6 +416,7 @@ SyncRepReleaseWaiters(void)
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
+	int			numapply = 0;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
@@ -462,12 +463,18 @@ SyncRepReleaseWaiters(void)
 		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
 		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
 	}
+	if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+	{
+		walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+		numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY);
+	}
 
 	LWLockRelease(SyncRepLock);
 
-	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X",
+	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%x",
 		 numwrite, (uint32) (MyWalSnd->write >> 32), (uint32) MyWalSnd->write,
-	   numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush);
+		 numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush,
+		 numapply, (uint32) (MyWalSnd->apply >> 32), (uint32) MyWalSnd->apply);
 
 	/*
 	 * If we are managing the highest priority standby, though we weren't
@@ -728,6 +735,9 @@ assign_synchronous_commit(int newval, void *extra)
 		case SYNCHRONOUS_COMMIT_REMOTE_FLUSH:
 			SyncRepWaitMode = SYNC_REP_WAIT_FLUSH;
 			break;
+		case SYNCHRONOUS_COMMIT_REMOTE_APPLY:
+			SyncRepWaitMode = SYNC_REP_WAIT_APPLY;
+			break;
 		default:
 			SyncRepWaitMode = SYNC_REP_NO_WAIT;
 			break;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 7b36e02..c19842e 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -101,6 +101,7 @@ static uint32 recvOff = 0;
  */
 static volatile sig_atomic_t got_SIGHUP = false;
 static volatile sig_atomic_t got_SIGTERM = false;
+static volatile sig_atomic_t got_SIGUSR2 = false;
 
 /*
  * LogstreamResult indicates the byte positions that we have already
@@ -150,9 +151,27 @@ static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
 /* Signal handlers */
 static void WalRcvSigHupHandler(SIGNAL_ARGS);
 static void WalRcvSigUsr1Handler(SIGNAL_ARGS);
+static void WalRcvSigUsr2Handler(SIGNAL_ARGS);
 static void WalRcvShutdownHandler(SIGNAL_ARGS);
 static void WalRcvQuickDieHandler(SIGNAL_ARGS);
 
+static void WalRcvBlockSigUsr2(void)
+{
+	sigset_t mask;
+
+	sigemptyset(&mask);
+	sigaddset(&mask, SIGUSR2);
+	sigprocmask(SIG_BLOCK, &mask, NULL);
+}
+
+static void WalRcvUnblockSigUsr2(void)
+{
+	sigset_t mask;
+
+	sigemptyset(&mask);
+	sigaddset(&mask, SIGUSR2);
+	sigprocmask(SIG_UNBLOCK, &mask, NULL);
+}
 
 static void
 ProcessWalRcvInterrupts(void)
@@ -200,6 +219,7 @@ WalReceiverMain(void)
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz last_recv_timestamp;
 	bool		ping_sent;
+	bool		forceReply;
 
 	/*
 	 * WalRcv should be set up already (if we are a backend, we inherit this
@@ -268,7 +288,7 @@ WalReceiverMain(void)
 	pqsignal(SIGALRM, SIG_IGN);
 	pqsignal(SIGPIPE, SIG_IGN);
 	pqsignal(SIGUSR1, WalRcvSigUsr1Handler);
-	pqsignal(SIGUSR2, SIG_IGN);
+	pqsignal(SIGUSR2, WalRcvSigUsr2Handler);
 
 	/* Reset some signals that are accepted by postmaster but not here */
 	pqsignal(SIGCHLD, SIG_DFL);
@@ -299,6 +319,10 @@ WalReceiverMain(void)
 	/* Unblock signals (they were blocked when the postmaster forked us) */
 	PG_SETMASK(&UnBlockSig);
 
+	/* Block SIGUSR2 (we unblock it only during network waits). */
+	WalRcvBlockSigUsr2();
+	got_SIGUSR2 = false;
+
 	/* Establish the connection to the primary for XLOG streaming */
 	EnableWalRcvImmediateExit();
 	walrcv_connect(conninfo);
@@ -408,7 +432,9 @@ WalReceiverMain(void)
 				}
 
 				/* Wait a while for data to arrive */
+				WalRcvUnblockSigUsr2();
 				len = walrcv_receive(NAPTIME_PER_CYCLE, &buf);
+				WalRcvBlockSigUsr2();
 				if (len != 0)
 				{
 					/*
@@ -439,11 +465,21 @@ WalReceiverMain(void)
 							endofwal = true;
 							break;
 						}
+						WalRcvUnblockSigUsr2();
 						len = walrcv_receive(0, &buf);
+						WalRcvBlockSigUsr2();
+					}
+
+					if (got_SIGUSR2)
+					{
+						/* The recovery process asked us to force a reply. */
+						got_SIGUSR2 = false;
+						forceReply = true;
 					}
 
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(false, false);
+					XLogWalRcvSendReply(forceReply, false);
+					forceReply = false;
 
 					/*
 					 * If we've written some records, flush them to disk and
@@ -498,7 +534,14 @@ WalReceiverMain(void)
 						}
 					}
 
-					XLogWalRcvSendReply(requestReply, requestReply);
+					if (got_SIGUSR2)
+					{
+						/* The recovery process asked us to force a reply. */
+						got_SIGUSR2 = false;
+						forceReply = true;
+					}
+					XLogWalRcvSendReply(requestReply || forceReply, requestReply);
+					forceReply = false;
 					XLogWalRcvSendHSFeedback(false);
 				}
 			}
@@ -740,6 +783,13 @@ WalRcvSigUsr1Handler(SIGNAL_ARGS)
 	errno = save_errno;
 }
 
+/* SIGUSR2: used to receive wakeups from recovery */
+static void
+WalRcvSigUsr2Handler(SIGNAL_ARGS)
+{
+	got_SIGUSR2 = true;
+}
+
 /* SIGTERM: set flag for main loop, or shutdown immediately if safe */
 static void
 WalRcvShutdownHandler(SIGNAL_ARGS)
@@ -1222,6 +1272,22 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 }
 
 /*
+ * Wake up the walreceiver if it happens to be blocked in walrcv_receive,
+ * and tell it that a commit record has been applied.
+ *
+ * This is called by the startup process whenever interesting xlog records
+ * are applied, so that walreceiver can check if it needs to send an apply
+ * notification back to the master which may be waiting in a COMMIT with
+ * synchronous_commit = remote_apply.
+ */
+void
+WalRcvWakeup(void)
+{
+	if (WalRcv->pid != 0)
+		kill(WalRcv->pid, SIGUSR2);
+}
+
+/*
  * Return a string constant representing the state. This is used
  * in system functions and views, and should *not* be translated.
  */
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index ea5a09a..a8eaa5f 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -345,12 +345,13 @@ static const struct config_enum_entry constraint_exclusion_options[] = {
 };
 
 /*
- * Although only "on", "off", "remote_write", and "local" are documented, we
- * accept all the likely variants of "on" and "off".
+ * Although only "on", "off", "remote_apply", "remote_write", and "local" are
+ * documented, we accept all the likely variants of "on" and "off".
  */
 static const struct config_enum_entry synchronous_commit_options[] = {
 	{"local", SYNCHRONOUS_COMMIT_LOCAL_FLUSH, false},
 	{"remote_write", SYNCHRONOUS_COMMIT_REMOTE_WRITE, false},
+	{"remote_apply", SYNCHRONOUS_COMMIT_REMOTE_APPLY, false},
 	{"on", SYNCHRONOUS_COMMIT_ON, false},
 	{"off", SYNCHRONOUS_COMMIT_OFF, false},
 	{"true", SYNCHRONOUS_COMMIT_ON, true},
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index ee3d378..085099c 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -177,7 +177,7 @@
 					# (change requires restart)
 #fsync = on				# turns forced synchronization on or off
 #synchronous_commit = on		# synchronization level;
-					# off, local, remote_write, or on
+					# off, local, remote_write, remote_apply, or on
 #wal_sync_method = fsync		# the default is the first option
 					# supported by the operating system:
 					#   open_datasync
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index ebeb582..ed8d22c 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -60,7 +60,9 @@ typedef enum
 	SYNCHRONOUS_COMMIT_LOCAL_FLUSH,		/* wait for local flush only */
 	SYNCHRONOUS_COMMIT_REMOTE_WRITE,	/* wait for local flush and remote
 										 * write */
-	SYNCHRONOUS_COMMIT_REMOTE_FLUSH		/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_FLUSH,	/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_APPLY		/* wait for local flush and remote
+										 * apply */
 }	SyncCommitLevel;
 
 /* Define the default setting for synchonous_commit */
@@ -144,10 +146,13 @@ typedef void (*SubXactCallback) (SubXactEvent event, SubTransactionId mySubid,
  * EOXact... routines which run at the end of the original transaction
  * completion.
  */
+#define XACT_COMPLETION_SYNC_APPLY_FEEDBACK		(1U << 29)
 #define XACT_COMPLETION_UPDATE_RELCACHE_FILE	(1U << 30)
 #define XACT_COMPLETION_FORCE_SYNC_COMMIT		(1U << 31)
 
 /* Access macros for above flags */
+#define XactCompletionSyncApplyFeedback(xinfo) \
+	(!!(xinfo & XACT_COMPLETION_SYNC_APPLY_FEEDBACK))
 #define XactCompletionRelcacheInitFileInval(xinfo) \
 	(!!(xinfo & XACT_COMPLETION_UPDATE_RELCACHE_FILE))
 #define XactCompletionForceSyncCommit(xinfo) \
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index ecd30ce..68e20e4 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -268,6 +268,8 @@ extern bool CheckPromoteSignal(void);
 extern void WakeupRecovery(void);
 extern void SetWalWriterSleeping(bool sleeping);
 
+extern void XLogRequestWalReceiverReply(void);
+
 extern void assign_max_wal_size(int newval, void *extra);
 extern void assign_checkpoint_completion_target(double newval, void *extra);
 
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index 96e059b..28b68f6 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -23,8 +23,9 @@
 #define SYNC_REP_NO_WAIT		-1
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
+#define SYNC_REP_WAIT_APPLY		2
 
-#define NUM_SYNC_REP_WAIT_MODE	2
+#define NUM_SYNC_REP_WAIT_MODE	3
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 6eacb09..3294df9 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -162,5 +162,6 @@ extern void RequestXLogStreaming(TimeLineID tli, XLogRecPtr recptr,
 extern XLogRecPtr GetWalRcvWriteRecPtr(XLogRecPtr *latestChunkStart, TimeLineID *receiveTLI);
 extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
+extern void WalRcvWakeup(void);
 
 #endif   /* _WALRECEIVER_H */
0002-replay-lag-v2.patchapplication/octet-stream; name=0002-replay-lag-v2.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 85459d0..818b952 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -820,6 +820,12 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       standby server</entry>
     </row>
     <row>
+     <entry><structfield>replay_lag</></entry>
+     <entry><type>interval</></entry>
+     <entry>Estimated time taken for recent WAL records to be replayed on this
+      standby server</entry>
+    </row>
+    <row>
      <entry><structfield>sync_priority</></entry>
      <entry><type>integer</></entry>
      <entry>Priority of this standby server for being chosen as the
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 5e8cc3d..220d238 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -5464,6 +5464,12 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 		XLogFlush(lsn);
 
 	/*
+	 * Record the primary's timestamp for the commit record, so it can be used
+	 * for tracking replay lag.
+	 */
+	SetXLogReplayTimestamp(parsed->xact_time);
+
+	/*
 	 * If asked by the primary (because someone is waiting for a synchronous
 	 * commit = remote_apply), we will need to ask walreceiver to send a
 	 * reply immediately.
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 0519357..64e05bf 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -81,6 +81,8 @@ extern uint32 bootstrap_data_checksum_version;
 #define PROMOTE_SIGNAL_FILE		"promote"
 #define FALLBACK_PROMOTE_SIGNAL_FILE "fallback_promote"
 
+/* Size of the circular buffer of timestamped LSNs. */
+#define MAX_TIMESTAMPED_LSNS 8192
 
 /* User-settable parameters */
 int			max_wal_size = 64;	/* 1 GB */
@@ -363,6 +365,13 @@ static bool doRequestWalReceiverReply;
  */
 static XLogRecPtr RedoStartLSN = InvalidXLogRecPtr;
 
+/*
+ * LastReplayedTimestamp can be set by redo handlers when they apply a record
+ * that carries a timestamp, by calling SetXLogReplayedTimestamp.  The xlog
+ * apply loop can then update the value in shared memory.
+ */
+static TimestampTz LastReplayedTimestamp = 0;
+
 /*----------
  * Shared-memory data structures for XLOG control
  *
@@ -637,6 +646,21 @@ typedef struct XLogCtlData
 	/* current effective recovery target timeline */
 	TimeLineID	RecoveryTargetTLI;
 
+	/* timestamp from the most recently applied record associated with a timestamp. */
+	TimestampTz lastReplayedTimestamp;
+
+	/*
+	 * We maintain a circular buffer of LSNs and associated timestamps.
+	 * Walreceiver writes into it using information from timestamps, and the
+	 * startup recovery process reads from it and notifies walreceiver when
+	 * LSNs are replayed so that the timestamps can eventually be fed back to
+	 * the upstream server, to track lag.
+	 */
+	Index			timestampedLsnRead;
+	Index			timestampedLsnWrite;
+	XLogRecPtr		timestampedLsn[MAX_TIMESTAMPED_LSNS];
+	TimestampTz		timestampedLsnTime[MAX_TIMESTAMPED_LSNS];
+
 	/*
 	 * timestamp of when we started replaying the current chunk of WAL data,
 	 * only relevant for replication or archive recovery
@@ -6907,20 +6931,51 @@ StartupXLOG(void)
 				error_context_stack = errcallback.previous;
 
 				/*
-				 * Update lastReplayedEndRecPtr after this record has been
-				 * successfully replayed.
+				 * Update lastReplayedEndRecPtr and lastReplayedTimestamp
+				 * after this record has been successfully replayed.
 				 */
 				SpinLockAcquire(&XLogCtl->info_lck);
 				XLogCtl->lastReplayedEndRecPtr = EndRecPtr;
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
+				if (LastReplayedTimestamp != 0)
+				{
+					/* If replaying a record produced a timestamp, use that. */
+					XLogCtl->lastReplayedTimestamp = LastReplayedTimestamp;
+					LastReplayedTimestamp = 0;
+				}
+				else
+				{
+					/*
+					 * If we have applied LSNs associated with timestamps
+					 * received by walreceiver, then use the recorded
+					 * timestamp.  We consume from the read end of the
+					 * circular buffer.
+					 */
+					while (XLogCtl->timestampedLsnRead !=
+						   XLogCtl->timestampedLsnWrite &&
+						   XLogCtl->timestampedLsn[XLogCtl->timestampedLsnRead]
+						   <= EndRecPtr)
+					{
+						if (XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead] >
+							XLogCtl->lastReplayedTimestamp)
+						{
+							XLogCtl->lastReplayedTimestamp =
+								XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead];
+							doRequestWalReceiverReply = true;
+						}
+						XLogCtl->timestampedLsnRead =
+							(XLogCtl->timestampedLsnRead + 1) % MAX_TIMESTAMPED_LSNS;
+					}
+				}
 				SpinLockRelease(&XLogCtl->info_lck);
 
 				/*
 				 * If rm_redo reported that it applied a commit record that
 				 * the master is waiting for by calling
-				 * XLogRequestWalReceiverReply, then we wake up the receiver
-				 * so that it notices the updated lastReplayedEndRecPtr and
-				 * sends a reply to the master.
+				 * XLogRequestWalReceiverReply, or we encountered a WAL
+				 * location that was associated with a timestamp above, then
+				 * we wake up the receiver so that it notices the updated
+				 * lastReplayedEndRecPtr and sends a reply to the master.
 				 */
 				if (doRequestWalReceiverReply)
 				{
@@ -11659,3 +11714,91 @@ XLogRequestWalReceiverReply(void)
 {
 	doRequestWalReceiverReply = true;
 }
+
+/*
+ * Record the timestamp that is associated with a WAL position.
+ *
+ * This is called by walreceiver on standby servers when keepalive messages
+ * arrive, using timestamps generated on the primary server.  The timestamp
+ * will be sent back to the primary server when the standby had applied this
+ * WAL position.  The primary can use the elapsed time to estimate the replay
+ * lag.
+ */
+void
+SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn)
+{
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn == XLogCtl->lastReplayedEndRecPtr)
+	{
+		/*
+		 * That is the last replayed LSN: we are fully replayed, so we can
+		 * update the replay timestamp immediately.
+		 */
+		XLogCtl->lastReplayedTimestamp = timestamp;
+	}
+	else
+	{
+		/*
+		 * There is WAL still to be applied.  We will associate the timestamp
+		 * with this WAL position and wait for it to be replayed.  We add it
+		 * at the 'write' end of the circular buffer of LSN/timestamp
+		 * mappings, which the replay loop will eventually read.
+		 */
+		Index w = XLogCtl->timestampedLsnWrite;
+		Index r = XLogCtl->timestampedLsnRead;
+
+		XLogCtl->timestampedLsn[w] = lsn;
+		XLogCtl->timestampedLsnTime[w] = timestamp;
+
+		/* Advance the write point. */
+		w = (w + 1) % MAX_TIMESTAMPED_LSNS;
+		XLogCtl->timestampedLsnWrite = w;
+		if (w == r)
+		{
+			/*
+			 * The buffer is full.  Advance the read point (throwing away
+			 * oldest values; we will begin to overestimate replay lag, until
+			 * lag decreases to a size our buffer can manage, or the next
+			 * commit record is replayed).
+			 */
+			r = (r + 1) % MAX_TIMESTAMPED_LSNS;
+			XLogCtl->timestampedLsnRead = r;
+		}
+	}
+	SpinLockRelease(&XLogCtl->info_lck);
+}
+
+/*
+ * Set the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary.  This can be called by redo handlers that have
+ * an appropriate timestamp (currently only commit records).  Updating the
+ * shared memory value is deferred until after the redo handler returns.
+ */
+void
+SetXLogReplayTimestamp(TimestampTz timestamp)
+{
+	LastReplayedTimestamp = timestamp;
+}
+
+/*
+ * Get the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary, and also the most recently applied LSN.  (Note
+ * that the timestamp and the LSN don't necessarily relate to the same
+ * record.)
+ *
+ * This is similar to GetLatestXTime, except that it is not only advanced by
+ * commit records (see SetXLogReplayTimestampAtLsn).
+ */
+TimestampTz
+GetXLogReplayTimestamp(XLogRecPtr *lsn)
+{
+	TimestampTz result;
+
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn)
+		*lsn = XLogCtl->lastReplayedEndRecPtr;
+	result = XLogCtl->lastReplayedTimestamp;
+	SpinLockRelease(&XLogCtl->info_lck);
+
+	return result;
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index abf9a70..f5a7c5c 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -661,6 +661,7 @@ CREATE VIEW pg_stat_replication AS
             W.write_location,
             W.flush_location,
             W.replay_location,
+	    W.replay_lag,
             W.sync_priority,
             W.sync_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index c19842e..d061dfe 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -85,6 +85,8 @@ walrcv_disconnect_type walrcv_disconnect = NULL;
 
 #define NAPTIME_PER_CYCLE 100	/* max sleep time between cycles (100ms) */
 
+#define MIN_TIME_BETWEEN_TIMESTAMPED_LSNS 1000 /* 1s */
+
 /*
  * These variables are used similarly to openLogFile/SegNo/Off,
  * but for walreceiver to write the XLOG. recvFileTLI is the TimeLineID
@@ -103,6 +105,8 @@ static volatile sig_atomic_t got_SIGHUP = false;
 static volatile sig_atomic_t got_SIGTERM = false;
 static volatile sig_atomic_t got_SIGUSR2 = false;
 
+static bool reply_sent_on_recovery_activity = false;
+
 /*
  * LogstreamResult indicates the byte positions that we have already
  * written/fsynced.
@@ -144,7 +148,7 @@ static void WalRcvDie(int code, Datum arg);
 static void XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len);
 static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
-static void XLogWalRcvSendReply(bool force, bool requestReply);
+static void XLogWalRcvSendReply(bool force, bool requestReply, bool includeApplyTimestamp);
 static void XLogWalRcvSendHSFeedback(bool immed);
 static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
 
@@ -219,7 +223,7 @@ WalReceiverMain(void)
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz last_recv_timestamp;
 	bool		ping_sent;
-	bool		forceReply;
+	bool		timestampedWalApplied;
 
 	/*
 	 * WalRcv should be set up already (if we are a backend, we inherit this
@@ -472,14 +476,15 @@ WalReceiverMain(void)
 
 					if (got_SIGUSR2)
 					{
-						/* The recovery process asked us to force a reply. */
+						/* The recovery process asked us to report an applied timestamp. */
 						got_SIGUSR2 = false;
-						forceReply = true;
+						timestampedWalApplied = true;
+						reply_sent_on_recovery_activity = true;
 					}
 
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(forceReply, false);
-					forceReply = false;
+					XLogWalRcvSendReply(timestampedWalApplied, false, timestampedWalApplied);
+					timestampedWalApplied = false;
 
 					/*
 					 * If we've written some records, flush them to disk and
@@ -536,12 +541,14 @@ WalReceiverMain(void)
 
 					if (got_SIGUSR2)
 					{
-						/* The recovery process asked us to force a reply. */
+						/* The recovery process asked us to report an apply timestamp. */
 						got_SIGUSR2 = false;
-						forceReply = true;
+						timestampedWalApplied = true;
+						reply_sent_on_recovery_activity = true;
 					}
-					XLogWalRcvSendReply(requestReply || forceReply, requestReply);
-					forceReply = false;
+					XLogWalRcvSendReply(requestReply || timestampedWalApplied, requestReply,
+										timestampedWalApplied);
+					timestampedWalApplied = false;
 					XLogWalRcvSendHSFeedback(false);
 				}
 			}
@@ -879,6 +886,8 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 			}
 		case 'k':				/* Keepalive */
 			{
+				bool reportApplyTimestamp = false;
+
 				/* copy message to StringInfo */
 				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
 				if (len != hdrlen)
@@ -895,9 +904,22 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 
 				ProcessWalSndrMessage(walEnd, sendTime);
 
-				/* If the primary requested a reply, send one immediately */
-				if (replyRequested)
-					XLogWalRcvSendReply(true, false);
+				/*
+				 * If no apply timestamps have been sent at the request of the
+				 * recovery process since we last received a keepalive, then
+				 * we will send one now.  This allows us to feed back
+				 * timestamps in response to pings if we are idle or if the
+				 * recovery process is somehow blocked, but we don't want to
+				 * do that if it's actively applying and periodically waking
+				 * us up.
+				 */
+				if (!reply_sent_on_recovery_activity)
+					reportApplyTimestamp = true;
+				reply_sent_on_recovery_activity = false;
+
+				/* If the primary requested a reply, send one immediately. */
+				if (replyRequested || reportApplyTimestamp)
+					XLogWalRcvSendReply(true, false, reportApplyTimestamp);
 				break;
 			}
 		default:
@@ -1060,7 +1082,7 @@ XLogWalRcvFlush(bool dying)
 		/* Also let the master know that we made some progress */
 		if (!dying)
 		{
-			XLogWalRcvSendReply(false, false);
+			XLogWalRcvSendReply(false, false, false);
 			XLogWalRcvSendHSFeedback(false);
 		}
 	}
@@ -1078,15 +1100,20 @@ XLogWalRcvFlush(bool dying)
  * If 'requestReply' is true, requests the server to reply immediately upon
  * receiving this message. This is used for heartbearts, when approaching
  * wal_receiver_timeout.
+ *
+ * If 'reportApplyTimestamp' is true, the latest apply timestamp is included.
+ * This is set to true only when this function is called after the recovery
+ * process has replayed a record with an associated timestamp.
  */
 static void
-XLogWalRcvSendReply(bool force, bool requestReply)
+XLogWalRcvSendReply(bool force, bool requestReply, bool reportApplyTimestamp)
 {
 	static XLogRecPtr writePtr = 0;
 	static XLogRecPtr flushPtr = 0;
 	XLogRecPtr	applyPtr;
 	static TimestampTz sendTime = 0;
 	TimestampTz now;
+	TimestampTz applyTimestamp = 0;
 
 	/*
 	 * If the user doesn't want status to be reported to the master, be sure
@@ -1102,10 +1129,8 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	 * We can compare the write and flush positions to the last message we
 	 * sent without taking any lock, but the apply position requires a spin
 	 * lock, so we don't check that unless something else has changed or 10
-	 * seconds have passed.  This means that the apply log position will
-	 * appear, from the master's point of view, to lag slightly, but since
-	 * this is only for reporting purposes and only on idle systems, that's
-	 * probably OK.
+	 * seconds have passed, or the force flag has been set (which happens when
+	 * apply feedback has been requested by the primary).
 	 */
 	if (!force
 		&& writePtr == LogstreamResult.Write
@@ -1118,7 +1143,10 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	/* Construct a new message */
 	writePtr = LogstreamResult.Write;
 	flushPtr = LogstreamResult.Flush;
-	applyPtr = GetXLogReplayRecPtr(NULL);
+	if (reportApplyTimestamp)
+		applyTimestamp = GetXLogReplayTimestamp(&applyPtr);
+	else
+		applyPtr = GetXLogReplayRecPtr(NULL);
 
 	resetStringInfo(&reply_message);
 	pq_sendbyte(&reply_message, 'r');
@@ -1126,6 +1154,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	pq_sendint64(&reply_message, flushPtr);
 	pq_sendint64(&reply_message, applyPtr);
 	pq_sendint64(&reply_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&reply_message, TimestampTzToIntegerTimestamp(applyTimestamp));
 	pq_sendbyte(&reply_message, requestReply ? 1 : 0);
 
 	/* Send it */
@@ -1230,8 +1259,8 @@ static void
 ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 {
 	WalRcvData *walrcv = WalRcv;
-
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
+	static TimestampTz lastRecordedTimestamp = 0;
 
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
@@ -1242,6 +1271,18 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
 	SpinLockRelease(&walrcv->mutex);
 
+	/*
+	 * Remember primary's timestamp at this WAL location.  We throw away
+	 * samples if they are coming too fast because we don't want to fill up
+	 * the finite circular buffer and have to throw away older samples.
+	 */
+	if (lastRecordedTimestamp < TimestampTzPlusMilliseconds(sendTime,
+															-MIN_TIME_BETWEEN_TIMESTAMPED_LSNS))
+	{
+		SetXLogReplayTimestampAtLsn(sendTime, walEnd);
+		lastRecordedTimestamp = sendTime;
+	}
+
 	if (log_min_messages <= DEBUG2)
 	{
 		char	   *sendtime;
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index c03e045..5bb142d 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1543,15 +1543,29 @@ ProcessStandbyReplyMessage(void)
 	XLogRecPtr	writePtr,
 				flushPtr,
 				applyPtr;
+	int			applyLagMs;
 	bool		replyRequested;
+	TimestampTz now = GetCurrentTimestamp();
+	TimestampTz applyTimestamp;
 
 	/* the caller already consumed the msgtype byte */
 	writePtr = pq_getmsgint64(&reply_message);
 	flushPtr = pq_getmsgint64(&reply_message);
 	applyPtr = pq_getmsgint64(&reply_message);
 	(void) pq_getmsgint64(&reply_message);		/* sendTime; not used ATM */
+	applyTimestamp = IntegerTimestampToTimestampTz(pq_getmsgint64(&reply_message));
 	replyRequested = pq_getmsgbyte(&reply_message);
 
+	/* Compute the apply lag in milliseconds. */
+	if (applyTimestamp == 0)
+		applyLagMs = -1;
+	else
+#ifdef HAVE_INT64_TIMESTAMP
+		applyLagMs = (now - applyTimestamp) / 1000;
+#else
+		applyLagMs = (now - applyTimestamp) * 1000.0;
+#endif
+
 	elog(DEBUG2, "write %X/%X flush %X/%X apply %X/%X%s",
 		 (uint32) (writePtr >> 32), (uint32) writePtr,
 		 (uint32) (flushPtr >> 32), (uint32) flushPtr,
@@ -1573,6 +1587,8 @@ ProcessStandbyReplyMessage(void)
 		walsnd->write = writePtr;
 		walsnd->flush = flushPtr;
 		walsnd->apply = applyPtr;
+		if (applyLagMs >= 0)
+			walsnd->applyLagMs = applyLagMs;
 		SpinLockRelease(&walsnd->mutex);
 	}
 
@@ -2743,7 +2759,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	8
+#define PG_STAT_GET_WAL_SENDERS_COLS	9
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2791,6 +2807,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		XLogRecPtr	write;
 		XLogRecPtr	flush;
 		XLogRecPtr	apply;
+		int			applyLagMs;
 		int			priority;
 		WalSndState state;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
@@ -2805,6 +2822,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
+		applyLagMs = walsnd->applyLagMs;
 		priority = walsnd->sync_standby_priority;
 		SpinLockRelease(&walsnd->mutex);
 
@@ -2839,6 +2857,23 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				nulls[5] = true;
 			values[5] = LSNGetDatum(apply);
 
+			if (applyLagMs < 0)
+				nulls[6] = true;
+			else
+			{
+				Interval *applyLagInterval = palloc(sizeof(Interval));
+
+				applyLagInterval->month = 0;
+				applyLagInterval->day = 0;
+#ifdef HAVE_INT64_TIMESTAMP
+				applyLagInterval->time = applyLagMs * 1000;
+#else
+				applyLagInterval->time = applyLagMs / 1000.0;
+#endif
+				nulls[6] = false;
+				values[6] = IntervalPGetDatum(applyLagInterval);
+			}
+
 			/*
 			 * Treat a standby such as a pg_basebackup background process
 			 * which always returns an invalid flush location, as an
@@ -2846,18 +2881,18 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 			 */
 			priority = XLogRecPtrIsInvalid(walsnd->flush) ? 0 : priority;
 
-			values[6] = Int32GetDatum(priority);
+			values[7] = Int32GetDatum(priority);
 
 			/*
 			 * More easily understood version of standby state. This is purely
 			 * informational, not different from priority.
 			 */
 			if (priority == 0)
-				values[7] = CStringGetTextDatum("async");
+				values[8] = CStringGetTextDatum("async");
 			else if (walsnd == sync_standby)
-				values[7] = CStringGetTextDatum("sync");
+				values[8] = CStringGetTextDatum("sync");
 			else
-				values[7] = CStringGetTextDatum("potential");
+				values[8] = CStringGetTextDatum("potential");
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
diff --git a/src/backend/utils/adt/timestamp.c b/src/backend/utils/adt/timestamp.c
index c4f556a..2032f61 100644
--- a/src/backend/utils/adt/timestamp.c
+++ b/src/backend/utils/adt/timestamp.c
@@ -1611,6 +1611,20 @@ IntegerTimestampToTimestampTz(int64 timestamp)
 #endif
 
 /*
+ * TimestampTzToIntegerTimestamp -- convert a native timestamp to int64 format
+ *
+ * When compiled with --enable-integer-datetimes, this is implemented as a
+ * no-op macro.
+ */
+#ifndef HAVE_INT64_TIMESTAMP
+int64
+TimestampTzToIntegerTimestamp(TimestampTz timestamp)
+{
+	return timestamp * 1000000;
+}
+#endif
+
+/*
  * TimestampDifference -- convert the difference between two timestamps
  *		into integer seconds and microseconds
  *
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 68e20e4..efb9719 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -236,6 +236,9 @@ extern void GetXLogReceiptTime(TimestampTz *rtime, bool *fromStream);
 extern XLogRecPtr GetXLogReplayRecPtr(TimeLineID *replayTLI);
 extern XLogRecPtr GetXLogInsertRecPtr(void);
 extern XLogRecPtr GetXLogWriteRecPtr(void);
+extern void SetXLogReplayTimestamp(TimestampTz timestamp);
+extern void SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn);
+extern TimestampTz GetXLogReplayTimestamp(XLogRecPtr *lsn);
 extern bool RecoveryIsPaused(void);
 extern void SetRecoveryPause(bool recoveryPause);
 extern TimestampTz GetLatestXTime(void);
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index a0f821a..811f10c 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2712,7 +2712,7 @@ DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f
 DESCR("statistics: information about currently active backends");
 DATA(insert OID = 3318 (  pg_stat_get_progress_info           PGNSP PGUID 12 1 100 0 0 f f f f f t s r 1 0 2249 "25" "{25,23,26,26,20,20,20,20,20,20,20,20,20,20}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{cmdtype,pid,datid,relid,param1,param2,param3,param4,param5,param6,param7,param8,param9,param10}" _null_ _null_ pg_stat_get_progress_info _null_ _null_ _null_ ));
 DESCR("statistics: information about progress of backends running maintenance command");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,23,25}" "{o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25}" "{o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 7794aa5..88ccdf0 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -46,6 +46,7 @@ typedef struct WalSnd
 	XLogRecPtr	write;
 	XLogRecPtr	flush;
 	XLogRecPtr	apply;
+	int			applyLagMs;
 
 	/* Protects shared variables shown above. */
 	slock_t		mutex;
diff --git a/src/include/utils/timestamp.h b/src/include/utils/timestamp.h
index fbead3a..297e151 100644
--- a/src/include/utils/timestamp.h
+++ b/src/include/utils/timestamp.h
@@ -227,9 +227,11 @@ extern bool TimestampDifferenceExceeds(TimestampTz start_time,
 #ifndef HAVE_INT64_TIMESTAMP
 extern int64 GetCurrentIntegerTimestamp(void);
 extern TimestampTz IntegerTimestampToTimestampTz(int64 timestamp);
+extern int64 TimestampTzToIntegerTimestamp(TimestampTz timestamp);
 #else
 #define GetCurrentIntegerTimestamp()	GetCurrentTimestamp()
 #define IntegerTimestampToTimestampTz(timestamp) (timestamp)
+#define TimestampTzToIntegerTimestamp(timestamp) (timestamp)
 #endif
 
 extern TimestampTz time_t_to_timestamptz(pg_time_t tm);
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 81bc5c9..6f80548 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1760,11 +1760,12 @@ pg_stat_replication| SELECT s.pid,
     w.write_location,
     w.flush_location,
     w.replay_location,
+    w.replay_lag,
     w.sync_priority,
     w.sync_state
    FROM pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, waiting, xact_start, query_start, backend_start, state_change, client_addr, client_hostname, client_port, backend_xid, backend_xmin, ssl, sslversion, sslcipher, sslbits, sslcompression, sslclientdn),
     pg_authid u,
-    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, sync_priority, sync_state)
+    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state)
   WHERE ((s.usesysid = u.oid) AND (s.pid = w.pid));
 pg_stat_ssl| SELECT s.pid,
     s.ssl,
0003-refactor-syncrep-exit-v2.patchapplication/octet-stream; name=0003-refactor-syncrep-exit-v2.patchDownload
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 4565348..bbc28a7 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -83,6 +83,64 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * ===========================================================
  */
 
+static bool
+SyncRepCheckEarlyExit(void)
+{
+	/*
+	 * If a wait for synchronous replication is pending, we can neither
+	 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
+	 * lead the client to believe that the transaction aborted, which
+	 * is not true: it's already committed locally. The former is no good
+	 * either: the client has requested synchronous replication, and is
+	 * entitled to assume that an acknowledged commit is also replicated,
+	 * which might not be true. So in this case we issue a WARNING (which
+	 * some clients may be able to interpret) and shut off further output.
+	 * We do NOT reset ProcDiePending, so that the process will die after
+	 * the commit is cleaned up.
+	 */
+	if (ProcDiePending)
+	{
+		ereport(WARNING,
+				(errcode(ERRCODE_ADMIN_SHUTDOWN),
+				 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * It's unclear what to do if a query cancel interrupt arrives.  We
+	 * can't actually abort at this point, but ignoring the interrupt
+	 * altogether is not helpful, so we just terminate the wait with a
+	 * suitable warning.
+	 */
+	if (QueryCancelPending)
+	{
+		QueryCancelPending = false;
+		ereport(WARNING,
+				(errmsg("canceling wait for synchronous replication due to user request"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * If the postmaster dies, we'll probably never get an
+	 * acknowledgement, because all the wal sender processes will exit. So
+	 * just bail out.
+	 */
+	if (!PostmasterIsAlive())
+	{
+		ProcDiePending = true;
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	return false;
+}
+
 /*
  * Wait for synchronous replication, if requested by user.
  *
@@ -180,57 +238,9 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 		if (syncRepState == SYNC_REP_WAIT_COMPLETE)
 			break;
 
-		/*
-		 * If a wait for synchronous replication is pending, we can neither
-		 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
-		 * lead the client to believe that the transaction aborted, which
-		 * is not true: it's already committed locally. The former is no good
-		 * either: the client has requested synchronous replication, and is
-		 * entitled to assume that an acknowledged commit is also replicated,
-		 * which might not be true. So in this case we issue a WARNING (which
-		 * some clients may be able to interpret) and shut off further output.
-		 * We do NOT reset ProcDiePending, so that the process will die after
-		 * the commit is cleaned up.
-		 */
-		if (ProcDiePending)
-		{
-			ereport(WARNING,
-					(errcode(ERRCODE_ADMIN_SHUTDOWN),
-					 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit())
 			break;
-		}
-
-		/*
-		 * It's unclear what to do if a query cancel interrupt arrives.  We
-		 * can't actually abort at this point, but ignoring the interrupt
-		 * altogether is not helpful, so we just terminate the wait with a
-		 * suitable warning.
-		 */
-		if (QueryCancelPending)
-		{
-			QueryCancelPending = false;
-			ereport(WARNING,
-					(errmsg("canceling wait for synchronous replication due to user request"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * If the postmaster dies, we'll probably never get an
-		 * acknowledgement, because all the wal sender processes will exit. So
-		 * just bail out.
-		 */
-		if (!PostmasterIsAlive())
-		{
-			ProcDiePending = true;
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
-			break;
-		}
 
 		/*
 		 * Wait on latch.  Any condition that should wake us up will set the
0004-causal-reads-v2.patchapplication/octet-stream; name=0004-causal-reads-v2.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index b21b4c0..de24f1c 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -2668,6 +2668,35 @@ include_dir 'conf.d'
      across the cluster without problems if that is required.
     </para>
 
+    <sect2 id="runtime-config-replication-all">
+     <title>All Servers</title>
+     <para>
+      These parameters can be set on the primary or any standby.
+     </para>
+     <variablelist>
+      <varlistentry id="guc-causal-reads" xreflabel="causal_reads">
+       <term><varname>causal_reads</varname> (<type>boolean</type>)
+       <indexterm>
+        <primary><varname>causal_reads</> configuration parameter</primary>
+       </indexterm>
+       </term>
+       <listitem>
+        <para>
+         Enables causal consistency between transactions run on different
+         servers.  A transaction that is run on a standby
+         with <varname>causal_reads</> set to <literal>on</> is guaranteed
+         either to see the effects of all completed transactions run on the
+         primary with the setting on, or to receive an error "standby is not
+         available for causal reads".  Note that both transactions involved in
+         a causal dependency (a write on the primary followed by a read on any
+         server which must see the write) must be run with the setting on.
+         See <xref linkend="causal-reads"> for more details.
+        </para>
+       </listitem>
+      </varlistentry>
+     </variablelist>     
+    </sect2>
+
     <sect2 id="runtime-config-replication-sender">
      <title>Sending Server(s)</title>
 
@@ -2899,6 +2928,48 @@ include_dir 'conf.d'
       </listitem>
      </varlistentry>
 
+     <varlistentry>
+      <term><varname>causal_reads_timeout</varname> (<type>integer</type>)
+       <indexterm>
+        <primary><varname>causal_reads_timeout</> configuration parameter</primary>
+       </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies the maximum replay lag the primary will tolerate from a
+        standby before dropping it from the set of standbys available for
+        causal reads.
+       </para>
+       <para>
+        This setting is also used to control the <firstterm>leases</> used to
+        maintain the causal reads guarantee.  It must be set to a value which
+        is at least 4 times the maximum possible difference in system clocks
+        between the primary and standby servers, as described
+        in <xref linkend="causal-reads">.
+       </para>
+      </listitem>
+     </varlistentry>
+
+     <varlistentry id="guc-causal-reads-standby-names" xreflabel="causal-reads-standby-names">
+      <term><varname>causal_reads_standby_names</varname> (<type>string</type>)
+      <indexterm>
+       <primary><varname>causal_reads_standby_names</> configuration parameter</primary>
+      </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies a comma-separated list of standby names that can support
+        <firstterm>causal reads</>, as described in
+        <xref linkend="causal-reads">.  Follows the same convention
+        as <link linkend="guc-synchronous-standby-names"><literal>synchronous_standby_name</></>.
+        The default is <literal>*</>, matching all standbys.
+       </para>
+       <para>
+        This setting has no effect if <varname>causal_reads_timeout</> is not set.
+       </para>
+      </listitem>
+     </varlistentry>
+
      </variablelist>
     </sect2>
 
diff --git a/doc/src/sgml/high-availability.sgml b/doc/src/sgml/high-availability.sgml
index 2600fba..a01df0d 100644
--- a/doc/src/sgml/high-availability.sgml
+++ b/doc/src/sgml/high-availability.sgml
@@ -1115,6 +1115,9 @@ primary_slot_name = 'node_a_slot'
     that it has replayed the transaction, making it visible to user queries.
     In simple cases, this allows for load balancing with causal consistency
     on a single hot standby.
+    (See also
+    <xref linkend="causal-reads"> which deals with multiple standbys and
+    standby failure.)
    </para>
 
    <para>
@@ -1233,6 +1236,119 @@ primary_slot_name = 'node_a_slot'
    </sect3>
   </sect2>
 
+  <sect2 id="causal-reads">
+   <title>Causal reads</title>
+   <indexterm>
+    <primary>causal reads</primary>
+    <secondary>in standby</secondary>
+   </indexterm>
+
+   <para>
+    The causal reads feature allows read-only queries to run on hot standby
+    servers without exposing stale data to the client, providing a form of
+    causal consistency.  Transactions can run on any standby with the
+    following guarantee about the visibility of preceding transactions: If you
+    set <varname>causal_reads</> to <literal>on</> in any pair of consecutive
+    transactions tx1, tx2 where tx2 begins after tx1 successfully returns,
+    then tx2 will either see tx1 or fail with a new error "standby is not
+    available for causal reads", no matter which server it runs on.  Although
+    the guarantee is expressed in terms of two individual transactions, the
+    GUC can also be set at session, role or system level to make the guarantee
+    generally, allowing for load balancing of applications that were not
+    designed with load balancing in mind.
+   </para>
+
+   <para>
+    In order to enable the feature, <varname>causal_reads_timeout</> must be
+    set to a non-zero value on the primary server.  The
+    GUC <varname>causal_reads_standby_names</> can be used to limit the set of
+    standbys that can join the dynamic set of causal reads standbys by
+    providing a comma-separated list of application names.  By default, all
+    standbys are candidates, if the feature is enabled.
+   </para>
+
+   <para>
+    The current set of servers that the primary considers to be available for
+    causal reads can be seen in
+    the <link linkend="monitoring-stats-views-table"> <literal>pg_stat_replication</></>
+    view.  Administrators, applications and load balancing middleware can use
+    this view to discover standbys that can currently handle causal reads
+    transactions without raising the error.  Since that information is only an
+    instantantaneous snapshot, clients should still be prepared for the error
+    to be raised at any time, and consider redirecting transactions to another
+    standby.
+   </para>
+
+   <para>
+    The advantages of the causal reads feature over simply
+    setting <varname>synchronous_commit</> to <literal>remote_apply</> are:
+    <orderedlist>
+      <listitem>
+       <para>
+        It allows the primary to wait for multiple standbys to replay
+        transactions.
+       </para>
+      </listitem>
+      <listitem>
+       <para>
+        It places a configurable limit on how much replay lag (and therefore
+        delay at commit time) the primary tolerates from standbys before it
+        drops them from the dynamic set of standbys it waits for.
+       </para>   
+      </listitem>
+      <listitem>
+       <para>
+        It upholds the causal reads guarantee during the transitions that
+        occur when new standbys are added or removed from the set of standbys,
+        including scenarios where contact has been lost between the primary
+        and standbys but the standby is still alive and running client
+        queries.
+       </para>
+      </listitem>
+    </orderedlist>
+   </para>
+
+   <para>
+    The protocol used to uphold the guarantee even in the case of network
+    failure depends on the system clocks of the primary and standby servers
+    being synchronized, with an allowance for a difference up to one quarter
+    of <varname>causal_reads_timeout</>.  For example,
+    if <varname>causal_reads_timeout</> is set to <literal>4s</>, then the
+    clocks must not be further than 1 second apart for the guarantee to be
+    upheld reliably during transitions.  The ubiquity of the Network Time
+    Protocol (NTP) on modern operating systems and availability of high
+    quality time servers makes it possible to choose a tolerance significantly
+    higher than the maximum expected clock difference.  An effort is
+    nevertheless made to detect and report misconfigured and faulty systems
+    with clock differences greater than the configured tolerance.
+   </para>
+
+   <note>
+    <para>
+     Current hardware clocks, NTP implementations and public time servers are
+     unlikely to allow the system clocks to differ more than tens or hundreds
+     of milliseconds, and systems synchronized with dedicated local time
+     servers may be considerably more accurate, but you should only consider
+     setting <varname>causal_reads_timeout</> below 4 seconds (allowing up to
+     1 second of clock difference) after researching your time synchronization
+     infrastructure thoroughly.
+    </para>  
+   </note>
+
+   <note>
+    <para>
+      While similar to synchronous replication in the sense that both involve
+      the primary server waiting for responses from standby servers, the
+      causal reads feature is not concerned with avoiding data loss.  A
+      primary configured for causal reads will drop all standbys that stop
+      responding or replay too slowly from the dynamic set that it waits for,
+      so you should consider configuring both synchronous replication and
+      causal reads if you need data loss avoidance guarantees and causal
+      consistency guarantees for load balancing.
+    </para>
+   </note>
+  </sect2>
+
   <sect2 id="continuous-archiving-in-standby">
    <title>Continuous archiving in standby</title>
 
@@ -1581,7 +1697,16 @@ if (!triggered)
     so there will be a measurable delay between primary and standby. Running the
     same query nearly simultaneously on both primary and standby might therefore
     return differing results. We say that data on the standby is
-    <firstterm>eventually consistent</firstterm> with the primary.  Once the
+    <firstterm>eventually consistent</firstterm> with the primary by default.
+    The data visible to a transaction running on a standby can be
+    made <firstterm>causally consistent</> with respect to a transaction that
+    has completed on the primary by setting <varname>causal_reads</>
+    to <literal>on</> in both transactions.  For more details,
+    see <xref linkend="causal-reads">.
+   </para>
+
+   <para>
+    Once the    
     commit record for a transaction is replayed on the standby, the changes
     made by that transaction will be visible to any new snapshots taken on
     the standby.  Snapshots may be taken at the start of each query or at the
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 818b952..5a87f37 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -836,6 +836,17 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
      <entry><type>text</></entry>
      <entry>Synchronous state of this standby server</entry>
     </row>
+    <row>
+     <entry><structfield>causal_reads_state</></entry>
+     <entry><type>text</></entry>
+     <entry>Causal reads state of this standby server.  This field will be
+     non-null only if <varname>cause_reads_timeout</> is set.  If a standby is
+     in <literal>available</> state, then it can currently serve causal reads
+     queries.  If it is not replaying fast enough or not responding to
+     keepalive messages, it will be in <literal>unavailable</> state, and if
+     it is currently transitioning to availability it will be
+     in <literal>joining</> state for a short time.</entry>
+    </row>
    </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 8a22836..1837c1b 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -2090,11 +2090,12 @@ RecordTransactionCommitPrepared(TransactionId xid,
 	END_CRIT_SECTION();
 
 	/*
-	 * Wait for synchronous replication, if required.
+	 * Wait for causal reads and synchronous replication, if required.
 	 *
 	 * Note that at this stage we have marked clog, but still show as running
 	 * in the procarray and continue to hold locks.
 	 */
+	CausalReadsWaitForLSN(recptr);
 	SyncRepWaitForLSN(recptr);
 }
 
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 220d238..63ed05d 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,10 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
+	{
+		CausalReadsWaitForLSN(XactLastRecEnd);
 		SyncRepWaitForLSN(XactLastRecEnd);
+	}
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5124,7 +5127,7 @@ XactLogCommitRecord(TimestampTz commit_time,
 	 * Check if the caller would like to ask standbys for immediate feedback
 	 * once this commit is applied.
 	*/
-	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY || causal_reads)
 		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
 
 	/*
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index f5a7c5c..1fb9812 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -663,7 +663,8 @@ CREATE VIEW pg_stat_replication AS
             W.replay_location,
 	    W.replay_lag,
             W.sync_priority,
-            W.sync_state
+            W.sync_state,
+	    W.causal_reads_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
             pg_stat_get_wal_senders() AS W
     WHERE S.usesysid = U.oid AND
diff --git a/src/backend/replication/README.causal_reads b/src/backend/replication/README.causal_reads
new file mode 100644
index 0000000..1fddd62
--- /dev/null
+++ b/src/backend/replication/README.causal_reads
@@ -0,0 +1,193 @@
+The causal reads guarantee says: If you run any two consecutive
+transactions tx1, tx2 where tx1 completes before tx2 begins, with
+causal_reads set to "on" in both transactions, tx2 will see tx1 or
+raise an error to complain that it can't guarantee causal consistency,
+no matter which servers (primary or any standby) you run each
+transaction on.
+
+When both transactions run on the primary, the guarantee is trivially
+upheld.
+
+To deal with read-only physical streaming standbys, the primary keeps
+track of a set of standbys that it considers to be currently
+"available" for causal reads, and sends a stream of "leases" to those
+standbys granting them the right to handle causal reads transactions
+for a short time without any further communication with the primary.
+
+In general, the primary provides the guarantee by waiting for all of
+the "available" standbys to report that they have applied a
+transaction.  However, the set of available standbys is dynamic, and
+things get more complicated during state transitions.  There are two
+types of transitions to consider:
+
+1.  unavailable->joining->available
+
+Standbys start out as "unavailable".  If a standby is unavailable and
+is applying fast enough and matches causal_reads_standby_names, the
+primary transitions it to "available", but first it sets it to
+"joining" until it is sure that any transaction committed while it was
+unavailable has definitely been applied on the standby.  This closes a
+race that would otherwise exist if we moved directly to available
+state: tx1 might not wait for a given standby because it's
+unavailable, then a lease might be granted, and then tx2 might run a
+causal reads transaction without error but see stale data.  The
+joining state acts as an airlock: while in joining state, the primary
+waits for that standby to replay causal reads transactions in
+anticipation of the move to available, but it doesn't progress to
+available state and grant a lease to the standby until everything
+preceding joining state has also been applied.
+
+2.  available->unavailable
+
+If a standby is not applying fast enough or not responding to
+keepalive messages, then the primary kicks that standby out of the
+dynamic set of available standbys, that is, marks it as "unavailable".
+In order to make sure that the standby has started rejecting causal
+reads transactions, it needs to revoke the lease it most recently
+granted.  It does that by waiting for the lease to expire before
+allowing any causal reads commits to return.  (In future there could
+be a fast-path revocation message which waits for a serial-numbered
+acknowledgement to reduce waiting in the case where the standby is
+lagging but still reachable and responding).
+
+The rest of this document illustrates how clock skew affects the
+available->unavailable transition.
+
+The following 4 variables are derived from a single GUC, and these
+values will be used in the following illustrations:
+
+causal_reads_timeout = 4s
+lease_time           = 4s (= causal_reads_timeout)
+keepalive_time       = 2s (= lease_time / 2)
+max_clock_skew       = 1s (= lease_time / 4)
+
+Every keepalive_time, the primary transmits a lease that expires at
+local_clock_time + lease_time - max_clock_skew, shown in the following
+diagram as 't' for transmission time and '|' for expiry time.  If
+contact is lost with a standby, the primary will wait until sent_time
++ lease_time for the most recently granted lease to expire, shown on
+the following diagram 'x', to be sure that the standby's clock has
+reached the expiry time even if its clock differs by up to
+max_clock_skew.  In other words, the primary tells the standby that
+the expiry time is at one time, but it trusts that the standby will
+surely agree if it gives it some extra time.  The extra time is
+max_clock_skew.  If the clocks differ by more than max_clock_skew, all
+bets are off (but see below for attempt to detect obvious cases).
+
+0     1     2     3     4     5     6     7     8     9
+t-----------------|-----x
+            t-----------------|-----x
+                        t-----------------|-----x
+                                    t-----------------|...
+                                                t------...
+
+A standby whose clock is 2 seconds ahead of the primary's clock
+perceives gaps in the stream of leases, and will reject causal_reads
+transactions in those intervals.  The causal reads guarantee is
+upheld, but spurious errors are raised between leases, as a
+consequence of the clock skew being greater than max_clock_skew.  In
+the following diagram 'r' shows reception time, and the timeline along
+the top shows the standby's local clock time.
+
+2     3     4     5     6     7     8     9    10    11
+r-----|
+            r-----|
+                        r-----|
+                                    r-----|
+                                                r-----|
+
+If there were no network latency, a standby whose clock is exactly 1
+second ahead of the primary's clock would perceive the stream of
+leases as being replaced just in time, so there is no gap.  Since in
+reality the time of receipt is some time after the time of
+transmission due to network latency, if the standby's clock is exactly
+1 second behind, then there will be small network-latency-sized gaps
+before the next lease arrives, but still no correctness problem with
+respect to the causal reads guarantee.
+
+1     2     3     4     5     6     7     8     9    10
+r-----------|
+            r-----------|
+                        r-----------|
+                                    r-----------|
+                                                r------...
+
+A standby whose clock is perfectly in sync with the primary's
+perceives the stream of leases overlapping (this matches the primary's
+perception of the leases it sent):
+
+0     1     2     3     4     5     6     7     8     9
+r-----------------|
+            r-----------------|
+                        r-----------------|
+                                    r-----------------|
+                                                r------...
+
+A standby whose clock is exactly 1 second behind the primary's
+perceives the stream of leases as overlapping even more, but the time
+of expiry as judged by the standby is no later than the time the
+primary will wait for if required ('x').  That is, if contact is lost
+with the standby, the primary can still reliably hold up causal reads
+commits until the standby has started raising the error in
+causal_reads transactions.
+
+-1    0     1     2     3     4     5     6     7     8
+r-----------------------|
+            r-----------------------|
+                        r-----------------------|
+                                    r------------------...
+                                                r------...
+
+
+A standby whose clock is 2 seconds behind the primary's would perceive
+the stream of leases overlapping even more, and the primary would no
+longer be able to wait for a lease to expire if it wanted to revoke
+it.  But because the expiry time is after local_clock_time +
+lease_time, the standby can immediately see that its own clock must be
+more than 1 second behind the primary's, so it ignores the lease and
+logs a clock skew warning.  In the following diagram a lease expiry
+time that is obviously generated by a primary with a clock set too far
+in the future compared to the local clock is shown with a '!'.
+
+-2    -1    0     1     2     3     4     5     6     7
+r-----------------------------!
+            r-----------------------------!
+                        r-----------------------------!
+                                    r------------------...
+                                                r------...
+
+A danger window exists when the standby's clock is more than
+max_clock_skew behind the primary's clock, but not more than
+max_clock_skew + network latency time behind.  If the clock difference
+is in that range, then the algorithm presented above which is based on
+time of receipt cannot detect that the local clock is too far behind.
+The consequence of this problem could be as follows:
+
+1.  The standby loses contact with the primary due to a network fault.
+
+2.  The primary decides to drop the standby from the set of available
+    causal reads standbys due to lack of keepalive responses or
+    excessive lag, which necessitates holding up commits of causal
+    reads transactions until the most recently sent lease expires, in
+    the belief that the standby will definitely have started raising
+    the 'causal reads unavailable' error in causal reads transactions
+    by that time, if it is still alive and servicing requests.
+
+3.  The standby still has clients connected and running queries.
+
+4.  Due to clock skew in the problematic range, in the standby's
+    opinion the lease lasts slightly longer than the primary waits.
+
+5.  For a short window at most the duration of the network latency
+    time, clients running causal reads transactions are allowed to see
+    potentially stale data.
+
+For this reason we say that the causal reads guarantee only holds as
+long as the absolute difference between the system clocks of the
+machines is no more than max_clock_skew.  The theory is that NTP makes
+it possible to reason about the maximum possible clock difference
+between machines and choose a value that allows for a much larger
+difference.  However, we do make a best effort attempt to detect
+wildly divergent systems as described above, to catch the case of
+servers not running a correctly configured ntp daemon, or with a clock
+so far out of whack that ntp refuses to fix it.
\ No newline at end of file
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index bbc28a7..4f6b1be 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -57,6 +57,11 @@
 #include "utils/builtins.h"
 #include "utils/ps_status.h"
 
+/* GUC variables */
+int causal_reads_timeout;
+bool causal_reads;
+char *causal_reads_standby_names;
+
 /* User-settable parameters for sync rep */
 char	   *SyncRepStandbyNames;
 
@@ -69,7 +74,7 @@ static int	SyncRepWaitMode = SYNC_REP_NO_WAIT;
 
 static void SyncRepQueueInsert(int mode);
 static void SyncRepCancelWait(void);
-static int	SyncRepWakeQueue(bool all, int mode);
+static int	SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn);
 
 static int	SyncRepGetStandbyPriority(void);
 
@@ -142,6 +147,198 @@ SyncRepCheckEarlyExit(void)
 }
 
 /*
+ * Check if we can stop waiting for causal consistency.  We can stop waiting
+ * when the following conditions are met:
+ *
+ * 1.  All walsenders currently in 'joining' or 'available' state have
+ * applied the target LSN.
+ *
+ * 2.  Any stall periods caused by standbys dropping out of 'available' state
+ * have passed, so that we can be sure that their leases have expired and they
+ * have started rejecting causal reads transactions.
+ *
+ * The output parameter 'waitingFor' is set to the number of nodes we are
+ * currently waiting for.  The output parameters 'stallTimeMillis' is set to
+ * the number of milliseconds we need to wait for to observe any current
+ * commit stall.
+ *
+ * Returns true if commit can return control, because every standby has either
+ * applied the LSN or started rejecting causal_reads transactions.
+ */
+static bool
+CausalReadsCommitCanReturn(XLogRecPtr XactCommitLSN,
+						   int *waitingFor,
+						   long *stallTimeMillis)
+{
+	int i;
+	TimestampTz now;
+
+	/* Count how many joining/available nodes we are waiting for. */
+	*waitingFor = 0;
+	for (i = 0; i < max_wal_senders; ++i)
+	{
+		WalSnd *walsnd = &WalSndCtl->walsnds[i];
+
+		/*
+		 * Assuming atomic read of pid_t, we can check walsnd->pid without
+		 * acquiring the spinlock to avoid memory synchronization costs for
+		 * unused walsender slots.  We see a value that existed sometime at
+		 * least as recently as the last memory barrier.
+		 */
+		if (walsnd->pid != 0)
+		{
+			/*
+			 * We need to hold the spinlock to read LSNs, because we can't be
+			 * sure they can be read atomically.
+			 */
+			SpinLockAcquire(&walsnd->mutex);
+			if (walsnd->pid != 0 && walsnd->causal_reads_state >= WALSNDCRSTATE_JOINING)
+			{
+				if (walsnd->apply < XactCommitLSN)
+					++*waitingFor;
+			}
+			SpinLockRelease(&walsnd->mutex);
+		}
+	}
+
+	/* Check if there is a stall in progress that we need to observe. */
+	now = GetCurrentTimestamp();
+	LWLockAcquire(SyncRepLock, LW_SHARED);
+	if (WalSndCtl->stall_causal_reads_until > now)
+	{
+		long seconds;
+		int usecs;
+
+		/* Compute how long we have to wait, rounded up to nearest ms. */
+		TimestampDifference(now, WalSndCtl->stall_causal_reads_until,
+							&seconds, &usecs);
+		*stallTimeMillis = seconds * 1000 + (usecs + 999) / 1000;
+	}
+	else
+		*stallTimeMillis = 0;
+	LWLockRelease(SyncRepLock);
+
+	/* We are done if we are not waiting for any nodes or stalls. */
+	return *waitingFor == 0 && *stallTimeMillis == 0;
+}
+
+/*
+ * Wait for causal consistency in causal_reads mode, if requested by user.
+ */
+void
+CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN)
+{
+	long stallTimeMillis;
+	int waitingFor;
+	char *ps_display_buffer = NULL;
+
+	/* Leave if we aren't in causal_reads mode. */
+	if (!causal_reads)
+		return;
+
+	for (;;)
+	{
+		/* Reset latch before checking state. */
+		ResetLatch(MyLatch);
+
+		/*
+		 * Join the queue to be woken up if any causal reads joining/available
+		 * standby applies XactCommitLSN or the set of causal reads standbys
+		 * changes (if we aren't already in the queue).  We don't actually know
+		 * if we need to wait for any peers to reach the target LSN yet, but
+		 * we have to register just in case before checking the walsenders'
+		 * state to avoid a race condition that could occur if we did it after
+		 * calling CausalReadsCommitCanReturn.  (SyncRepWaitForLSN doesn't
+		 * have to do this because it can check the highest-seen LSN in
+		 * walsndctl->lsn[mode] which is protected by SyncRepLock, the same
+		 * lock as the queues.  We can't do that here, because there is no
+		 * single highest-seen LSN that is useful.  We must check
+		 * walsnd->apply for all relevant walsenders.  Therefore we must
+		 * register for notifications first, so that we can be notified via
+		 * our latch of any standby applying the LSN we're interested in after
+		 * we check but before we start waiting, or we could wait forever for
+		 * something that has already happened.)
+		 */
+		LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+		if (MyProc->syncRepState != SYNC_REP_WAITING)
+		{
+			MyProc->waitLSN = XactCommitLSN;
+			MyProc->syncRepState = SYNC_REP_WAITING;
+			SyncRepQueueInsert(SYNC_REP_WAIT_CAUSAL_READS);
+			Assert(SyncRepQueueIsOrderedByLSN(SYNC_REP_WAIT_CAUSAL_READS));
+		}
+		LWLockRelease(SyncRepLock);
+
+		/* Check if we're done. */
+		if (CausalReadsCommitCanReturn(XactCommitLSN, &waitingFor, &stallTimeMillis))
+		{
+			SyncRepCancelWait();
+			break;
+		}
+
+		Assert(waitingFor > 0 || stallTimeMillis > 0);
+
+		/* If we aren't actually waiting for any standbys, leave the queue. */
+		if (waitingFor == 0)
+			SyncRepCancelWait();
+
+		/* Update the ps title. */
+		if (update_process_title)
+		{
+			char buffer[80];
+
+			/* Remember the old value if this is our first update. */
+			if (ps_display_buffer == NULL)
+			{
+				int len;
+				const char *ps_display = get_ps_display(&len);
+
+				ps_display_buffer = palloc(len + 1);
+				memcpy(ps_display_buffer, ps_display, len);
+				ps_display_buffer[len] = '\0';
+			}
+
+			snprintf(buffer, sizeof(buffer),
+					 "waiting for %d peer(s) to apply %X/%X%s",
+					 waitingFor,
+					 (uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN,
+					 stallTimeMillis > 0 ? " (stalling)" : "");
+			set_ps_display(buffer, false);
+		}
+
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit()) /* Calls SyncRepCancelWait() if true. */
+			break;
+
+		/*
+		 * If are still waiting for peers, then we wait for any joining or
+		 * available peer to reach the LSN (or possibly stop being in one of
+		 * those states or go away).
+		 *
+		 * If not, there must be a non-zero stall time, so we wait for that to
+		 * elapse.
+		 */
+		if (waitingFor > 0)
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH, -1);
+		else
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH | WL_TIMEOUT,
+					  stallTimeMillis);
+	}
+
+	/* There is no way out of the loop that could leave us in the queue. */
+	Assert(SHMQueueIsDetached(&(MyProc->syncRepLinks)));
+	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
+	MyProc->waitLSN = 0;
+
+	/* Restore the ps display. */
+	if (ps_display_buffer != NULL)
+	{
+		set_ps_display(ps_display_buffer, false);
+		pfree(ps_display_buffer);
+	}
+}
+
+/*
  * Wait for synchronous replication, if requested by user.
  *
  * Initially backends start in state SYNC_REP_NOT_WAITING and then
@@ -413,6 +610,53 @@ SyncRepGetSynchronousStandby(void)
 }
 
 /*
+ * Check if the current WALSender process's application_name matches a name in
+ * causal_reads_standby_names (including '*' for wildcard).
+ */
+bool
+CausalReadsPotentialStandby(void)
+{
+	char *rawstring;
+	List	   *elemlist;
+	ListCell   *l;
+	bool		found = false;
+
+	/* If the feature is disable, then no. */
+	if (causal_reads_timeout == 0)
+		return false;
+
+	/* Need a modifiable copy of string */
+	rawstring = pstrdup(causal_reads_standby_names);
+
+	/* Parse string into list of identifiers */
+	if (!SplitIdentifierString(rawstring, ',', &elemlist))
+	{
+		/* syntax error in list */
+		pfree(rawstring);
+		list_free(elemlist);
+		/* GUC machinery will have already complained - no need to do again */
+		return 0;
+	}
+
+	foreach(l, elemlist)
+	{
+		char	   *standby_name = (char *) lfirst(l);
+
+		if (pg_strcasecmp(standby_name, application_name) == 0 ||
+			pg_strcasecmp(standby_name, "*") == 0)
+		{
+			found = true;
+			break;
+		}
+	}
+
+	pfree(rawstring);
+	list_free(elemlist);
+
+	return found;
+}
+
+/*
  * Update the LSNs on each queue based upon our latest state. This
  * implements a simple policy of first-valid-standby-releases-waiter.
  *
@@ -420,13 +664,15 @@ SyncRepGetSynchronousStandby(void)
  * perhaps also which information we store as well.
  */
 void
-SyncRepReleaseWaiters(void)
+SyncRepReleaseWaiters(bool walsender_cr_available_or_joining)
 {
 	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
 	int			numapply = 0;
+	int			numcausalreadsapply = 0;
+	bool		is_highest_priority_sync_standby;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
@@ -446,13 +692,19 @@ SyncRepReleaseWaiters(void)
 	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
 	syncWalSnd = SyncRepGetSynchronousStandby();
 
-	/* We should have found ourselves at least */
-	Assert(syncWalSnd != NULL);
+	/*
+	 * If we aren't managing the highest priority standby then make a note of
+	 * that so we can announce a takeover in the log if we ever get that job.
+	 */
+	is_highest_priority_sync_standby = syncWalSnd == MyWalSnd;
+	if (!is_highest_priority_sync_standby)
+		announce_next_takeover = true;
 
 	/*
-	 * If we aren't managing the highest priority standby then just leave.
+	 * If we aren't managing the highest priority standby or a standby in
+	 * causal reads 'joining' or 'available' state, then just leave.
 	 */
-	if (syncWalSnd != MyWalSnd)
+	if (!is_highest_priority_sync_standby && !walsender_cr_available_or_joining)
 	{
 		LWLockRelease(SyncRepLock);
 		announce_next_takeover = true;
@@ -461,24 +713,45 @@ SyncRepReleaseWaiters(void)
 
 	/*
 	 * Set the lsn first so that when we wake backends they will release up to
-	 * this location.
+	 * this location.  For the single-standby synchronous commit levels, we
+	 * only do this if we are the current synchronous standby and we are
+	 * advancing the LSN further than it has been advanced before, so that
+	 * SyncRepWaitForLSN can skip waiting in some cases.
 	 */
-	if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
-		numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
-		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+	if (is_highest_priority_sync_standby)
 	{
-		walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
-		numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY);
+		if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
+			numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE,
+										MyWalSnd->write);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
+			numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH,
+										MyWalSnd->flush);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+			numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY,
+										MyWalSnd->apply);
+		}
 	}
 
+	/*
+	 * For causal_reads, all walsenders currently in available or joining
+	 * state must reach the LSN on their own, and standbys will reach LSNs in
+	 * any order.  It doesn't make sense to keep the highest seen LSN in a
+	 * single walsndctl->lsn element.  (CausalReadsWaitForLSN has handling for
+	 * LSNs that have already been reached).
+	 */
+	if (walsender_cr_available_or_joining)
+		numcausalreadsapply =
+			SyncRepWakeQueue(false, SYNC_REP_WAIT_CAUSAL_READS,
+							 MyWalSnd->apply);
+
 	LWLockRelease(SyncRepLock);
 
 	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%x",
@@ -490,7 +763,7 @@ SyncRepReleaseWaiters(void)
 	 * If we are managing the highest priority standby, though we weren't
 	 * prior to this, then announce we are now the sync standby.
 	 */
-	if (announce_next_takeover)
+	if (is_highest_priority_sync_standby && announce_next_takeover)
 	{
 		announce_next_takeover = false;
 		ereport(LOG,
@@ -565,9 +838,8 @@ SyncRepGetStandbyPriority(void)
  * Must hold SyncRepLock.
  */
 static int
-SyncRepWakeQueue(bool all, int mode)
+SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn)
 {
-	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	PGPROC	   *proc = NULL;
 	PGPROC	   *thisproc = NULL;
 	int			numprocs = 0;
@@ -584,7 +856,7 @@ SyncRepWakeQueue(bool all, int mode)
 		/*
 		 * Assume the queue is ordered by LSN
 		 */
-		if (!all && walsndctl->lsn[mode] < proc->waitLSN)
+		if (!all && lsn < proc->waitLSN)
 			return numprocs;
 
 		/*
@@ -644,7 +916,7 @@ SyncRepUpdateSyncStandbysDefined(void)
 			int			i;
 
 			for (i = 0; i < NUM_SYNC_REP_WAIT_MODE; i++)
-				SyncRepWakeQueue(true, i);
+				SyncRepWakeQueue(true, i, InvalidXLogRecPtr);
 		}
 
 		/*
@@ -696,13 +968,31 @@ SyncRepQueueIsOrderedByLSN(int mode)
 #endif
 
 /*
+ * Make sure that CausalReadsWaitForLSN can't return until after the given
+ * lease expiry time has been reached.  In other words, revoke the lease.
+ *
+ * Wake up all backends waiting in CausalReadsWaitForLSN, because the set of
+ * available/joining peers has changed, and there is a new stall time they
+ * need to observe.
+ */
+void
+CausalReadsBeginStall(TimestampTz lease_expiry_time)
+{
+	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+	WalSndCtl->stall_causal_reads_until =
+		Max(WalSndCtl->stall_causal_reads_until, lease_expiry_time);
+	SyncRepWakeQueue(true, SYNC_REP_WAIT_CAUSAL_READS, InvalidXLogRecPtr);
+	LWLockRelease(SyncRepLock);
+}
+
+/*
  * ===========================================================
  * Synchronous Replication functions executed by any process
  * ===========================================================
  */
 
 bool
-check_synchronous_standby_names(char **newval, void **extra, GucSource source)
+check_standby_names(char **newval, void **extra, GucSource source)
 {
 	char	   *rawstring;
 	List	   *elemlist;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index d061dfe..e68384c 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -55,6 +55,7 @@
 #include "libpq/pqformat.h"
 #include "libpq/pqsignal.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "replication/walsender.h"
 #include "storage/ipc.h"
@@ -150,7 +151,8 @@ static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
 static void XLogWalRcvSendReply(bool force, bool requestReply, bool includeApplyTimestamp);
 static void XLogWalRcvSendHSFeedback(bool immed);
-static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
+static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+								  TimestampTz *causalReadsUntil);
 
 /* Signal handlers */
 static void WalRcvSigHupHandler(SIGNAL_ARGS);
@@ -857,6 +859,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 	XLogRecPtr	walEnd;
 	TimestampTz sendTime;
 	bool		replyRequested;
+	TimestampTz causalReadsLease;
 
 	resetStringInfo(&incoming_message);
 
@@ -877,7 +880,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				walEnd = pq_getmsgint64(&incoming_message);
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, NULL);
 
 				buf += hdrlen;
 				len -= hdrlen;
@@ -889,7 +892,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				bool reportApplyTimestamp = false;
 
 				/* copy message to StringInfo */
-				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
+				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char) + sizeof(int64);
 				if (len != hdrlen)
 					ereport(ERROR,
 							(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -901,8 +904,10 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
 				replyRequested = pq_getmsgbyte(&incoming_message);
+				causalReadsLease = IntegerTimestampToTimestampTz(
+					pq_getmsgint64(&incoming_message));
 
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, &causalReadsLease);
 
 				/*
 				 * If no apply timestamps have been sent at the request of the
@@ -1253,15 +1258,52 @@ XLogWalRcvSendHSFeedback(bool immed)
  * Update shared memory status upon receiving a message from primary.
  *
  * 'walEnd' and 'sendTime' are the end-of-WAL and timestamp of the latest
- * message, reported by primary.
+ * message, reported by primary.  'causalReadsLease' is a pointer to
+ * the time the primary promises that this standby can safely claim to be
+ * causally consistent, to 0 if it cannot, or a NULL pointer for no change.
  */
 static void
-ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
+ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+					  TimestampTz *causalReadsLease)
 {
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
 	static TimestampTz lastRecordedTimestamp = 0;
 
+	/* Sanity check for the causalReadsLease time. */
+	if (causalReadsLease != NULL && *causalReadsLease != 0)
+	{
+		/* Deduce max_clock_skew from the causalReadsLease and sendTime. */
+#ifdef HAVE_INT64_TIMESTAMP
+		int64 diffMillis = (*causalReadsLease - sendTime) / 1000;
+#else
+		int64 diffMillis = (*causalReadsLease - sendTime) * 1000;
+#endif
+		int64 max_clock_skew = diffMillis / (CAUSAL_READS_CLOCK_SKEW_RATIO - 1);
+
+		if (sendTime > TimestampTzPlusMilliseconds(lastMsgReceiptTime, max_clock_skew))
+		{
+			/*
+			 * The primary's clock is more than max_clock_skew + network
+			 * latency ahead of the standby's clock.  (If the primary's clock
+			 * is more than max_clock_skew ahead of the standby's clock, but
+			 * by less than the network latency, then there isn't much we can
+			 * do to detect that; but it still seems useful to have this basic
+			 * sanity check for wildly misconfigured servers.)
+			 */
+			elog(LOG, "the primary server's clock time is too far ahead");
+			causalReadsLease = NULL;
+		}
+		/*
+		 * We could also try to detect cases where sendTime is more than
+		 * max_clock_skew in the past according to the standby's clock, but
+		 * that is indistinguishable from network latency/buffering, so we
+		 * could produce misleading error messages; if we do nothing, the
+		 * consequence is 'standby is not available for causal reads' errors
+		 * which should cause the user to investigate.
+		 */
+	}
+
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
 	if (walrcv->latestWalEnd < walEnd)
@@ -1269,6 +1311,8 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->latestWalEnd = walEnd;
 	walrcv->lastMsgSendTime = sendTime;
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
+	if (causalReadsLease != NULL)
+		walrcv->causalReadsLease = *causalReadsLease;
 	SpinLockRelease(&walrcv->mutex);
 
 	/*
diff --git a/src/backend/replication/walreceiverfuncs.c b/src/backend/replication/walreceiverfuncs.c
index 5f6e423..e502f74 100644
--- a/src/backend/replication/walreceiverfuncs.c
+++ b/src/backend/replication/walreceiverfuncs.c
@@ -28,6 +28,7 @@
 #include "replication/walreceiver.h"
 #include "storage/pmsignal.h"
 #include "storage/shmem.h"
+#include "utils/guc.h"
 #include "utils/timestamp.h"
 
 WalRcvData *WalRcv = NULL;
@@ -374,3 +375,21 @@ GetReplicationTransferLatency(void)
 
 	return ms;
 }
+
+/*
+ * Used by snapmgr to check if this standby has a valid lease, granting it the
+ * right to consider itself available for causal reads.
+ */
+bool
+WalRcvCausalReadsAvailable(void)
+{
+	WalRcvData *walrcv = WalRcv;
+	TimestampTz now = GetCurrentTimestamp();
+	bool result;
+
+	SpinLockAcquire(&walrcv->mutex);
+	result = walrcv->causalReadsLease != 0 && now <= walrcv->causalReadsLease;
+	SpinLockRelease(&walrcv->mutex);
+
+	return result;
+}
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 5bb142d..adb8e3e 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -153,9 +153,20 @@ static StringInfoData tmpbuf;
  */
 static TimestampTz last_reply_timestamp = 0;
 
+static TimestampTz last_keepalive_timestamp = 0;
+
 /* Have we sent a heartbeat message asking for reply, since last reply? */
 static bool waiting_for_ping_response = false;
 
+/* How long do need to stay in JOINING state? */
+static XLogRecPtr causal_reads_joining_until = 0;
+
+/* The last causal reads lease sent to the standby. */
+static TimestampTz causal_reads_last_lease = 0;
+
+/* Is this WALSender listed in causal_reads_standby_names? */
+static bool am_potential_causal_reads_standby = false;
+
 /*
  * While streaming WAL in Copy mode, streamingDoneSending is set to true
  * after we have sent CopyDone. We should not send any more CopyData messages
@@ -241,6 +252,57 @@ InitWalSender(void)
 	SendPostmasterSignal(PMSIGNAL_ADVANCE_STATE_MACHINE);
 }
 
+ /*
+ * If we are exiting unexpectedly, we may need to communicate with concurrent
+ * causal_reads commits to maintain the causal consistency guarantee.
+ */
+static void
+PrepareUncleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * We've lost contact with the standby, but it may still be alive.  We
+		 * can't let any causal_reads transactions return until we've stalled
+		 * for long enough for a zombie standby to start raising errors
+		 * because its lease has expired.
+		 */
+		elog(LOG, "standby \"%s\" is lost (no longer available for causal reads)", application_name);
+		CausalReadsBeginStall(causal_reads_last_lease);
+
+		/*
+		 * We set the state to a lower level _after_ beginning the stall,
+		 * otherwise there would be a tiny window where commits could return
+		 * without observing the stall.
+		 */
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
+ * We are shutting down because we received a goodbye message from the
+ * walreceiver.
+ */
+static void
+PrepareCleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * The standby is shutting down, so it won't be running any more
+		 * transactions.  It is therefore safe to stop waiting for it, and no
+		 * stall is necessary.
+		 */
+		elog(LOG, "standby \"%s\" is leaving (no longer available for causal reads)", application_name);
+
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
 /*
  * Clean up after an error.
  *
@@ -264,7 +326,10 @@ WalSndErrorCleanup(void)
 
 	replication_active = false;
 	if (walsender_ready_to_stop)
+	{
+		PrepareUncleanExit();
 		proc_exit(0);
+	}
 
 	/* Revert back to startup state */
 	WalSndSetState(WALSNDSTATE_STARTUP);
@@ -276,6 +341,8 @@ WalSndErrorCleanup(void)
 static void
 WalSndShutdown(void)
 {
+	PrepareUncleanExit();
+
 	/*
 	 * Reset whereToSendOutput to prevent ereport from attempting to send any
 	 * more messages to the standby.
@@ -1386,6 +1453,7 @@ ProcessRepliesIfAny(void)
 		if (r < 0)
 		{
 			/* unexpected error or EOF */
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1402,6 +1470,7 @@ ProcessRepliesIfAny(void)
 		resetStringInfo(&reply_message);
 		if (pq_getmessage(&reply_message, 0))
 		{
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1451,6 +1520,7 @@ ProcessRepliesIfAny(void)
 				 * 'X' means that the standby is closing down the socket.
 				 */
 			case 'X':
+				PrepareCleanExit();
 				proc_exit(0);
 
 			default:
@@ -1582,6 +1652,83 @@ ProcessStandbyReplyMessage(void)
 	 */
 	{
 		WalSnd *walsnd = MyWalSnd;
+		WalSndCausalReadsState causal_reads_state = walsnd->causal_reads_state;
+		bool causal_reads_state_changed = false;
+		bool causal_reads_set_joining_until = false;
+
+		/*
+		 * Handle causal reads state transitions, if a causal_reads_timeout is
+		 * configured, this standby is listed in causal_reads_standby_names,
+		 * and we are a primary database (not a cascading standby).
+		 */
+		if (am_potential_causal_reads_standby &&
+			!am_cascading_walsender &&
+			applyLagMs >= 0)
+		{
+			if (applyLagMs < causal_reads_timeout)
+			{
+				if (causal_reads_state == WALSNDCRSTATE_UNAVAILABLE)
+				{
+					/*
+					 * The standby is applying fast enough.  We can't grant a
+					 * lease yet though, we need to wait for everything that
+					 * was committed while this standby was unavailable to be
+					 * applied first.  We move to joining state while we wait
+					 * for the standby to catch up.
+					 */
+					causal_reads_state = WALSNDCRSTATE_JOINING;
+					causal_reads_set_joining_until = true;
+					causal_reads_state_changed = true;
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING &&
+						 applyPtr >= causal_reads_joining_until)
+				{
+					/*
+					 * The standby has applied everything committed before we
+					 * reached joining state, and has been waiting for remote
+					 * apply on this standby while it's been in joining state,
+					 * so it is safe to move to available state and send a
+					 * lease.
+					 */
+					causal_reads_state = WALSNDCRSTATE_AVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+			else
+			{
+				if (causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+					/*
+					 * We are dropping a causal reads available standby, so we
+					 * mustn't let any commit command that is waiting in
+					 * CausalReadsWaitForLSN return until we are sure that the
+					 * standby definitely knows that it's not available and
+					 * starts raising errors for causal_reads transactions.
+					 * TODO: We could just wait until the standby acks that
+					 * its lease has been cancelled, and start numbering
+					 * keepalives and sending the number back in replies, so
+					 * we know it's acking the right message; then lagging
+					 * standbys would be less disruptive, but for now we just
+					 * wait for the lease to expire, as we do when we lose
+					 * contact with a standby, for the sake of simplicity.
+					 */
+					CausalReadsBeginStall(causal_reads_last_lease);
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING)
+				{
+					/*
+					 * Dropping a joining standby doesn't require a stall,
+					 * because the standby doesn't think it's available, so
+					 * it's already raising the error for causal_reads
+					 * transactions.
+					 */
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+		}
 
 		SpinLockAcquire(&walsnd->mutex);
 		walsnd->write = writePtr;
@@ -1589,11 +1736,33 @@ ProcessStandbyReplyMessage(void)
 		walsnd->apply = applyPtr;
 		if (applyLagMs >= 0)
 			walsnd->applyLagMs = applyLagMs;
+		walsnd->causal_reads_state = causal_reads_state;
 		SpinLockRelease(&walsnd->mutex);
+
+		if (causal_reads_set_joining_until)
+		{
+			/*
+			 * Record the end of the primary's WAL at some arbitrary point
+			 * observed _after_ we moved to joining state (so that causal
+			 * reads commits start waiting, closing a race).  The standby
+			 * won't become available until it has replayed up to here.
+			 */
+			causal_reads_joining_until = GetFlushRecPtr();
+		}
+
+		if (causal_reads_state_changed)
+		{
+			WalSndKeepalive(true);
+			elog(LOG, "standby \"%s\" is %s", application_name,
+				 causal_reads_state == WALSNDCRSTATE_UNAVAILABLE ? "unavailable for causal reads" :
+				 causal_reads_state == WALSNDCRSTATE_JOINING ? "joining as a causal reads standby..." :
+				 causal_reads_state == WALSNDCRSTATE_AVAILABLE ? "available for causal reads" :
+				 "UNKNOWN");
+		}
 	}
 
 	if (!am_cascading_walsender)
-		SyncRepReleaseWaiters();
+		SyncRepReleaseWaiters(MyWalSnd->causal_reads_state >= WALSNDCRSTATE_JOINING);
 
 	/*
 	 * Advance our local xmin horizon when the client confirmed a flush.
@@ -1734,33 +1903,53 @@ ProcessStandbyHSFeedbackMessage(void)
  * If wal_sender_timeout is enabled we want to wake up in time to send
  * keepalives and to abort the connection if wal_sender_timeout has been
  * reached.
+ *
+ * But if causal_reads_timeout is enabled, we override that and send
+ * keepalives at a constant rate to replace expiring leases.
  */
 static long
 WalSndComputeSleeptime(TimestampTz now)
 {
 	long		sleeptime = 10000;		/* 10 s */
 
-	if (wal_sender_timeout > 0 && last_reply_timestamp > 0)
+	if ((wal_sender_timeout > 0 && last_reply_timestamp > 0) ||
+		am_potential_causal_reads_standby)
 	{
 		TimestampTz wakeup_time;
 		long		sec_to_timeout;
 		int			microsec_to_timeout;
 
-		/*
-		 * At the latest stop sleeping once wal_sender_timeout has been
-		 * reached.
-		 */
-		wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-												  wal_sender_timeout);
-
-		/*
-		 * If no ping has been sent yet, wakeup when it's time to do so.
-		 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
-		 * the timeout passed without a response.
-		 */
-		if (!waiting_for_ping_response)
+		if (am_potential_causal_reads_standby)
+		{
+			/*
+			 * Leases last for a period of between 50% and 100% of
+			 * causal_reads_timeout, depending on clock skew, assuming clock
+			 * skew is under the 25% of causal_reads_timeout.  We send new
+			 * leases every half a lease, so that there are no gaps between
+			 * leases.
+			 */
+			wakeup_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+													  causal_reads_timeout /
+													  CAUSAL_READS_KEEPALIVE_RATIO);
+		}
+		else
+		{
+			/*
+			 * At the latest stop sleeping once wal_sender_timeout has been
+			 * reached.
+			 */
 			wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-													  wal_sender_timeout / 2);
+													  wal_sender_timeout);
+
+			/*
+			 * If no ping has been sent yet, wakeup when it's time to do so.
+			 * WalSndKeepaliveIfNecessary() wants to send a keepalive once
+			 * half of the timeout passed without a response.
+			 */
+			if (!waiting_for_ping_response)
+				wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+														  wal_sender_timeout / 2);
+		}
 
 		/* Compute relative time until wakeup. */
 		TimestampDifference(now, wakeup_time,
@@ -1776,20 +1965,33 @@ WalSndComputeSleeptime(TimestampTz now)
 /*
  * Check whether there have been responses by the client within
  * wal_sender_timeout and shutdown if not.
+ *
+ * If causal_reads_timeout is configured we override that, so that
+ * unresponsive standbys are detected sooner.
  */
 static void
 WalSndCheckTimeOut(TimestampTz now)
 {
 	TimestampTz timeout;
+	int allowed_time;
 
 	/* don't bail out if we're doing something that doesn't require timeouts */
 	if (last_reply_timestamp <= 0)
 		return;
 
-	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
-										  wal_sender_timeout);
+	/*
+	 * If a causal_reads_timeout is configured, it is used instead of
+	 * wal_sender_timeout, to limit the time before an unresponsive causal
+	 * reads standby is dropped.
+	 */
+	if (am_potential_causal_reads_standby)
+		allowed_time = causal_reads_timeout;
+	else
+		allowed_time = wal_sender_timeout;
 
-	if (wal_sender_timeout > 0 && now >= timeout)
+	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
+										  allowed_time);
+	if (allowed_time > 0 && now >= timeout)
 	{
 		/*
 		 * Since typically expiration of replication timeout means
@@ -1822,6 +2024,10 @@ WalSndLoop(WalSndSendDataCallback send_data)
 	last_reply_timestamp = GetCurrentTimestamp();
 	waiting_for_ping_response = false;
 
+	/* Check if we are managing potential causal_reads standby. */
+	am_potential_causal_reads_standby = CausalReadsPotentialStandby();
+	elog(WARNING, "am_potential_causal_reads_standby = %d", am_potential_causal_reads_standby);
+
 	/*
 	 * Loop until we reach the end of this timeline or the client requests to
 	 * stop streaming.
@@ -1982,6 +2188,7 @@ InitWalSenderSlot(void)
 			walsnd->flush = InvalidXLogRecPtr;
 			walsnd->apply = InvalidXLogRecPtr;
 			walsnd->state = WALSNDSTATE_STARTUP;
+			walsnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
 			walsnd->latch = &MyProc->procLatch;
 			SpinLockRelease(&walsnd->mutex);
 			/* don't need the lock anymore */
@@ -2751,6 +2958,24 @@ WalSndGetStateString(WalSndState state)
 	return "UNKNOWN";
 }
 
+/*
+ * Return a string constant representing the causal reads state. This is used
+ * in system views, and should *not* be translated.
+ */
+static const char *
+WalSndGetCausalReadsStateString(WalSndCausalReadsState causal_reads_state)
+{
+	switch (causal_reads_state)
+	{
+		case WALSNDCRSTATE_UNAVAILABLE:
+			return "unavailable";
+		case WALSNDCRSTATE_JOINING:
+			return "joining";
+		case WALSNDCRSTATE_AVAILABLE:
+			return "available";
+	}
+	return "UNKNOWN";
+}
 
 /*
  * Returns activity of walsenders, including pids and xlog locations sent to
@@ -2759,7 +2984,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	9
+#define PG_STAT_GET_WAL_SENDERS_COLS	10
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2810,6 +3035,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		int			applyLagMs;
 		int			priority;
 		WalSndState state;
+		WalSndCausalReadsState causalReadsState;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
 		bool		nulls[PG_STAT_GET_WAL_SENDERS_COLS];
 
@@ -2819,6 +3045,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		SpinLockAcquire(&walsnd->mutex);
 		sentPtr = walsnd->sentPtr;
 		state = walsnd->state;
+		causalReadsState = walsnd->causal_reads_state;
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
@@ -2893,6 +3120,9 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				values[8] = CStringGetTextDatum("sync");
 			else
 				values[8] = CStringGetTextDatum("potential");
+
+			values[9] =
+				CStringGetTextDatum(WalSndGetCausalReadsStateString(causalReadsState));
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
@@ -2912,14 +3142,52 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 static void
 WalSndKeepalive(bool requestReply)
 {
+	TimestampTz now;
+	TimestampTz causal_reads_lease;
+
 	elog(DEBUG2, "sending replication keepalive");
 
+	/*
+	 * If the walsender currently deems the standby to be available for causal
+	 * reads, then it grants a causal reads lease.  The lease authorizes the
+	 * standby to consider itself available for causal reads until a short
+	 * time in the future.  The primary promises to uphold the causal reads
+	 * guarantee until that time, by stalling commits until the the lease has
+	 * expired if necessary.
+	 */
+	now = GetCurrentTimestamp();
+	if (MyWalSnd->causal_reads_state < WALSNDCRSTATE_AVAILABLE)
+		causal_reads_lease = 0; /* Not available, no lease granted. */
+	else
+	{
+		/*
+		 * Since this timestamp is being sent to the standby where it will be
+		 * compared against a time generated by the standby's system clock, we
+		 * must consider clock skew.  First, we decide on a maximum tolerable
+		 * difference between system clocks.  If the primary's clock is ahead
+		 * of the standby's by more than this, then all bets are off (the
+		 * standby could falsely believe it has a valid lease).  If the
+		 * primary's clock is behind the standby's by more than this, then the
+		 * standby will err the other way and generate spurious errors in
+		 * causal_reads mode.  Rather than having a separate GUC for this, we
+		 * derive it from causal_reads_timeout.
+		 */
+		int max_clock_skew = causal_reads_timeout / CAUSAL_READS_CLOCK_SKEW_RATIO;
+
+		/* Compute and remember the expiry time of the lease we're granting. */
+		causal_reads_last_lease = TimestampTzPlusMilliseconds(now, causal_reads_timeout);
+		/* The version we'll send to the standby is adjusted to tolerate clock skew. */
+		causal_reads_lease =
+			TimestampTzPlusMilliseconds(causal_reads_last_lease, -max_clock_skew);
+	}
+
 	/* construct the message... */
 	resetStringInfo(&output_message);
 	pq_sendbyte(&output_message, 'k');
 	pq_sendint64(&output_message, sentPtr);
-	pq_sendint64(&output_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(now));
 	pq_sendbyte(&output_message, requestReply ? 1 : 0);
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(causal_reads_lease));
 
 	/* ... and send it wrapped in CopyData */
 	pq_putmessage_noblock('d', output_message.data, output_message.len);
@@ -2937,23 +3205,35 @@ WalSndKeepaliveIfNecessary(TimestampTz now)
 	 * Don't send keepalive messages if timeouts are globally disabled or
 	 * we're doing something not partaking in timeouts.
 	 */
-	if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
-		return;
-
-	if (waiting_for_ping_response)
-		return;
+	if (!am_potential_causal_reads_standby)
+	{
+		if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
+			return;
+		if (waiting_for_ping_response)
+			return;
+	}
 
 	/*
 	 * If half of wal_sender_timeout has lapsed without receiving any reply
 	 * from the standby, send a keep-alive message to the standby requesting
 	 * an immediate reply.
+	 *
+	 * If causal_reads_timeout has been configured, use it to control
+	 * keepalive intervals rather than wal_sender_timeout, so that we can keep
+	 * replacing leases at the right frequency.
 	 */
-	ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-											wal_sender_timeout / 2);
+	if (am_potential_causal_reads_standby)
+		ping_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+												causal_reads_timeout /
+												CAUSAL_READS_KEEPALIVE_RATIO);
+	else
+		ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+												wal_sender_timeout / 2);
 	if (now >= ping_time)
 	{
 		WalSndKeepalive(true);
 		waiting_for_ping_response = true;
+		last_keepalive_timestamp = now;
 
 		/* Try to flush pending output to the client */
 		if (pq_flush_if_writable() != 0)
diff --git a/src/backend/utils/errcodes.txt b/src/backend/utils/errcodes.txt
index 04c9c00..d4bf0c0 100644
--- a/src/backend/utils/errcodes.txt
+++ b/src/backend/utils/errcodes.txt
@@ -302,6 +302,7 @@ Section: Class 40 - Transaction Rollback
 40001    E    ERRCODE_T_R_SERIALIZATION_FAILURE                              serialization_failure
 40003    E    ERRCODE_T_R_STATEMENT_COMPLETION_UNKNOWN                       statement_completion_unknown
 40P01    E    ERRCODE_T_R_DEADLOCK_DETECTED                                  deadlock_detected
+40P02    E    ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE                         causal_reads_not_available
 
 Section: Class 42 - Syntax Error or Access Rule Violation
 
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index a8eaa5f..cd69889 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -1633,6 +1633,16 @@ static struct config_bool ConfigureNamesBool[] =
 		NULL, NULL, NULL
 	},
 
+	{
+		{"causal_reads", PGC_USERSET, REPLICATION_STANDBY,
+		 gettext_noop("Enables causal reads."),
+		 NULL
+		},
+		&causal_reads,
+		false,
+		NULL, NULL, NULL
+	},
+
 	/* End-of-list marker */
 	{
 		{NULL, 0, 0, NULL, NULL}, NULL, false, NULL, NULL, NULL
@@ -1791,6 +1801,17 @@ static struct config_int ConfigureNamesInt[] =
 	},
 
 	{
+		{"causal_reads_timeout", PGC_SIGHUP, REPLICATION_STANDBY,
+			gettext_noop("Sets the maximum apply lag before causal reads standbys are no longer available."),
+			NULL,
+			GUC_UNIT_MS
+		},
+		&causal_reads_timeout,
+		0, 0, INT_MAX,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"max_connections", PGC_POSTMASTER, CONN_AUTH_SETTINGS,
 			gettext_noop("Sets the maximum number of concurrent connections."),
 			NULL
@@ -3387,10 +3408,21 @@ static struct config_string ConfigureNamesString[] =
 		},
 		&SyncRepStandbyNames,
 		"",
-		check_synchronous_standby_names, NULL, NULL
+		check_standby_names, NULL, NULL
 	},
 
 	{
+		{"causal_reads_standby_names", PGC_SIGHUP, REPLICATION_MASTER,
+			gettext_noop("List of names of potential causal reads standbys."),
+			NULL,
+			GUC_LIST_INPUT
+		},
+		&causal_reads_standby_names,
+		"*",
+		check_standby_names, NULL, NULL
+ 	},
+
+	{
 		{"default_text_search_config", PGC_USERSET, CLIENT_CONN_LOCALE,
 			gettext_noop("Sets default text search configuration."),
 			NULL
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index 085099c..733b9e5 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -244,6 +244,15 @@
 				# from standby(s); '*' = all
 #vacuum_defer_cleanup_age = 0	# number of xacts by which cleanup is delayed
 
+#causal_reads_timeout = 0s      # maximum replication delay to tolerate from
+                                # standbys before dropping them from the set of
+				# available causal reads peers; 0 to disable
+				# causal reads
+
+#causal_reads_standy_names = '*'
+                                # standby servers that can potentially become
+				# available for causal reads; '*' = all
+
 # - Standby Servers -
 
 # These settings are ignored on a master server.
@@ -266,6 +275,14 @@
 #wal_retrieve_retry_interval = 5s	# time to wait before retrying to
 					# retrieve WAL after a failed attempt
 
+# - All Servers -
+
+#causal_reads = off                     # "on" in any pair of consecutive
+                                        # transactions guarantees that the second
+					# can see the first (even if the second
+					# is run on a standby), or will raise an
+					# error to report that the standby is
+					# unavailable for causal reads
 
 #------------------------------------------------------------------------------
 # QUERY TUNING
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index b88e012..6336240 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -46,8 +46,11 @@
 
 #include "access/transam.h"
 #include "access/xact.h"
+#include "access/xlog.h"
 #include "lib/pairingheap.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
+#include "replication/walreceiver.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
 #include "storage/procarray.h"
@@ -209,6 +212,16 @@ GetTransactionSnapshot(void)
 				 "cannot take query snapshot during a parallel operation");
 
 		/*
+		 * In causal_reads mode on a standby, check if we have definitely
+		 * applied WAL for any COMMIT that returned successfully on the
+		 * primary.
+		 */
+		if (causal_reads && RecoveryInProgress() && !WalRcvCausalReadsAvailable())
+			ereport(ERROR,
+					(errcode(ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE),
+					 errmsg("standby is not available for causal reads")));
+
+		/*
 		 * In transaction-snapshot mode, the first snapshot must live until
 		 * end of xact regardless of what the caller does with it, so we must
 		 * make a copy of it rather than returning CurrentSnapshotData
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 811f10c..dba207c 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2712,7 +2712,7 @@ DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f
 DESCR("statistics: information about currently active backends");
 DATA(insert OID = 3318 (  pg_stat_get_progress_info           PGNSP PGUID 12 1 100 0 0 f f f f f t s r 1 0 2249 "25" "{25,23,26,26,20,20,20,20,20,20,20,20,20,20}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{cmdtype,pid,datid,relid,param1,param2,param3,param4,param5,param6,param7,param8,param9,param10}" _null_ _null_ pg_stat_get_progress_info _null_ _null_ _null_ ));
 DESCR("statistics: information about progress of backends running maintenance command");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25}" "{o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25,25}" "{o,o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state,causal_reads_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index 28b68f6..253f831 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -24,14 +24,33 @@
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
 #define SYNC_REP_WAIT_APPLY		2
+#define SYNC_REP_WAIT_CAUSAL_READS 3
 
-#define NUM_SYNC_REP_WAIT_MODE	3
+#define NUM_SYNC_REP_WAIT_MODE	4
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
 #define SYNC_REP_WAITING			1
 #define SYNC_REP_WAIT_COMPLETE		2
 
+/*
+ * ratio of causal_read_timeout to max_clock_skew (4 means than the maximum
+ * tolerated clock difference between primary and standbys using causal_reads
+ * is 1/4 of causal_reads_timeout)
+ */
+#define CAUSAL_READS_CLOCK_SKEW_RATIO 4
+
+/*
+ * ratio of causal_reads_timeout to keepalive time (2 means that the effective
+ * keepalive time is 1/2 of the causal_reads_timeout GUC when it is non-zero)
+ */
+#define CAUSAL_READS_KEEPALIVE_RATIO 2
+
+/* GUC variables */
+extern int causal_reads_timeout;
+extern bool causal_reads;
+extern char *causal_reads_standby_names;
+
 /* user-settable parameters for synchronous replication */
 extern char *SyncRepStandbyNames;
 
@@ -43,16 +62,23 @@ extern void SyncRepCleanupAtProcExit(void);
 
 /* called by wal sender */
 extern void SyncRepInitConfig(void);
-extern void SyncRepReleaseWaiters(void);
+extern void SyncRepReleaseWaiters(bool walsender_cr_available_or_joining);
 
 /* called by checkpointer */
 extern void SyncRepUpdateSyncStandbysDefined(void);
 
+/* called by user backend (xact.c) */
+extern void CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN);
+
+/* called by wal sender */
+extern void CausalReadsBeginStall(TimestampTz lease_expiry_time);
+extern bool CausalReadsPotentialStandby(void);
+
 /* forward declaration to avoid pulling in walsender_private.h */
 struct WalSnd;
 extern struct WalSnd *SyncRepGetSynchronousStandby(void);
 
-extern bool check_synchronous_standby_names(char **newval, void **extra, GucSource source);
+extern bool check_standby_names(char **newval, void **extra, GucSource source);
 extern void assign_synchronous_commit(int newval, void *extra);
 
 #endif   /* _SYNCREP_H */
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 3294df9..7f83934 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -80,6 +80,13 @@ typedef struct
 	TimeLineID	receivedTLI;
 
 	/*
+	 * causalReadsLease is the time until which the primary has authorized
+	 * this standby to consider itself available for causal_reads mode, or 0
+	 * for not authorized.
+	 */
+	TimestampTz causalReadsLease;
+
+	/*
 	 * latestChunkStart is the starting byte position of the current "batch"
 	 * of received WAL.  It's actually the same as the previous value of
 	 * receivedUpto before the last flush to disk.  Startup process can use
@@ -164,4 +171,6 @@ extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
 extern void WalRcvWakeup(void);
 
+extern bool WalRcvCausalReadsAvailable(void);
+
 #endif   /* _WALRECEIVER_H */
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 88ccdf0..81a2776 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -27,6 +27,13 @@ typedef enum WalSndState
 	WALSNDSTATE_STREAMING
 } WalSndState;
 
+typedef enum WalSndCausalReadsState
+{
+	WALSNDCRSTATE_UNAVAILABLE = 0,
+	WALSNDCRSTATE_JOINING,
+	WALSNDCRSTATE_AVAILABLE
+} WalSndCausalReadsState;
+
 /*
  * Each walsender has a WalSnd struct in shared memory.
  */
@@ -34,6 +41,7 @@ typedef struct WalSnd
 {
 	pid_t		pid;			/* this walsender's process id, or 0 */
 	WalSndState state;			/* this walsender's state */
+	WalSndCausalReadsState causal_reads_state; /* the walsender's causal reads state */
 	XLogRecPtr	sentPtr;		/* WAL has been sent up to this point */
 	bool		needreload;		/* does currently-open file need to be
 								 * reloaded? */
@@ -89,6 +97,12 @@ typedef struct
 	 */
 	bool		sync_standbys_defined;
 
+	/*
+	 * Until when must commits in causal_reads stall?  This is used to wait
+	 * for causal reads leases to expire.
+	 */
+	TimestampTz	stall_causal_reads_until;
+
 	WalSnd		walsnds[FLEXIBLE_ARRAY_MEMBER];
 } WalSndCtlData;
 
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 6f80548..2727533 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1762,10 +1762,11 @@ pg_stat_replication| SELECT s.pid,
     w.replay_location,
     w.replay_lag,
     w.sync_priority,
-    w.sync_state
+    w.sync_state,
+    w.causal_reads_state
    FROM pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, waiting, xact_start, query_start, backend_start, state_change, client_addr, client_hostname, client_port, backend_xid, backend_xmin, ssl, sslversion, sslcipher, sslbits, sslcompression, sslclientdn),
     pg_authid u,
-    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state)
+    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state, causal_reads_state)
   WHERE ((s.usesysid = u.oid) AND (s.pid = w.pid));
 pg_stat_ssl| SELECT s.pid,
     s.ssl,
test-causal-reads.ctext/x-csrc; charset=US-ASCII; name=test-causal-reads.cDownload
#47Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Thomas Munro (#46)
4 attachment(s)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Thu, Mar 10, 2016 at 12:35 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

I looked into COMMIT PREPARED replay feedback and realised that it
doesn't need any special handling beyond what is already in
xact_redo_commit. However, I see now that I *do* need to do something
when replaying PREPARE TRANSACTION, as you said. Not for causal reads
though -- it doesn't care about an operation with no visible effect --
but for synchronous_commit = remote_apply. I am thinking about how to
fix that.

Ok, here is a new version with the following changes:

1. If you PREPARE TRANSACTION with synchronous_commit = remote_apply,
it just waits for remote WAL flush. When you eventually
COMMIT/ROLLBACK PREPARED, it waits for remote apply. Also, the
XACT_COMPLETION_SYNC_APPLY_FEEDBACK bit is now set in appropriate
abort records, and then handled in recovery, just as for commits, so
that ROLLBACK and ROLLBACK PREPARED return at the right time in
remote_apply.

2. I fixed a recently introduced stupid bug that caused causal reads
to be broken when sync rep wasn't also configured (I had lost a change
to stop SyncRepReleaseWaiters from leaving early, when I split up the
patch... oops).

3. I switched the pg_stat_replication.replay_lag accounting from
milliseconds to microseconds. The measured lag with fast machines on
local networks can sometimes be sub-millisecond or very low numbers of
milliseconds, so it's interesting to see more detail (depending on the
primary's clock resolution).

--
Thomas Munro
http://www.enterprisedb.com

Attachments:

0001-remote-apply-v3.patchapplication/octet-stream; name=0001-remote-apply-v3.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index a09ceb2..b21b4c0 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -2091,7 +2091,7 @@ include_dir 'conf.d'
         Specifies whether transaction commit will wait for WAL records
         to be written to disk before the command returns a <quote>success</>
         indication to the client.  Valid values are <literal>on</>,
-        <literal>remote_write</>, <literal>local</>, and <literal>off</>.
+        <literal>remote_write</>, <literal> remote_apply</>, <literal>local</>, and <literal>off</>.
         The default, and safe, setting
         is <literal>on</>.  When <literal>off</>, there can be a delay between
         when success is reported to the client and when the transaction is
@@ -2125,6 +2125,10 @@ include_dir 'conf.d'
         ensure data preservation even if the standby instance of
         <productname>PostgreSQL</> were to crash, but not if the standby
         suffers an operating-system-level crash.
+        When set to <literal>remote_apply</>, commits will wait until a reply
+        from the current synchronous stanbyindicates it has received the
+        commit record of the transaction and applied it, so that it has become
+        visible to queries.
        </para>
        <para>
         When synchronous
diff --git a/doc/src/sgml/high-availability.sgml b/doc/src/sgml/high-availability.sgml
index 6cb690c..2600fba 100644
--- a/doc/src/sgml/high-availability.sgml
+++ b/doc/src/sgml/high-availability.sgml
@@ -1081,6 +1081,9 @@ primary_slot_name = 'node_a_slot'
     WAL record is then sent to the standby. The standby sends reply
     messages each time a new batch of WAL data is written to disk, unless
     <varname>wal_receiver_status_interval</> is set to zero on the standby.
+    In the case that <varname>synchronous_commit</> is set to
+    <literal>remote_apply</>, the standby sends reply messages when the commit
+    record is replayed, making the transaction visible.
     If the standby is the first matching standby, as specified in
     <varname>synchronous_standby_names</> on the primary, the reply
     messages from that standby will be used to wake users waiting for
@@ -1107,6 +1110,14 @@ primary_slot_name = 'node_a_slot'
    </para>
 
    <para>
+    Setting <varname>synchronous_commit</> to <literal>remote_apply</> will
+    cause each commit to wait until the current synchronous standby reports
+    that it has replayed the transaction, making it visible to user queries.
+    In simple cases, this allows for load balancing with causal consistency
+    on a single hot standby.
+   </para>
+
+   <para>
     Users will stop waiting if a fast shutdown is requested.  However, as
     when using asynchronous replication, the server will not fully
     shutdown until all outstanding WAL records are transferred to the currently
@@ -1160,9 +1171,10 @@ primary_slot_name = 'node_a_slot'
     <title>Planning for High Availability</title>
 
    <para>
-    Commits made when <varname>synchronous_commit</> is set to <literal>on</>
-    or <literal>remote_write</> will wait until the synchronous standby responds. The response
-    may never occur if the last, or only, standby should crash.
+    Commits made when <varname>synchronous_commit</> is set to <literal>on</>,
+    <literal>remote_write</> or <literal>remote_apply</> will wait until the
+    synchronous standby responds. The response may never occur if the last, or
+    only, standby should crash.
    </para>
 
    <para>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 8a22836..1330a45 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -1103,7 +1103,20 @@ EndPrepare(GlobalTransaction gxact)
 	 * Note that at this stage we have marked the prepare, but still show as
 	 * running in the procarray (twice!) and continue to hold locks.
 	 */
-	SyncRepWaitForLSN(gxact->prepare_end_lsn);
+	{
+		/*
+		 * Don't wait for the prepare to be applied remotely in remote_apply
+		 * mode, just wait for it to be flushed to the WAL.  We will wait for
+		 * apply when the transaction eventuallly commits or aborts.
+		 */
+		if (synchronous_commit == SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+			assign_synchronous_commit(SYNCHRONOUS_COMMIT_REMOTE_FLUSH, NULL);
+
+		SyncRepWaitForLSN(gxact->prepare_end_lsn);
+
+		if (synchronous_commit == SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+			assign_synchronous_commit(SYNCHRONOUS_COMMIT_REMOTE_APPLY, NULL);
+	}
 
 	records.tail = records.head = NULL;
 	records.num_chunks = 0;
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index b491735..551e350 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -5121,6 +5121,13 @@ XactLogCommitRecord(TimestampTz commit_time,
 		xl_xinfo.xinfo |= XACT_COMPLETION_FORCE_SYNC_COMMIT;
 
 	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this commit is applied.
+	*/
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
+	/*
 	 * Relcache invalidations requires information about the current database
 	 * and so does logical decoding.
 	 */
@@ -5298,6 +5305,13 @@ XactLogAbortRecord(TimestampTz abort_time,
 	if (xl_xinfo.xinfo & XACT_XINFO_HAS_TWOPHASE)
 		XLogRegisterData((char *) (&xl_twophase), sizeof(xl_xact_twophase));
 
+	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this abort is applied.
+	*/
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
 	return XLogInsert(RM_XACT_ID, info);
 }
 
@@ -5456,6 +5470,13 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 	if (XactCompletionForceSyncCommit(parsed->xinfo))
 		XLogFlush(lsn);
 
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit = remote_apply), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 /*
@@ -5542,6 +5563,14 @@ xact_redo_abort(xl_xact_parsed_abort *parsed, TransactionId xid)
 		smgrdounlink(srel, true);
 		smgrclose(srel);
 	}
+
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit = remote_apply), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 void
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 5b1c361..5744429 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -346,6 +346,12 @@ static XLogRecPtr RedoRecPtr;
 static bool doPageWrites;
 
 /*
+ * doRequestWalReceiverReply is used by recovery code to ask the main recovery
+ * loop to trigger a walreceiver reply.
+ */
+static bool doRequestWalReceiverReply;
+
+/*
  * RedoStartLSN points to the checkpoint's REDO location which is specified
  * in a backup label file, backup history file or control file. In standby
  * mode, XLOG streaming usually starts from the position where an invalid
@@ -6879,6 +6885,19 @@ StartupXLOG(void)
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
 				SpinLockRelease(&XLogCtl->info_lck);
 
+				/*
+				 * If rm_redo reported that it applied a commit record that
+				 * the master is waiting for by calling
+				 * XLogRequestWalReceiverReply, then we wake up the receiver
+				 * so that it notices the updated lastReplayedEndRecPtr and
+				 * sends a reply to the master.
+				 */
+				if (doRequestWalReceiverReply)
+				{
+					doRequestWalReceiverReply = false;
+					WalRcvWakeup();
+				}
+
 				/* Remember this record as the last-applied one */
 				LastRec = ReadRecPtr;
 
@@ -11596,3 +11615,13 @@ SetWalWriterSleeping(bool sleeping)
 	XLogCtl->WalWriterSleeping = sleeping;
 	SpinLockRelease(&XLogCtl->info_lck);
 }
+
+/*
+ * Called by redo code to indicate that the xlog replay loop should wake up
+ * the walreceiver process so that a reply can be sent to the primary.
+ */
+void
+XLogRequestWalReceiverReply(void)
+{
+	doRequestWalReceiverReply = true;
+}
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 92faf4e..4565348 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -416,6 +416,7 @@ SyncRepReleaseWaiters(void)
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
+	int			numapply = 0;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
@@ -462,12 +463,18 @@ SyncRepReleaseWaiters(void)
 		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
 		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
 	}
+	if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+	{
+		walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+		numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY);
+	}
 
 	LWLockRelease(SyncRepLock);
 
-	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X",
+	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%x",
 		 numwrite, (uint32) (MyWalSnd->write >> 32), (uint32) MyWalSnd->write,
-	   numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush);
+		 numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush,
+		 numapply, (uint32) (MyWalSnd->apply >> 32), (uint32) MyWalSnd->apply);
 
 	/*
 	 * If we are managing the highest priority standby, though we weren't
@@ -728,6 +735,9 @@ assign_synchronous_commit(int newval, void *extra)
 		case SYNCHRONOUS_COMMIT_REMOTE_FLUSH:
 			SyncRepWaitMode = SYNC_REP_WAIT_FLUSH;
 			break;
+		case SYNCHRONOUS_COMMIT_REMOTE_APPLY:
+			SyncRepWaitMode = SYNC_REP_WAIT_APPLY;
+			break;
 		default:
 			SyncRepWaitMode = SYNC_REP_NO_WAIT;
 			break;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 7b36e02..c19842e 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -101,6 +101,7 @@ static uint32 recvOff = 0;
  */
 static volatile sig_atomic_t got_SIGHUP = false;
 static volatile sig_atomic_t got_SIGTERM = false;
+static volatile sig_atomic_t got_SIGUSR2 = false;
 
 /*
  * LogstreamResult indicates the byte positions that we have already
@@ -150,9 +151,27 @@ static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
 /* Signal handlers */
 static void WalRcvSigHupHandler(SIGNAL_ARGS);
 static void WalRcvSigUsr1Handler(SIGNAL_ARGS);
+static void WalRcvSigUsr2Handler(SIGNAL_ARGS);
 static void WalRcvShutdownHandler(SIGNAL_ARGS);
 static void WalRcvQuickDieHandler(SIGNAL_ARGS);
 
+static void WalRcvBlockSigUsr2(void)
+{
+	sigset_t mask;
+
+	sigemptyset(&mask);
+	sigaddset(&mask, SIGUSR2);
+	sigprocmask(SIG_BLOCK, &mask, NULL);
+}
+
+static void WalRcvUnblockSigUsr2(void)
+{
+	sigset_t mask;
+
+	sigemptyset(&mask);
+	sigaddset(&mask, SIGUSR2);
+	sigprocmask(SIG_UNBLOCK, &mask, NULL);
+}
 
 static void
 ProcessWalRcvInterrupts(void)
@@ -200,6 +219,7 @@ WalReceiverMain(void)
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz last_recv_timestamp;
 	bool		ping_sent;
+	bool		forceReply;
 
 	/*
 	 * WalRcv should be set up already (if we are a backend, we inherit this
@@ -268,7 +288,7 @@ WalReceiverMain(void)
 	pqsignal(SIGALRM, SIG_IGN);
 	pqsignal(SIGPIPE, SIG_IGN);
 	pqsignal(SIGUSR1, WalRcvSigUsr1Handler);
-	pqsignal(SIGUSR2, SIG_IGN);
+	pqsignal(SIGUSR2, WalRcvSigUsr2Handler);
 
 	/* Reset some signals that are accepted by postmaster but not here */
 	pqsignal(SIGCHLD, SIG_DFL);
@@ -299,6 +319,10 @@ WalReceiverMain(void)
 	/* Unblock signals (they were blocked when the postmaster forked us) */
 	PG_SETMASK(&UnBlockSig);
 
+	/* Block SIGUSR2 (we unblock it only during network waits). */
+	WalRcvBlockSigUsr2();
+	got_SIGUSR2 = false;
+
 	/* Establish the connection to the primary for XLOG streaming */
 	EnableWalRcvImmediateExit();
 	walrcv_connect(conninfo);
@@ -408,7 +432,9 @@ WalReceiverMain(void)
 				}
 
 				/* Wait a while for data to arrive */
+				WalRcvUnblockSigUsr2();
 				len = walrcv_receive(NAPTIME_PER_CYCLE, &buf);
+				WalRcvBlockSigUsr2();
 				if (len != 0)
 				{
 					/*
@@ -439,11 +465,21 @@ WalReceiverMain(void)
 							endofwal = true;
 							break;
 						}
+						WalRcvUnblockSigUsr2();
 						len = walrcv_receive(0, &buf);
+						WalRcvBlockSigUsr2();
+					}
+
+					if (got_SIGUSR2)
+					{
+						/* The recovery process asked us to force a reply. */
+						got_SIGUSR2 = false;
+						forceReply = true;
 					}
 
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(false, false);
+					XLogWalRcvSendReply(forceReply, false);
+					forceReply = false;
 
 					/*
 					 * If we've written some records, flush them to disk and
@@ -498,7 +534,14 @@ WalReceiverMain(void)
 						}
 					}
 
-					XLogWalRcvSendReply(requestReply, requestReply);
+					if (got_SIGUSR2)
+					{
+						/* The recovery process asked us to force a reply. */
+						got_SIGUSR2 = false;
+						forceReply = true;
+					}
+					XLogWalRcvSendReply(requestReply || forceReply, requestReply);
+					forceReply = false;
 					XLogWalRcvSendHSFeedback(false);
 				}
 			}
@@ -740,6 +783,13 @@ WalRcvSigUsr1Handler(SIGNAL_ARGS)
 	errno = save_errno;
 }
 
+/* SIGUSR2: used to receive wakeups from recovery */
+static void
+WalRcvSigUsr2Handler(SIGNAL_ARGS)
+{
+	got_SIGUSR2 = true;
+}
+
 /* SIGTERM: set flag for main loop, or shutdown immediately if safe */
 static void
 WalRcvShutdownHandler(SIGNAL_ARGS)
@@ -1222,6 +1272,22 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 }
 
 /*
+ * Wake up the walreceiver if it happens to be blocked in walrcv_receive,
+ * and tell it that a commit record has been applied.
+ *
+ * This is called by the startup process whenever interesting xlog records
+ * are applied, so that walreceiver can check if it needs to send an apply
+ * notification back to the master which may be waiting in a COMMIT with
+ * synchronous_commit = remote_apply.
+ */
+void
+WalRcvWakeup(void)
+{
+	if (WalRcv->pid != 0)
+		kill(WalRcv->pid, SIGUSR2);
+}
+
+/*
  * Return a string constant representing the state. This is used
  * in system functions and views, and should *not* be translated.
  */
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 0be64a1..acedccd 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -345,12 +345,13 @@ static const struct config_enum_entry constraint_exclusion_options[] = {
 };
 
 /*
- * Although only "on", "off", "remote_write", and "local" are documented, we
- * accept all the likely variants of "on" and "off".
+ * Although only "on", "off", "remote_apply", "remote_write", and "local" are
+ * documented, we accept all the likely variants of "on" and "off".
  */
 static const struct config_enum_entry synchronous_commit_options[] = {
 	{"local", SYNCHRONOUS_COMMIT_LOCAL_FLUSH, false},
 	{"remote_write", SYNCHRONOUS_COMMIT_REMOTE_WRITE, false},
+	{"remote_apply", SYNCHRONOUS_COMMIT_REMOTE_APPLY, false},
 	{"on", SYNCHRONOUS_COMMIT_ON, false},
 	{"off", SYNCHRONOUS_COMMIT_OFF, false},
 	{"true", SYNCHRONOUS_COMMIT_ON, true},
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index ee3d378..085099c 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -177,7 +177,7 @@
 					# (change requires restart)
 #fsync = on				# turns forced synchronization on or off
 #synchronous_commit = on		# synchronization level;
-					# off, local, remote_write, or on
+					# off, local, remote_write, remote_apply, or on
 #wal_sync_method = fsync		# the default is the first option
 					# supported by the operating system:
 					#   open_datasync
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index ebeb582..ed8d22c 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -60,7 +60,9 @@ typedef enum
 	SYNCHRONOUS_COMMIT_LOCAL_FLUSH,		/* wait for local flush only */
 	SYNCHRONOUS_COMMIT_REMOTE_WRITE,	/* wait for local flush and remote
 										 * write */
-	SYNCHRONOUS_COMMIT_REMOTE_FLUSH		/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_FLUSH,	/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_APPLY		/* wait for local flush and remote
+										 * apply */
 }	SyncCommitLevel;
 
 /* Define the default setting for synchonous_commit */
@@ -144,10 +146,13 @@ typedef void (*SubXactCallback) (SubXactEvent event, SubTransactionId mySubid,
  * EOXact... routines which run at the end of the original transaction
  * completion.
  */
+#define XACT_COMPLETION_SYNC_APPLY_FEEDBACK		(1U << 29)
 #define XACT_COMPLETION_UPDATE_RELCACHE_FILE	(1U << 30)
 #define XACT_COMPLETION_FORCE_SYNC_COMMIT		(1U << 31)
 
 /* Access macros for above flags */
+#define XactCompletionSyncApplyFeedback(xinfo) \
+	(!!(xinfo & XACT_COMPLETION_SYNC_APPLY_FEEDBACK))
 #define XactCompletionRelcacheInitFileInval(xinfo) \
 	(!!(xinfo & XACT_COMPLETION_UPDATE_RELCACHE_FILE))
 #define XactCompletionForceSyncCommit(xinfo) \
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index ecd30ce..68e20e4 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -268,6 +268,8 @@ extern bool CheckPromoteSignal(void);
 extern void WakeupRecovery(void);
 extern void SetWalWriterSleeping(bool sleeping);
 
+extern void XLogRequestWalReceiverReply(void);
+
 extern void assign_max_wal_size(int newval, void *extra);
 extern void assign_checkpoint_completion_target(double newval, void *extra);
 
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index 96e059b..28b68f6 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -23,8 +23,9 @@
 #define SYNC_REP_NO_WAIT		-1
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
+#define SYNC_REP_WAIT_APPLY		2
 
-#define NUM_SYNC_REP_WAIT_MODE	2
+#define NUM_SYNC_REP_WAIT_MODE	3
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 6eacb09..3294df9 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -162,5 +162,6 @@ extern void RequestXLogStreaming(TimeLineID tli, XLogRecPtr recptr,
 extern XLogRecPtr GetWalRcvWriteRecPtr(XLogRecPtr *latestChunkStart, TimeLineID *receiveTLI);
 extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
+extern void WalRcvWakeup(void);
 
 #endif   /* _WALRECEIVER_H */
0002-replay-lag-v3.patchapplication/octet-stream; name=0002-replay-lag-v3.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 85459d0..818b952 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -820,6 +820,12 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       standby server</entry>
     </row>
     <row>
+     <entry><structfield>replay_lag</></entry>
+     <entry><type>interval</></entry>
+     <entry>Estimated time taken for recent WAL records to be replayed on this
+      standby server</entry>
+    </row>
+    <row>
      <entry><structfield>sync_priority</></entry>
      <entry><type>integer</></entry>
      <entry>Priority of this standby server for being chosen as the
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 551e350..939d71c 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -5471,6 +5471,12 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 		XLogFlush(lsn);
 
 	/*
+	 * Record the primary's timestamp for the commit record, so it can be used
+	 * for tracking replay lag.
+	 */
+	SetXLogReplayTimestamp(parsed->xact_time);
+
+	/*
 	 * If asked by the primary (because someone is waiting for a synchronous
 	 * commit = remote_apply), we will need to ask walreceiver to send a
 	 * reply immediately.
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 5744429..5abfc61 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -81,6 +81,8 @@ extern uint32 bootstrap_data_checksum_version;
 #define PROMOTE_SIGNAL_FILE		"promote"
 #define FALLBACK_PROMOTE_SIGNAL_FILE "fallback_promote"
 
+/* Size of the circular buffer of timestamped LSNs. */
+#define MAX_TIMESTAMPED_LSNS 8192
 
 /* User-settable parameters */
 int			max_wal_size = 64;	/* 1 GB */
@@ -363,6 +365,13 @@ static bool doRequestWalReceiverReply;
  */
 static XLogRecPtr RedoStartLSN = InvalidXLogRecPtr;
 
+/*
+ * LastReplayedTimestamp can be set by redo handlers when they apply a record
+ * that carries a timestamp, by calling SetXLogReplayedTimestamp.  The xlog
+ * apply loop can then update the value in shared memory.
+ */
+static TimestampTz LastReplayedTimestamp = 0;
+
 /*----------
  * Shared-memory data structures for XLOG control
  *
@@ -637,6 +646,21 @@ typedef struct XLogCtlData
 	/* current effective recovery target timeline */
 	TimeLineID	RecoveryTargetTLI;
 
+	/* timestamp from the most recently applied record associated with a timestamp. */
+	TimestampTz lastReplayedTimestamp;
+
+	/*
+	 * We maintain a circular buffer of LSNs and associated timestamps.
+	 * Walreceiver writes into it using information from timestamps, and the
+	 * startup recovery process reads from it and notifies walreceiver when
+	 * LSNs are replayed so that the timestamps can eventually be fed back to
+	 * the upstream server, to track lag.
+	 */
+	Index			timestampedLsnRead;
+	Index			timestampedLsnWrite;
+	XLogRecPtr		timestampedLsn[MAX_TIMESTAMPED_LSNS];
+	TimestampTz		timestampedLsnTime[MAX_TIMESTAMPED_LSNS];
+
 	/*
 	 * timestamp of when we started replaying the current chunk of WAL data,
 	 * only relevant for replication or archive recovery
@@ -6877,20 +6901,51 @@ StartupXLOG(void)
 				error_context_stack = errcallback.previous;
 
 				/*
-				 * Update lastReplayedEndRecPtr after this record has been
-				 * successfully replayed.
+				 * Update lastReplayedEndRecPtr and lastReplayedTimestamp
+				 * after this record has been successfully replayed.
 				 */
 				SpinLockAcquire(&XLogCtl->info_lck);
 				XLogCtl->lastReplayedEndRecPtr = EndRecPtr;
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
+				if (LastReplayedTimestamp != 0)
+				{
+					/* If replaying a record produced a timestamp, use that. */
+					XLogCtl->lastReplayedTimestamp = LastReplayedTimestamp;
+					LastReplayedTimestamp = 0;
+				}
+				else
+				{
+					/*
+					 * If we have applied LSNs associated with timestamps
+					 * received by walreceiver, then use the recorded
+					 * timestamp.  We consume from the read end of the
+					 * circular buffer.
+					 */
+					while (XLogCtl->timestampedLsnRead !=
+						   XLogCtl->timestampedLsnWrite &&
+						   XLogCtl->timestampedLsn[XLogCtl->timestampedLsnRead]
+						   <= EndRecPtr)
+					{
+						if (XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead] >
+							XLogCtl->lastReplayedTimestamp)
+						{
+							XLogCtl->lastReplayedTimestamp =
+								XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead];
+							doRequestWalReceiverReply = true;
+						}
+						XLogCtl->timestampedLsnRead =
+							(XLogCtl->timestampedLsnRead + 1) % MAX_TIMESTAMPED_LSNS;
+					}
+				}
 				SpinLockRelease(&XLogCtl->info_lck);
 
 				/*
 				 * If rm_redo reported that it applied a commit record that
 				 * the master is waiting for by calling
-				 * XLogRequestWalReceiverReply, then we wake up the receiver
-				 * so that it notices the updated lastReplayedEndRecPtr and
-				 * sends a reply to the master.
+				 * XLogRequestWalReceiverReply, or we encountered a WAL
+				 * location that was associated with a timestamp above, then
+				 * we wake up the receiver so that it notices the updated
+				 * lastReplayedEndRecPtr and sends a reply to the master.
 				 */
 				if (doRequestWalReceiverReply)
 				{
@@ -11625,3 +11680,91 @@ XLogRequestWalReceiverReply(void)
 {
 	doRequestWalReceiverReply = true;
 }
+
+/*
+ * Record the timestamp that is associated with a WAL position.
+ *
+ * This is called by walreceiver on standby servers when keepalive messages
+ * arrive, using timestamps generated on the primary server.  The timestamp
+ * will be sent back to the primary server when the standby had applied this
+ * WAL position.  The primary can use the elapsed time to estimate the replay
+ * lag.
+ */
+void
+SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn)
+{
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn == XLogCtl->lastReplayedEndRecPtr)
+	{
+		/*
+		 * That is the last replayed LSN: we are fully replayed, so we can
+		 * update the replay timestamp immediately.
+		 */
+		XLogCtl->lastReplayedTimestamp = timestamp;
+	}
+	else
+	{
+		/*
+		 * There is WAL still to be applied.  We will associate the timestamp
+		 * with this WAL position and wait for it to be replayed.  We add it
+		 * at the 'write' end of the circular buffer of LSN/timestamp
+		 * mappings, which the replay loop will eventually read.
+		 */
+		Index w = XLogCtl->timestampedLsnWrite;
+		Index r = XLogCtl->timestampedLsnRead;
+
+		XLogCtl->timestampedLsn[w] = lsn;
+		XLogCtl->timestampedLsnTime[w] = timestamp;
+
+		/* Advance the write point. */
+		w = (w + 1) % MAX_TIMESTAMPED_LSNS;
+		XLogCtl->timestampedLsnWrite = w;
+		if (w == r)
+		{
+			/*
+			 * The buffer is full.  Advance the read point (throwing away
+			 * oldest values; we will begin to overestimate replay lag, until
+			 * lag decreases to a size our buffer can manage, or the next
+			 * commit record is replayed).
+			 */
+			r = (r + 1) % MAX_TIMESTAMPED_LSNS;
+			XLogCtl->timestampedLsnRead = r;
+		}
+	}
+	SpinLockRelease(&XLogCtl->info_lck);
+}
+
+/*
+ * Set the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary.  This can be called by redo handlers that have
+ * an appropriate timestamp (currently only commit records).  Updating the
+ * shared memory value is deferred until after the redo handler returns.
+ */
+void
+SetXLogReplayTimestamp(TimestampTz timestamp)
+{
+	LastReplayedTimestamp = timestamp;
+}
+
+/*
+ * Get the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary, and also the most recently applied LSN.  (Note
+ * that the timestamp and the LSN don't necessarily relate to the same
+ * record.)
+ *
+ * This is similar to GetLatestXTime, except that it is not only advanced by
+ * commit records (see SetXLogReplayTimestampAtLsn).
+ */
+TimestampTz
+GetXLogReplayTimestamp(XLogRecPtr *lsn)
+{
+	TimestampTz result;
+
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn)
+		*lsn = XLogCtl->lastReplayedEndRecPtr;
+	result = XLogCtl->lastReplayedTimestamp;
+	SpinLockRelease(&XLogCtl->info_lck);
+
+	return result;
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index abf9a70..f5a7c5c 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -661,6 +661,7 @@ CREATE VIEW pg_stat_replication AS
             W.write_location,
             W.flush_location,
             W.replay_location,
+	    W.replay_lag,
             W.sync_priority,
             W.sync_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index c19842e..d061dfe 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -85,6 +85,8 @@ walrcv_disconnect_type walrcv_disconnect = NULL;
 
 #define NAPTIME_PER_CYCLE 100	/* max sleep time between cycles (100ms) */
 
+#define MIN_TIME_BETWEEN_TIMESTAMPED_LSNS 1000 /* 1s */
+
 /*
  * These variables are used similarly to openLogFile/SegNo/Off,
  * but for walreceiver to write the XLOG. recvFileTLI is the TimeLineID
@@ -103,6 +105,8 @@ static volatile sig_atomic_t got_SIGHUP = false;
 static volatile sig_atomic_t got_SIGTERM = false;
 static volatile sig_atomic_t got_SIGUSR2 = false;
 
+static bool reply_sent_on_recovery_activity = false;
+
 /*
  * LogstreamResult indicates the byte positions that we have already
  * written/fsynced.
@@ -144,7 +148,7 @@ static void WalRcvDie(int code, Datum arg);
 static void XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len);
 static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
-static void XLogWalRcvSendReply(bool force, bool requestReply);
+static void XLogWalRcvSendReply(bool force, bool requestReply, bool includeApplyTimestamp);
 static void XLogWalRcvSendHSFeedback(bool immed);
 static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
 
@@ -219,7 +223,7 @@ WalReceiverMain(void)
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz last_recv_timestamp;
 	bool		ping_sent;
-	bool		forceReply;
+	bool		timestampedWalApplied;
 
 	/*
 	 * WalRcv should be set up already (if we are a backend, we inherit this
@@ -472,14 +476,15 @@ WalReceiverMain(void)
 
 					if (got_SIGUSR2)
 					{
-						/* The recovery process asked us to force a reply. */
+						/* The recovery process asked us to report an applied timestamp. */
 						got_SIGUSR2 = false;
-						forceReply = true;
+						timestampedWalApplied = true;
+						reply_sent_on_recovery_activity = true;
 					}
 
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(forceReply, false);
-					forceReply = false;
+					XLogWalRcvSendReply(timestampedWalApplied, false, timestampedWalApplied);
+					timestampedWalApplied = false;
 
 					/*
 					 * If we've written some records, flush them to disk and
@@ -536,12 +541,14 @@ WalReceiverMain(void)
 
 					if (got_SIGUSR2)
 					{
-						/* The recovery process asked us to force a reply. */
+						/* The recovery process asked us to report an apply timestamp. */
 						got_SIGUSR2 = false;
-						forceReply = true;
+						timestampedWalApplied = true;
+						reply_sent_on_recovery_activity = true;
 					}
-					XLogWalRcvSendReply(requestReply || forceReply, requestReply);
-					forceReply = false;
+					XLogWalRcvSendReply(requestReply || timestampedWalApplied, requestReply,
+										timestampedWalApplied);
+					timestampedWalApplied = false;
 					XLogWalRcvSendHSFeedback(false);
 				}
 			}
@@ -879,6 +886,8 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 			}
 		case 'k':				/* Keepalive */
 			{
+				bool reportApplyTimestamp = false;
+
 				/* copy message to StringInfo */
 				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
 				if (len != hdrlen)
@@ -895,9 +904,22 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 
 				ProcessWalSndrMessage(walEnd, sendTime);
 
-				/* If the primary requested a reply, send one immediately */
-				if (replyRequested)
-					XLogWalRcvSendReply(true, false);
+				/*
+				 * If no apply timestamps have been sent at the request of the
+				 * recovery process since we last received a keepalive, then
+				 * we will send one now.  This allows us to feed back
+				 * timestamps in response to pings if we are idle or if the
+				 * recovery process is somehow blocked, but we don't want to
+				 * do that if it's actively applying and periodically waking
+				 * us up.
+				 */
+				if (!reply_sent_on_recovery_activity)
+					reportApplyTimestamp = true;
+				reply_sent_on_recovery_activity = false;
+
+				/* If the primary requested a reply, send one immediately. */
+				if (replyRequested || reportApplyTimestamp)
+					XLogWalRcvSendReply(true, false, reportApplyTimestamp);
 				break;
 			}
 		default:
@@ -1060,7 +1082,7 @@ XLogWalRcvFlush(bool dying)
 		/* Also let the master know that we made some progress */
 		if (!dying)
 		{
-			XLogWalRcvSendReply(false, false);
+			XLogWalRcvSendReply(false, false, false);
 			XLogWalRcvSendHSFeedback(false);
 		}
 	}
@@ -1078,15 +1100,20 @@ XLogWalRcvFlush(bool dying)
  * If 'requestReply' is true, requests the server to reply immediately upon
  * receiving this message. This is used for heartbearts, when approaching
  * wal_receiver_timeout.
+ *
+ * If 'reportApplyTimestamp' is true, the latest apply timestamp is included.
+ * This is set to true only when this function is called after the recovery
+ * process has replayed a record with an associated timestamp.
  */
 static void
-XLogWalRcvSendReply(bool force, bool requestReply)
+XLogWalRcvSendReply(bool force, bool requestReply, bool reportApplyTimestamp)
 {
 	static XLogRecPtr writePtr = 0;
 	static XLogRecPtr flushPtr = 0;
 	XLogRecPtr	applyPtr;
 	static TimestampTz sendTime = 0;
 	TimestampTz now;
+	TimestampTz applyTimestamp = 0;
 
 	/*
 	 * If the user doesn't want status to be reported to the master, be sure
@@ -1102,10 +1129,8 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	 * We can compare the write and flush positions to the last message we
 	 * sent without taking any lock, but the apply position requires a spin
 	 * lock, so we don't check that unless something else has changed or 10
-	 * seconds have passed.  This means that the apply log position will
-	 * appear, from the master's point of view, to lag slightly, but since
-	 * this is only for reporting purposes and only on idle systems, that's
-	 * probably OK.
+	 * seconds have passed, or the force flag has been set (which happens when
+	 * apply feedback has been requested by the primary).
 	 */
 	if (!force
 		&& writePtr == LogstreamResult.Write
@@ -1118,7 +1143,10 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	/* Construct a new message */
 	writePtr = LogstreamResult.Write;
 	flushPtr = LogstreamResult.Flush;
-	applyPtr = GetXLogReplayRecPtr(NULL);
+	if (reportApplyTimestamp)
+		applyTimestamp = GetXLogReplayTimestamp(&applyPtr);
+	else
+		applyPtr = GetXLogReplayRecPtr(NULL);
 
 	resetStringInfo(&reply_message);
 	pq_sendbyte(&reply_message, 'r');
@@ -1126,6 +1154,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	pq_sendint64(&reply_message, flushPtr);
 	pq_sendint64(&reply_message, applyPtr);
 	pq_sendint64(&reply_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&reply_message, TimestampTzToIntegerTimestamp(applyTimestamp));
 	pq_sendbyte(&reply_message, requestReply ? 1 : 0);
 
 	/* Send it */
@@ -1230,8 +1259,8 @@ static void
 ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 {
 	WalRcvData *walrcv = WalRcv;
-
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
+	static TimestampTz lastRecordedTimestamp = 0;
 
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
@@ -1242,6 +1271,18 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
 	SpinLockRelease(&walrcv->mutex);
 
+	/*
+	 * Remember primary's timestamp at this WAL location.  We throw away
+	 * samples if they are coming too fast because we don't want to fill up
+	 * the finite circular buffer and have to throw away older samples.
+	 */
+	if (lastRecordedTimestamp < TimestampTzPlusMilliseconds(sendTime,
+															-MIN_TIME_BETWEEN_TIMESTAMPED_LSNS))
+	{
+		SetXLogReplayTimestampAtLsn(sendTime, walEnd);
+		lastRecordedTimestamp = sendTime;
+	}
+
 	if (log_min_messages <= DEBUG2)
 	{
 		char	   *sendtime;
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index c03e045..a3a2097 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1543,15 +1543,29 @@ ProcessStandbyReplyMessage(void)
 	XLogRecPtr	writePtr,
 				flushPtr,
 				applyPtr;
+	int64		applyLagUs;
 	bool		replyRequested;
+	TimestampTz now = GetCurrentTimestamp();
+	TimestampTz applyTimestamp;
 
 	/* the caller already consumed the msgtype byte */
 	writePtr = pq_getmsgint64(&reply_message);
 	flushPtr = pq_getmsgint64(&reply_message);
 	applyPtr = pq_getmsgint64(&reply_message);
 	(void) pq_getmsgint64(&reply_message);		/* sendTime; not used ATM */
+	applyTimestamp = IntegerTimestampToTimestampTz(pq_getmsgint64(&reply_message));
 	replyRequested = pq_getmsgbyte(&reply_message);
 
+	/* Compute the apply lag in milliseconds. */
+	if (applyTimestamp == 0)
+		applyLagUs = -1;
+	else
+#ifdef HAVE_INT64_TIMESTAMP
+		applyLagUs = now - applyTimestamp;
+#else
+		applyLagUs = (now - applyTimestamp) * 1000000;
+#endif
+
 	elog(DEBUG2, "write %X/%X flush %X/%X apply %X/%X%s",
 		 (uint32) (writePtr >> 32), (uint32) writePtr,
 		 (uint32) (flushPtr >> 32), (uint32) flushPtr,
@@ -1573,6 +1587,8 @@ ProcessStandbyReplyMessage(void)
 		walsnd->write = writePtr;
 		walsnd->flush = flushPtr;
 		walsnd->apply = applyPtr;
+		if (applyLagUs >= 0)
+			walsnd->applyLagUs = applyLagUs;
 		SpinLockRelease(&walsnd->mutex);
 	}
 
@@ -2743,7 +2759,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	8
+#define PG_STAT_GET_WAL_SENDERS_COLS	9
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2791,6 +2807,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		XLogRecPtr	write;
 		XLogRecPtr	flush;
 		XLogRecPtr	apply;
+		int64		applyLagUs;
 		int			priority;
 		WalSndState state;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
@@ -2805,6 +2822,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
+		applyLagUs = walsnd->applyLagUs;
 		priority = walsnd->sync_standby_priority;
 		SpinLockRelease(&walsnd->mutex);
 
@@ -2839,6 +2857,23 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				nulls[5] = true;
 			values[5] = LSNGetDatum(apply);
 
+			if (applyLagUs < 0)
+				nulls[6] = true;
+			else
+			{
+				Interval *applyLagInterval = palloc(sizeof(Interval));
+
+				applyLagInterval->month = 0;
+				applyLagInterval->day = 0;
+#ifdef HAVE_INT64_TIMESTAMP
+				applyLagInterval->time = applyLagUs;
+#else
+				applyLagInterval->time = applyLagUs / 1000000.0;
+#endif
+				nulls[6] = false;
+				values[6] = IntervalPGetDatum(applyLagInterval);
+			}
+
 			/*
 			 * Treat a standby such as a pg_basebackup background process
 			 * which always returns an invalid flush location, as an
@@ -2846,18 +2881,18 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 			 */
 			priority = XLogRecPtrIsInvalid(walsnd->flush) ? 0 : priority;
 
-			values[6] = Int32GetDatum(priority);
+			values[7] = Int32GetDatum(priority);
 
 			/*
 			 * More easily understood version of standby state. This is purely
 			 * informational, not different from priority.
 			 */
 			if (priority == 0)
-				values[7] = CStringGetTextDatum("async");
+				values[8] = CStringGetTextDatum("async");
 			else if (walsnd == sync_standby)
-				values[7] = CStringGetTextDatum("sync");
+				values[8] = CStringGetTextDatum("sync");
 			else
-				values[7] = CStringGetTextDatum("potential");
+				values[8] = CStringGetTextDatum("potential");
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
diff --git a/src/backend/utils/adt/timestamp.c b/src/backend/utils/adt/timestamp.c
index c4f556a..2032f61 100644
--- a/src/backend/utils/adt/timestamp.c
+++ b/src/backend/utils/adt/timestamp.c
@@ -1611,6 +1611,20 @@ IntegerTimestampToTimestampTz(int64 timestamp)
 #endif
 
 /*
+ * TimestampTzToIntegerTimestamp -- convert a native timestamp to int64 format
+ *
+ * When compiled with --enable-integer-datetimes, this is implemented as a
+ * no-op macro.
+ */
+#ifndef HAVE_INT64_TIMESTAMP
+int64
+TimestampTzToIntegerTimestamp(TimestampTz timestamp)
+{
+	return timestamp * 1000000;
+}
+#endif
+
+/*
  * TimestampDifference -- convert the difference between two timestamps
  *		into integer seconds and microseconds
  *
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 68e20e4..efb9719 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -236,6 +236,9 @@ extern void GetXLogReceiptTime(TimestampTz *rtime, bool *fromStream);
 extern XLogRecPtr GetXLogReplayRecPtr(TimeLineID *replayTLI);
 extern XLogRecPtr GetXLogInsertRecPtr(void);
 extern XLogRecPtr GetXLogWriteRecPtr(void);
+extern void SetXLogReplayTimestamp(TimestampTz timestamp);
+extern void SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn);
+extern TimestampTz GetXLogReplayTimestamp(XLogRecPtr *lsn);
 extern bool RecoveryIsPaused(void);
 extern void SetRecoveryPause(bool recoveryPause);
 extern TimestampTz GetLatestXTime(void);
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index a0f821a..811f10c 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2712,7 +2712,7 @@ DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f
 DESCR("statistics: information about currently active backends");
 DATA(insert OID = 3318 (  pg_stat_get_progress_info           PGNSP PGUID 12 1 100 0 0 f f f f f t s r 1 0 2249 "25" "{25,23,26,26,20,20,20,20,20,20,20,20,20,20}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{cmdtype,pid,datid,relid,param1,param2,param3,param4,param5,param6,param7,param8,param9,param10}" _null_ _null_ pg_stat_get_progress_info _null_ _null_ _null_ ));
 DESCR("statistics: information about progress of backends running maintenance command");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,23,25}" "{o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25}" "{o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 7794aa5..4de43e8 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -46,6 +46,7 @@ typedef struct WalSnd
 	XLogRecPtr	write;
 	XLogRecPtr	flush;
 	XLogRecPtr	apply;
+	int64		applyLagUs;
 
 	/* Protects shared variables shown above. */
 	slock_t		mutex;
diff --git a/src/include/utils/timestamp.h b/src/include/utils/timestamp.h
index fbead3a..297e151 100644
--- a/src/include/utils/timestamp.h
+++ b/src/include/utils/timestamp.h
@@ -227,9 +227,11 @@ extern bool TimestampDifferenceExceeds(TimestampTz start_time,
 #ifndef HAVE_INT64_TIMESTAMP
 extern int64 GetCurrentIntegerTimestamp(void);
 extern TimestampTz IntegerTimestampToTimestampTz(int64 timestamp);
+extern int64 TimestampTzToIntegerTimestamp(TimestampTz timestamp);
 #else
 #define GetCurrentIntegerTimestamp()	GetCurrentTimestamp()
 #define IntegerTimestampToTimestampTz(timestamp) (timestamp)
+#define TimestampTzToIntegerTimestamp(timestamp) (timestamp)
 #endif
 
 extern TimestampTz time_t_to_timestamptz(pg_time_t tm);
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 81bc5c9..6f80548 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1760,11 +1760,12 @@ pg_stat_replication| SELECT s.pid,
     w.write_location,
     w.flush_location,
     w.replay_location,
+    w.replay_lag,
     w.sync_priority,
     w.sync_state
    FROM pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, waiting, xact_start, query_start, backend_start, state_change, client_addr, client_hostname, client_port, backend_xid, backend_xmin, ssl, sslversion, sslcipher, sslbits, sslcompression, sslclientdn),
     pg_authid u,
-    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, sync_priority, sync_state)
+    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state)
   WHERE ((s.usesysid = u.oid) AND (s.pid = w.pid));
 pg_stat_ssl| SELECT s.pid,
     s.ssl,
0003-refactor-syncrep-exit-v3.patchapplication/octet-stream; name=0003-refactor-syncrep-exit-v3.patchDownload
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 4565348..bbc28a7 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -83,6 +83,64 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * ===========================================================
  */
 
+static bool
+SyncRepCheckEarlyExit(void)
+{
+	/*
+	 * If a wait for synchronous replication is pending, we can neither
+	 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
+	 * lead the client to believe that the transaction aborted, which
+	 * is not true: it's already committed locally. The former is no good
+	 * either: the client has requested synchronous replication, and is
+	 * entitled to assume that an acknowledged commit is also replicated,
+	 * which might not be true. So in this case we issue a WARNING (which
+	 * some clients may be able to interpret) and shut off further output.
+	 * We do NOT reset ProcDiePending, so that the process will die after
+	 * the commit is cleaned up.
+	 */
+	if (ProcDiePending)
+	{
+		ereport(WARNING,
+				(errcode(ERRCODE_ADMIN_SHUTDOWN),
+				 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * It's unclear what to do if a query cancel interrupt arrives.  We
+	 * can't actually abort at this point, but ignoring the interrupt
+	 * altogether is not helpful, so we just terminate the wait with a
+	 * suitable warning.
+	 */
+	if (QueryCancelPending)
+	{
+		QueryCancelPending = false;
+		ereport(WARNING,
+				(errmsg("canceling wait for synchronous replication due to user request"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * If the postmaster dies, we'll probably never get an
+	 * acknowledgement, because all the wal sender processes will exit. So
+	 * just bail out.
+	 */
+	if (!PostmasterIsAlive())
+	{
+		ProcDiePending = true;
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	return false;
+}
+
 /*
  * Wait for synchronous replication, if requested by user.
  *
@@ -180,57 +238,9 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 		if (syncRepState == SYNC_REP_WAIT_COMPLETE)
 			break;
 
-		/*
-		 * If a wait for synchronous replication is pending, we can neither
-		 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
-		 * lead the client to believe that the transaction aborted, which
-		 * is not true: it's already committed locally. The former is no good
-		 * either: the client has requested synchronous replication, and is
-		 * entitled to assume that an acknowledged commit is also replicated,
-		 * which might not be true. So in this case we issue a WARNING (which
-		 * some clients may be able to interpret) and shut off further output.
-		 * We do NOT reset ProcDiePending, so that the process will die after
-		 * the commit is cleaned up.
-		 */
-		if (ProcDiePending)
-		{
-			ereport(WARNING,
-					(errcode(ERRCODE_ADMIN_SHUTDOWN),
-					 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit())
 			break;
-		}
-
-		/*
-		 * It's unclear what to do if a query cancel interrupt arrives.  We
-		 * can't actually abort at this point, but ignoring the interrupt
-		 * altogether is not helpful, so we just terminate the wait with a
-		 * suitable warning.
-		 */
-		if (QueryCancelPending)
-		{
-			QueryCancelPending = false;
-			ereport(WARNING,
-					(errmsg("canceling wait for synchronous replication due to user request"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * If the postmaster dies, we'll probably never get an
-		 * acknowledgement, because all the wal sender processes will exit. So
-		 * just bail out.
-		 */
-		if (!PostmasterIsAlive())
-		{
-			ProcDiePending = true;
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
-			break;
-		}
 
 		/*
 		 * Wait on latch.  Any condition that should wake us up will set the
0004-causal-reads-v3.patchapplication/octet-stream; name=0004-causal-reads-v3.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index b21b4c0..de24f1c 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -2668,6 +2668,35 @@ include_dir 'conf.d'
      across the cluster without problems if that is required.
     </para>
 
+    <sect2 id="runtime-config-replication-all">
+     <title>All Servers</title>
+     <para>
+      These parameters can be set on the primary or any standby.
+     </para>
+     <variablelist>
+      <varlistentry id="guc-causal-reads" xreflabel="causal_reads">
+       <term><varname>causal_reads</varname> (<type>boolean</type>)
+       <indexterm>
+        <primary><varname>causal_reads</> configuration parameter</primary>
+       </indexterm>
+       </term>
+       <listitem>
+        <para>
+         Enables causal consistency between transactions run on different
+         servers.  A transaction that is run on a standby
+         with <varname>causal_reads</> set to <literal>on</> is guaranteed
+         either to see the effects of all completed transactions run on the
+         primary with the setting on, or to receive an error "standby is not
+         available for causal reads".  Note that both transactions involved in
+         a causal dependency (a write on the primary followed by a read on any
+         server which must see the write) must be run with the setting on.
+         See <xref linkend="causal-reads"> for more details.
+        </para>
+       </listitem>
+      </varlistentry>
+     </variablelist>     
+    </sect2>
+
     <sect2 id="runtime-config-replication-sender">
      <title>Sending Server(s)</title>
 
@@ -2899,6 +2928,48 @@ include_dir 'conf.d'
       </listitem>
      </varlistentry>
 
+     <varlistentry>
+      <term><varname>causal_reads_timeout</varname> (<type>integer</type>)
+       <indexterm>
+        <primary><varname>causal_reads_timeout</> configuration parameter</primary>
+       </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies the maximum replay lag the primary will tolerate from a
+        standby before dropping it from the set of standbys available for
+        causal reads.
+       </para>
+       <para>
+        This setting is also used to control the <firstterm>leases</> used to
+        maintain the causal reads guarantee.  It must be set to a value which
+        is at least 4 times the maximum possible difference in system clocks
+        between the primary and standby servers, as described
+        in <xref linkend="causal-reads">.
+       </para>
+      </listitem>
+     </varlistentry>
+
+     <varlistentry id="guc-causal-reads-standby-names" xreflabel="causal-reads-standby-names">
+      <term><varname>causal_reads_standby_names</varname> (<type>string</type>)
+      <indexterm>
+       <primary><varname>causal_reads_standby_names</> configuration parameter</primary>
+      </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies a comma-separated list of standby names that can support
+        <firstterm>causal reads</>, as described in
+        <xref linkend="causal-reads">.  Follows the same convention
+        as <link linkend="guc-synchronous-standby-names"><literal>synchronous_standby_name</></>.
+        The default is <literal>*</>, matching all standbys.
+       </para>
+       <para>
+        This setting has no effect if <varname>causal_reads_timeout</> is not set.
+       </para>
+      </listitem>
+     </varlistentry>
+
      </variablelist>
     </sect2>
 
diff --git a/doc/src/sgml/high-availability.sgml b/doc/src/sgml/high-availability.sgml
index 2600fba..a01df0d 100644
--- a/doc/src/sgml/high-availability.sgml
+++ b/doc/src/sgml/high-availability.sgml
@@ -1115,6 +1115,9 @@ primary_slot_name = 'node_a_slot'
     that it has replayed the transaction, making it visible to user queries.
     In simple cases, this allows for load balancing with causal consistency
     on a single hot standby.
+    (See also
+    <xref linkend="causal-reads"> which deals with multiple standbys and
+    standby failure.)
    </para>
 
    <para>
@@ -1233,6 +1236,119 @@ primary_slot_name = 'node_a_slot'
    </sect3>
   </sect2>
 
+  <sect2 id="causal-reads">
+   <title>Causal reads</title>
+   <indexterm>
+    <primary>causal reads</primary>
+    <secondary>in standby</secondary>
+   </indexterm>
+
+   <para>
+    The causal reads feature allows read-only queries to run on hot standby
+    servers without exposing stale data to the client, providing a form of
+    causal consistency.  Transactions can run on any standby with the
+    following guarantee about the visibility of preceding transactions: If you
+    set <varname>causal_reads</> to <literal>on</> in any pair of consecutive
+    transactions tx1, tx2 where tx2 begins after tx1 successfully returns,
+    then tx2 will either see tx1 or fail with a new error "standby is not
+    available for causal reads", no matter which server it runs on.  Although
+    the guarantee is expressed in terms of two individual transactions, the
+    GUC can also be set at session, role or system level to make the guarantee
+    generally, allowing for load balancing of applications that were not
+    designed with load balancing in mind.
+   </para>
+
+   <para>
+    In order to enable the feature, <varname>causal_reads_timeout</> must be
+    set to a non-zero value on the primary server.  The
+    GUC <varname>causal_reads_standby_names</> can be used to limit the set of
+    standbys that can join the dynamic set of causal reads standbys by
+    providing a comma-separated list of application names.  By default, all
+    standbys are candidates, if the feature is enabled.
+   </para>
+
+   <para>
+    The current set of servers that the primary considers to be available for
+    causal reads can be seen in
+    the <link linkend="monitoring-stats-views-table"> <literal>pg_stat_replication</></>
+    view.  Administrators, applications and load balancing middleware can use
+    this view to discover standbys that can currently handle causal reads
+    transactions without raising the error.  Since that information is only an
+    instantantaneous snapshot, clients should still be prepared for the error
+    to be raised at any time, and consider redirecting transactions to another
+    standby.
+   </para>
+
+   <para>
+    The advantages of the causal reads feature over simply
+    setting <varname>synchronous_commit</> to <literal>remote_apply</> are:
+    <orderedlist>
+      <listitem>
+       <para>
+        It allows the primary to wait for multiple standbys to replay
+        transactions.
+       </para>
+      </listitem>
+      <listitem>
+       <para>
+        It places a configurable limit on how much replay lag (and therefore
+        delay at commit time) the primary tolerates from standbys before it
+        drops them from the dynamic set of standbys it waits for.
+       </para>   
+      </listitem>
+      <listitem>
+       <para>
+        It upholds the causal reads guarantee during the transitions that
+        occur when new standbys are added or removed from the set of standbys,
+        including scenarios where contact has been lost between the primary
+        and standbys but the standby is still alive and running client
+        queries.
+       </para>
+      </listitem>
+    </orderedlist>
+   </para>
+
+   <para>
+    The protocol used to uphold the guarantee even in the case of network
+    failure depends on the system clocks of the primary and standby servers
+    being synchronized, with an allowance for a difference up to one quarter
+    of <varname>causal_reads_timeout</>.  For example,
+    if <varname>causal_reads_timeout</> is set to <literal>4s</>, then the
+    clocks must not be further than 1 second apart for the guarantee to be
+    upheld reliably during transitions.  The ubiquity of the Network Time
+    Protocol (NTP) on modern operating systems and availability of high
+    quality time servers makes it possible to choose a tolerance significantly
+    higher than the maximum expected clock difference.  An effort is
+    nevertheless made to detect and report misconfigured and faulty systems
+    with clock differences greater than the configured tolerance.
+   </para>
+
+   <note>
+    <para>
+     Current hardware clocks, NTP implementations and public time servers are
+     unlikely to allow the system clocks to differ more than tens or hundreds
+     of milliseconds, and systems synchronized with dedicated local time
+     servers may be considerably more accurate, but you should only consider
+     setting <varname>causal_reads_timeout</> below 4 seconds (allowing up to
+     1 second of clock difference) after researching your time synchronization
+     infrastructure thoroughly.
+    </para>  
+   </note>
+
+   <note>
+    <para>
+      While similar to synchronous replication in the sense that both involve
+      the primary server waiting for responses from standby servers, the
+      causal reads feature is not concerned with avoiding data loss.  A
+      primary configured for causal reads will drop all standbys that stop
+      responding or replay too slowly from the dynamic set that it waits for,
+      so you should consider configuring both synchronous replication and
+      causal reads if you need data loss avoidance guarantees and causal
+      consistency guarantees for load balancing.
+    </para>
+   </note>
+  </sect2>
+
   <sect2 id="continuous-archiving-in-standby">
    <title>Continuous archiving in standby</title>
 
@@ -1581,7 +1697,16 @@ if (!triggered)
     so there will be a measurable delay between primary and standby. Running the
     same query nearly simultaneously on both primary and standby might therefore
     return differing results. We say that data on the standby is
-    <firstterm>eventually consistent</firstterm> with the primary.  Once the
+    <firstterm>eventually consistent</firstterm> with the primary by default.
+    The data visible to a transaction running on a standby can be
+    made <firstterm>causally consistent</> with respect to a transaction that
+    has completed on the primary by setting <varname>causal_reads</>
+    to <literal>on</> in both transactions.  For more details,
+    see <xref linkend="causal-reads">.
+   </para>
+
+   <para>
+    Once the    
     commit record for a transaction is replayed on the standby, the changes
     made by that transaction will be visible to any new snapshots taken on
     the standby.  Snapshots may be taken at the start of each query or at the
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 818b952..5a87f37 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -836,6 +836,17 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
      <entry><type>text</></entry>
      <entry>Synchronous state of this standby server</entry>
     </row>
+    <row>
+     <entry><structfield>causal_reads_state</></entry>
+     <entry><type>text</></entry>
+     <entry>Causal reads state of this standby server.  This field will be
+     non-null only if <varname>cause_reads_timeout</> is set.  If a standby is
+     in <literal>available</> state, then it can currently serve causal reads
+     queries.  If it is not replaying fast enough or not responding to
+     keepalive messages, it will be in <literal>unavailable</> state, and if
+     it is currently transitioning to availability it will be
+     in <literal>joining</> state for a short time.</entry>
+    </row>
    </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 1330a45..9ee6549 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -2103,11 +2103,12 @@ RecordTransactionCommitPrepared(TransactionId xid,
 	END_CRIT_SECTION();
 
 	/*
-	 * Wait for synchronous replication, if required.
+	 * Wait for causal reads and synchronous replication, if required.
 	 *
 	 * Note that at this stage we have marked clog, but still show as running
 	 * in the procarray and continue to hold locks.
 	 */
+	CausalReadsWaitForLSN(recptr);
 	SyncRepWaitForLSN(recptr);
 }
 
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 939d71c..0497ff2 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,10 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
+	{
+		CausalReadsWaitForLSN(XactLastRecEnd);
 		SyncRepWaitForLSN(XactLastRecEnd);
+	}
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5124,7 +5127,7 @@ XactLogCommitRecord(TimestampTz commit_time,
 	 * Check if the caller would like to ask standbys for immediate feedback
 	 * once this commit is applied.
 	*/
-	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY || causal_reads)
 		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
 
 	/*
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index f5a7c5c..1fb9812 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -663,7 +663,8 @@ CREATE VIEW pg_stat_replication AS
             W.replay_location,
 	    W.replay_lag,
             W.sync_priority,
-            W.sync_state
+            W.sync_state,
+	    W.causal_reads_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
             pg_stat_get_wal_senders() AS W
     WHERE S.usesysid = U.oid AND
diff --git a/src/backend/replication/README.causal_reads b/src/backend/replication/README.causal_reads
new file mode 100644
index 0000000..1fddd62
--- /dev/null
+++ b/src/backend/replication/README.causal_reads
@@ -0,0 +1,193 @@
+The causal reads guarantee says: If you run any two consecutive
+transactions tx1, tx2 where tx1 completes before tx2 begins, with
+causal_reads set to "on" in both transactions, tx2 will see tx1 or
+raise an error to complain that it can't guarantee causal consistency,
+no matter which servers (primary or any standby) you run each
+transaction on.
+
+When both transactions run on the primary, the guarantee is trivially
+upheld.
+
+To deal with read-only physical streaming standbys, the primary keeps
+track of a set of standbys that it considers to be currently
+"available" for causal reads, and sends a stream of "leases" to those
+standbys granting them the right to handle causal reads transactions
+for a short time without any further communication with the primary.
+
+In general, the primary provides the guarantee by waiting for all of
+the "available" standbys to report that they have applied a
+transaction.  However, the set of available standbys is dynamic, and
+things get more complicated during state transitions.  There are two
+types of transitions to consider:
+
+1.  unavailable->joining->available
+
+Standbys start out as "unavailable".  If a standby is unavailable and
+is applying fast enough and matches causal_reads_standby_names, the
+primary transitions it to "available", but first it sets it to
+"joining" until it is sure that any transaction committed while it was
+unavailable has definitely been applied on the standby.  This closes a
+race that would otherwise exist if we moved directly to available
+state: tx1 might not wait for a given standby because it's
+unavailable, then a lease might be granted, and then tx2 might run a
+causal reads transaction without error but see stale data.  The
+joining state acts as an airlock: while in joining state, the primary
+waits for that standby to replay causal reads transactions in
+anticipation of the move to available, but it doesn't progress to
+available state and grant a lease to the standby until everything
+preceding joining state has also been applied.
+
+2.  available->unavailable
+
+If a standby is not applying fast enough or not responding to
+keepalive messages, then the primary kicks that standby out of the
+dynamic set of available standbys, that is, marks it as "unavailable".
+In order to make sure that the standby has started rejecting causal
+reads transactions, it needs to revoke the lease it most recently
+granted.  It does that by waiting for the lease to expire before
+allowing any causal reads commits to return.  (In future there could
+be a fast-path revocation message which waits for a serial-numbered
+acknowledgement to reduce waiting in the case where the standby is
+lagging but still reachable and responding).
+
+The rest of this document illustrates how clock skew affects the
+available->unavailable transition.
+
+The following 4 variables are derived from a single GUC, and these
+values will be used in the following illustrations:
+
+causal_reads_timeout = 4s
+lease_time           = 4s (= causal_reads_timeout)
+keepalive_time       = 2s (= lease_time / 2)
+max_clock_skew       = 1s (= lease_time / 4)
+
+Every keepalive_time, the primary transmits a lease that expires at
+local_clock_time + lease_time - max_clock_skew, shown in the following
+diagram as 't' for transmission time and '|' for expiry time.  If
+contact is lost with a standby, the primary will wait until sent_time
++ lease_time for the most recently granted lease to expire, shown on
+the following diagram 'x', to be sure that the standby's clock has
+reached the expiry time even if its clock differs by up to
+max_clock_skew.  In other words, the primary tells the standby that
+the expiry time is at one time, but it trusts that the standby will
+surely agree if it gives it some extra time.  The extra time is
+max_clock_skew.  If the clocks differ by more than max_clock_skew, all
+bets are off (but see below for attempt to detect obvious cases).
+
+0     1     2     3     4     5     6     7     8     9
+t-----------------|-----x
+            t-----------------|-----x
+                        t-----------------|-----x
+                                    t-----------------|...
+                                                t------...
+
+A standby whose clock is 2 seconds ahead of the primary's clock
+perceives gaps in the stream of leases, and will reject causal_reads
+transactions in those intervals.  The causal reads guarantee is
+upheld, but spurious errors are raised between leases, as a
+consequence of the clock skew being greater than max_clock_skew.  In
+the following diagram 'r' shows reception time, and the timeline along
+the top shows the standby's local clock time.
+
+2     3     4     5     6     7     8     9    10    11
+r-----|
+            r-----|
+                        r-----|
+                                    r-----|
+                                                r-----|
+
+If there were no network latency, a standby whose clock is exactly 1
+second ahead of the primary's clock would perceive the stream of
+leases as being replaced just in time, so there is no gap.  Since in
+reality the time of receipt is some time after the time of
+transmission due to network latency, if the standby's clock is exactly
+1 second behind, then there will be small network-latency-sized gaps
+before the next lease arrives, but still no correctness problem with
+respect to the causal reads guarantee.
+
+1     2     3     4     5     6     7     8     9    10
+r-----------|
+            r-----------|
+                        r-----------|
+                                    r-----------|
+                                                r------...
+
+A standby whose clock is perfectly in sync with the primary's
+perceives the stream of leases overlapping (this matches the primary's
+perception of the leases it sent):
+
+0     1     2     3     4     5     6     7     8     9
+r-----------------|
+            r-----------------|
+                        r-----------------|
+                                    r-----------------|
+                                                r------...
+
+A standby whose clock is exactly 1 second behind the primary's
+perceives the stream of leases as overlapping even more, but the time
+of expiry as judged by the standby is no later than the time the
+primary will wait for if required ('x').  That is, if contact is lost
+with the standby, the primary can still reliably hold up causal reads
+commits until the standby has started raising the error in
+causal_reads transactions.
+
+-1    0     1     2     3     4     5     6     7     8
+r-----------------------|
+            r-----------------------|
+                        r-----------------------|
+                                    r------------------...
+                                                r------...
+
+
+A standby whose clock is 2 seconds behind the primary's would perceive
+the stream of leases overlapping even more, and the primary would no
+longer be able to wait for a lease to expire if it wanted to revoke
+it.  But because the expiry time is after local_clock_time +
+lease_time, the standby can immediately see that its own clock must be
+more than 1 second behind the primary's, so it ignores the lease and
+logs a clock skew warning.  In the following diagram a lease expiry
+time that is obviously generated by a primary with a clock set too far
+in the future compared to the local clock is shown with a '!'.
+
+-2    -1    0     1     2     3     4     5     6     7
+r-----------------------------!
+            r-----------------------------!
+                        r-----------------------------!
+                                    r------------------...
+                                                r------...
+
+A danger window exists when the standby's clock is more than
+max_clock_skew behind the primary's clock, but not more than
+max_clock_skew + network latency time behind.  If the clock difference
+is in that range, then the algorithm presented above which is based on
+time of receipt cannot detect that the local clock is too far behind.
+The consequence of this problem could be as follows:
+
+1.  The standby loses contact with the primary due to a network fault.
+
+2.  The primary decides to drop the standby from the set of available
+    causal reads standbys due to lack of keepalive responses or
+    excessive lag, which necessitates holding up commits of causal
+    reads transactions until the most recently sent lease expires, in
+    the belief that the standby will definitely have started raising
+    the 'causal reads unavailable' error in causal reads transactions
+    by that time, if it is still alive and servicing requests.
+
+3.  The standby still has clients connected and running queries.
+
+4.  Due to clock skew in the problematic range, in the standby's
+    opinion the lease lasts slightly longer than the primary waits.
+
+5.  For a short window at most the duration of the network latency
+    time, clients running causal reads transactions are allowed to see
+    potentially stale data.
+
+For this reason we say that the causal reads guarantee only holds as
+long as the absolute difference between the system clocks of the
+machines is no more than max_clock_skew.  The theory is that NTP makes
+it possible to reason about the maximum possible clock difference
+between machines and choose a value that allows for a much larger
+difference.  However, we do make a best effort attempt to detect
+wildly divergent systems as described above, to catch the case of
+servers not running a correctly configured ntp daemon, or with a clock
+so far out of whack that ntp refuses to fix it.
\ No newline at end of file
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index bbc28a7..80502f3 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -57,6 +57,11 @@
 #include "utils/builtins.h"
 #include "utils/ps_status.h"
 
+/* GUC variables */
+int causal_reads_timeout;
+bool causal_reads;
+char *causal_reads_standby_names;
+
 /* User-settable parameters for sync rep */
 char	   *SyncRepStandbyNames;
 
@@ -69,7 +74,7 @@ static int	SyncRepWaitMode = SYNC_REP_NO_WAIT;
 
 static void SyncRepQueueInsert(int mode);
 static void SyncRepCancelWait(void);
-static int	SyncRepWakeQueue(bool all, int mode);
+static int	SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn);
 
 static int	SyncRepGetStandbyPriority(void);
 
@@ -142,6 +147,198 @@ SyncRepCheckEarlyExit(void)
 }
 
 /*
+ * Check if we can stop waiting for causal consistency.  We can stop waiting
+ * when the following conditions are met:
+ *
+ * 1.  All walsenders currently in 'joining' or 'available' state have
+ * applied the target LSN.
+ *
+ * 2.  Any stall periods caused by standbys dropping out of 'available' state
+ * have passed, so that we can be sure that their leases have expired and they
+ * have started rejecting causal reads transactions.
+ *
+ * The output parameter 'waitingFor' is set to the number of nodes we are
+ * currently waiting for.  The output parameters 'stallTimeMillis' is set to
+ * the number of milliseconds we need to wait for to observe any current
+ * commit stall.
+ *
+ * Returns true if commit can return control, because every standby has either
+ * applied the LSN or started rejecting causal_reads transactions.
+ */
+static bool
+CausalReadsCommitCanReturn(XLogRecPtr XactCommitLSN,
+						   int *waitingFor,
+						   long *stallTimeMillis)
+{
+	int i;
+	TimestampTz now;
+
+	/* Count how many joining/available nodes we are waiting for. */
+	*waitingFor = 0;
+	for (i = 0; i < max_wal_senders; ++i)
+	{
+		WalSnd *walsnd = &WalSndCtl->walsnds[i];
+
+		/*
+		 * Assuming atomic read of pid_t, we can check walsnd->pid without
+		 * acquiring the spinlock to avoid memory synchronization costs for
+		 * unused walsender slots.  We see a value that existed sometime at
+		 * least as recently as the last memory barrier.
+		 */
+		if (walsnd->pid != 0)
+		{
+			/*
+			 * We need to hold the spinlock to read LSNs, because we can't be
+			 * sure they can be read atomically.
+			 */
+			SpinLockAcquire(&walsnd->mutex);
+			if (walsnd->pid != 0 && walsnd->causal_reads_state >= WALSNDCRSTATE_JOINING)
+			{
+				if (walsnd->apply < XactCommitLSN)
+					++*waitingFor;
+			}
+			SpinLockRelease(&walsnd->mutex);
+		}
+	}
+
+	/* Check if there is a stall in progress that we need to observe. */
+	now = GetCurrentTimestamp();
+	LWLockAcquire(SyncRepLock, LW_SHARED);
+	if (WalSndCtl->stall_causal_reads_until > now)
+	{
+		long seconds;
+		int usecs;
+
+		/* Compute how long we have to wait, rounded up to nearest ms. */
+		TimestampDifference(now, WalSndCtl->stall_causal_reads_until,
+							&seconds, &usecs);
+		*stallTimeMillis = seconds * 1000 + (usecs + 999) / 1000;
+	}
+	else
+		*stallTimeMillis = 0;
+	LWLockRelease(SyncRepLock);
+
+	/* We are done if we are not waiting for any nodes or stalls. */
+	return *waitingFor == 0 && *stallTimeMillis == 0;
+}
+
+/*
+ * Wait for causal consistency in causal_reads mode, if requested by user.
+ */
+void
+CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN)
+{
+	long stallTimeMillis;
+	int waitingFor;
+	char *ps_display_buffer = NULL;
+
+	/* Leave if we aren't in causal_reads mode. */
+	if (!causal_reads)
+		return;
+
+	for (;;)
+	{
+		/* Reset latch before checking state. */
+		ResetLatch(MyLatch);
+
+		/*
+		 * Join the queue to be woken up if any causal reads joining/available
+		 * standby applies XactCommitLSN or the set of causal reads standbys
+		 * changes (if we aren't already in the queue).  We don't actually know
+		 * if we need to wait for any peers to reach the target LSN yet, but
+		 * we have to register just in case before checking the walsenders'
+		 * state to avoid a race condition that could occur if we did it after
+		 * calling CausalReadsCommitCanReturn.  (SyncRepWaitForLSN doesn't
+		 * have to do this because it can check the highest-seen LSN in
+		 * walsndctl->lsn[mode] which is protected by SyncRepLock, the same
+		 * lock as the queues.  We can't do that here, because there is no
+		 * single highest-seen LSN that is useful.  We must check
+		 * walsnd->apply for all relevant walsenders.  Therefore we must
+		 * register for notifications first, so that we can be notified via
+		 * our latch of any standby applying the LSN we're interested in after
+		 * we check but before we start waiting, or we could wait forever for
+		 * something that has already happened.)
+		 */
+		LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+		if (MyProc->syncRepState != SYNC_REP_WAITING)
+		{
+			MyProc->waitLSN = XactCommitLSN;
+			MyProc->syncRepState = SYNC_REP_WAITING;
+			SyncRepQueueInsert(SYNC_REP_WAIT_CAUSAL_READS);
+			Assert(SyncRepQueueIsOrderedByLSN(SYNC_REP_WAIT_CAUSAL_READS));
+		}
+		LWLockRelease(SyncRepLock);
+
+		/* Check if we're done. */
+		if (CausalReadsCommitCanReturn(XactCommitLSN, &waitingFor, &stallTimeMillis))
+		{
+			SyncRepCancelWait();
+			break;
+		}
+
+		Assert(waitingFor > 0 || stallTimeMillis > 0);
+
+		/* If we aren't actually waiting for any standbys, leave the queue. */
+		if (waitingFor == 0)
+			SyncRepCancelWait();
+
+		/* Update the ps title. */
+		if (update_process_title)
+		{
+			char buffer[80];
+
+			/* Remember the old value if this is our first update. */
+			if (ps_display_buffer == NULL)
+			{
+				int len;
+				const char *ps_display = get_ps_display(&len);
+
+				ps_display_buffer = palloc(len + 1);
+				memcpy(ps_display_buffer, ps_display, len);
+				ps_display_buffer[len] = '\0';
+			}
+
+			snprintf(buffer, sizeof(buffer),
+					 "waiting for %d peer(s) to apply %X/%X%s",
+					 waitingFor,
+					 (uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN,
+					 stallTimeMillis > 0 ? " (stalling)" : "");
+			set_ps_display(buffer, false);
+		}
+
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit()) /* Calls SyncRepCancelWait() if true. */
+			break;
+
+		/*
+		 * If are still waiting for peers, then we wait for any joining or
+		 * available peer to reach the LSN (or possibly stop being in one of
+		 * those states or go away).
+		 *
+		 * If not, there must be a non-zero stall time, so we wait for that to
+		 * elapse.
+		 */
+		if (waitingFor > 0)
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH, -1);
+		else
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH | WL_TIMEOUT,
+					  stallTimeMillis);
+	}
+
+	/* There is no way out of the loop that could leave us in the queue. */
+	Assert(SHMQueueIsDetached(&(MyProc->syncRepLinks)));
+	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
+	MyProc->waitLSN = 0;
+
+	/* Restore the ps display. */
+	if (ps_display_buffer != NULL)
+	{
+		set_ps_display(ps_display_buffer, false);
+		pfree(ps_display_buffer);
+	}
+}
+
+/*
  * Wait for synchronous replication, if requested by user.
  *
  * Initially backends start in state SYNC_REP_NOT_WAITING and then
@@ -413,6 +610,53 @@ SyncRepGetSynchronousStandby(void)
 }
 
 /*
+ * Check if the current WALSender process's application_name matches a name in
+ * causal_reads_standby_names (including '*' for wildcard).
+ */
+bool
+CausalReadsPotentialStandby(void)
+{
+	char *rawstring;
+	List	   *elemlist;
+	ListCell   *l;
+	bool		found = false;
+
+	/* If the feature is disable, then no. */
+	if (causal_reads_timeout == 0)
+		return false;
+
+	/* Need a modifiable copy of string */
+	rawstring = pstrdup(causal_reads_standby_names);
+
+	/* Parse string into list of identifiers */
+	if (!SplitIdentifierString(rawstring, ',', &elemlist))
+	{
+		/* syntax error in list */
+		pfree(rawstring);
+		list_free(elemlist);
+		/* GUC machinery will have already complained - no need to do again */
+		return 0;
+	}
+
+	foreach(l, elemlist)
+	{
+		char	   *standby_name = (char *) lfirst(l);
+
+		if (pg_strcasecmp(standby_name, application_name) == 0 ||
+			pg_strcasecmp(standby_name, "*") == 0)
+		{
+			found = true;
+			break;
+		}
+	}
+
+	pfree(rawstring);
+	list_free(elemlist);
+
+	return found;
+}
+
+/*
  * Update the LSNs on each queue based upon our latest state. This
  * implements a simple policy of first-valid-standby-releases-waiter.
  *
@@ -420,23 +664,27 @@ SyncRepGetSynchronousStandby(void)
  * perhaps also which information we store as well.
  */
 void
-SyncRepReleaseWaiters(void)
+SyncRepReleaseWaiters(bool walsender_cr_available_or_joining)
 {
 	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
 	int			numapply = 0;
+	int			numcausalreadsapply = 0;
+	bool		is_highest_priority_sync_standby;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
-	 * potential sync standbys then we have nothing to do. If we are still
-	 * starting up, still running base backup or the current flush position
-	 * is still invalid, then leave quickly also.
+	 * potential sync standbys and not in a state that causal_reads waits for,
+	 * then we have nothing to do. If we are still starting up, still running
+	 * base backup or the current flush position is still invalid, then leave
+	 * quickly also.
 	 */
-	if (MyWalSnd->sync_standby_priority == 0 ||
-		MyWalSnd->state < WALSNDSTATE_STREAMING ||
-		XLogRecPtrIsInvalid(MyWalSnd->flush))
+	if (!walsender_cr_available_or_joining &&
+		(MyWalSnd->sync_standby_priority == 0 ||
+		 MyWalSnd->state < WALSNDSTATE_STREAMING ||
+		 XLogRecPtrIsInvalid(MyWalSnd->flush)))
 		return;
 
 	/*
@@ -446,13 +694,19 @@ SyncRepReleaseWaiters(void)
 	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
 	syncWalSnd = SyncRepGetSynchronousStandby();
 
-	/* We should have found ourselves at least */
-	Assert(syncWalSnd != NULL);
+	/*
+	 * If we aren't managing the highest priority standby then make a note of
+	 * that so we can announce a takeover in the log if we ever get that job.
+	 */
+	is_highest_priority_sync_standby = syncWalSnd == MyWalSnd;
+	if (!is_highest_priority_sync_standby)
+		announce_next_takeover = true;
 
 	/*
-	 * If we aren't managing the highest priority standby then just leave.
+	 * If we aren't managing the highest priority standby or a standby in
+	 * causal reads 'joining' or 'available' state, then just leave.
 	 */
-	if (syncWalSnd != MyWalSnd)
+	if (!is_highest_priority_sync_standby && !walsender_cr_available_or_joining)
 	{
 		LWLockRelease(SyncRepLock);
 		announce_next_takeover = true;
@@ -461,24 +715,45 @@ SyncRepReleaseWaiters(void)
 
 	/*
 	 * Set the lsn first so that when we wake backends they will release up to
-	 * this location.
+	 * this location.  For the single-standby synchronous commit levels, we
+	 * only do this if we are the current synchronous standby and we are
+	 * advancing the LSN further than it has been advanced before, so that
+	 * SyncRepWaitForLSN can skip waiting in some cases.
 	 */
-	if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
-		numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
-		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+	if (is_highest_priority_sync_standby)
 	{
-		walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
-		numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY);
+		if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
+			numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE,
+										MyWalSnd->write);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
+			numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH,
+										MyWalSnd->flush);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+			numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY,
+										MyWalSnd->apply);
+		}
 	}
 
+	/*
+	 * For causal_reads, all walsenders currently in available or joining
+	 * state must reach the LSN on their own, and standbys will reach LSNs in
+	 * any order.  It doesn't make sense to keep the highest seen LSN in a
+	 * single walsndctl->lsn element.  (CausalReadsWaitForLSN has handling for
+	 * LSNs that have already been reached).
+	 */
+	if (walsender_cr_available_or_joining)
+		numcausalreadsapply =
+			SyncRepWakeQueue(false, SYNC_REP_WAIT_CAUSAL_READS,
+							 MyWalSnd->apply);
+
 	LWLockRelease(SyncRepLock);
 
 	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%x",
@@ -490,7 +765,7 @@ SyncRepReleaseWaiters(void)
 	 * If we are managing the highest priority standby, though we weren't
 	 * prior to this, then announce we are now the sync standby.
 	 */
-	if (announce_next_takeover)
+	if (is_highest_priority_sync_standby && announce_next_takeover)
 	{
 		announce_next_takeover = false;
 		ereport(LOG,
@@ -565,9 +840,8 @@ SyncRepGetStandbyPriority(void)
  * Must hold SyncRepLock.
  */
 static int
-SyncRepWakeQueue(bool all, int mode)
+SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn)
 {
-	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	PGPROC	   *proc = NULL;
 	PGPROC	   *thisproc = NULL;
 	int			numprocs = 0;
@@ -584,7 +858,7 @@ SyncRepWakeQueue(bool all, int mode)
 		/*
 		 * Assume the queue is ordered by LSN
 		 */
-		if (!all && walsndctl->lsn[mode] < proc->waitLSN)
+		if (!all && lsn < proc->waitLSN)
 			return numprocs;
 
 		/*
@@ -644,7 +918,7 @@ SyncRepUpdateSyncStandbysDefined(void)
 			int			i;
 
 			for (i = 0; i < NUM_SYNC_REP_WAIT_MODE; i++)
-				SyncRepWakeQueue(true, i);
+				SyncRepWakeQueue(true, i, InvalidXLogRecPtr);
 		}
 
 		/*
@@ -696,13 +970,31 @@ SyncRepQueueIsOrderedByLSN(int mode)
 #endif
 
 /*
+ * Make sure that CausalReadsWaitForLSN can't return until after the given
+ * lease expiry time has been reached.  In other words, revoke the lease.
+ *
+ * Wake up all backends waiting in CausalReadsWaitForLSN, because the set of
+ * available/joining peers has changed, and there is a new stall time they
+ * need to observe.
+ */
+void
+CausalReadsBeginStall(TimestampTz lease_expiry_time)
+{
+	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+	WalSndCtl->stall_causal_reads_until =
+		Max(WalSndCtl->stall_causal_reads_until, lease_expiry_time);
+	SyncRepWakeQueue(true, SYNC_REP_WAIT_CAUSAL_READS, InvalidXLogRecPtr);
+	LWLockRelease(SyncRepLock);
+}
+
+/*
  * ===========================================================
  * Synchronous Replication functions executed by any process
  * ===========================================================
  */
 
 bool
-check_synchronous_standby_names(char **newval, void **extra, GucSource source)
+check_standby_names(char **newval, void **extra, GucSource source)
 {
 	char	   *rawstring;
 	List	   *elemlist;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index d061dfe..e68384c 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -55,6 +55,7 @@
 #include "libpq/pqformat.h"
 #include "libpq/pqsignal.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "replication/walsender.h"
 #include "storage/ipc.h"
@@ -150,7 +151,8 @@ static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
 static void XLogWalRcvSendReply(bool force, bool requestReply, bool includeApplyTimestamp);
 static void XLogWalRcvSendHSFeedback(bool immed);
-static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
+static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+								  TimestampTz *causalReadsUntil);
 
 /* Signal handlers */
 static void WalRcvSigHupHandler(SIGNAL_ARGS);
@@ -857,6 +859,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 	XLogRecPtr	walEnd;
 	TimestampTz sendTime;
 	bool		replyRequested;
+	TimestampTz causalReadsLease;
 
 	resetStringInfo(&incoming_message);
 
@@ -877,7 +880,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				walEnd = pq_getmsgint64(&incoming_message);
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, NULL);
 
 				buf += hdrlen;
 				len -= hdrlen;
@@ -889,7 +892,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				bool reportApplyTimestamp = false;
 
 				/* copy message to StringInfo */
-				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
+				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char) + sizeof(int64);
 				if (len != hdrlen)
 					ereport(ERROR,
 							(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -901,8 +904,10 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
 				replyRequested = pq_getmsgbyte(&incoming_message);
+				causalReadsLease = IntegerTimestampToTimestampTz(
+					pq_getmsgint64(&incoming_message));
 
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, &causalReadsLease);
 
 				/*
 				 * If no apply timestamps have been sent at the request of the
@@ -1253,15 +1258,52 @@ XLogWalRcvSendHSFeedback(bool immed)
  * Update shared memory status upon receiving a message from primary.
  *
  * 'walEnd' and 'sendTime' are the end-of-WAL and timestamp of the latest
- * message, reported by primary.
+ * message, reported by primary.  'causalReadsLease' is a pointer to
+ * the time the primary promises that this standby can safely claim to be
+ * causally consistent, to 0 if it cannot, or a NULL pointer for no change.
  */
 static void
-ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
+ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+					  TimestampTz *causalReadsLease)
 {
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
 	static TimestampTz lastRecordedTimestamp = 0;
 
+	/* Sanity check for the causalReadsLease time. */
+	if (causalReadsLease != NULL && *causalReadsLease != 0)
+	{
+		/* Deduce max_clock_skew from the causalReadsLease and sendTime. */
+#ifdef HAVE_INT64_TIMESTAMP
+		int64 diffMillis = (*causalReadsLease - sendTime) / 1000;
+#else
+		int64 diffMillis = (*causalReadsLease - sendTime) * 1000;
+#endif
+		int64 max_clock_skew = diffMillis / (CAUSAL_READS_CLOCK_SKEW_RATIO - 1);
+
+		if (sendTime > TimestampTzPlusMilliseconds(lastMsgReceiptTime, max_clock_skew))
+		{
+			/*
+			 * The primary's clock is more than max_clock_skew + network
+			 * latency ahead of the standby's clock.  (If the primary's clock
+			 * is more than max_clock_skew ahead of the standby's clock, but
+			 * by less than the network latency, then there isn't much we can
+			 * do to detect that; but it still seems useful to have this basic
+			 * sanity check for wildly misconfigured servers.)
+			 */
+			elog(LOG, "the primary server's clock time is too far ahead");
+			causalReadsLease = NULL;
+		}
+		/*
+		 * We could also try to detect cases where sendTime is more than
+		 * max_clock_skew in the past according to the standby's clock, but
+		 * that is indistinguishable from network latency/buffering, so we
+		 * could produce misleading error messages; if we do nothing, the
+		 * consequence is 'standby is not available for causal reads' errors
+		 * which should cause the user to investigate.
+		 */
+	}
+
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
 	if (walrcv->latestWalEnd < walEnd)
@@ -1269,6 +1311,8 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->latestWalEnd = walEnd;
 	walrcv->lastMsgSendTime = sendTime;
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
+	if (causalReadsLease != NULL)
+		walrcv->causalReadsLease = *causalReadsLease;
 	SpinLockRelease(&walrcv->mutex);
 
 	/*
diff --git a/src/backend/replication/walreceiverfuncs.c b/src/backend/replication/walreceiverfuncs.c
index 5f6e423..e502f74 100644
--- a/src/backend/replication/walreceiverfuncs.c
+++ b/src/backend/replication/walreceiverfuncs.c
@@ -28,6 +28,7 @@
 #include "replication/walreceiver.h"
 #include "storage/pmsignal.h"
 #include "storage/shmem.h"
+#include "utils/guc.h"
 #include "utils/timestamp.h"
 
 WalRcvData *WalRcv = NULL;
@@ -374,3 +375,21 @@ GetReplicationTransferLatency(void)
 
 	return ms;
 }
+
+/*
+ * Used by snapmgr to check if this standby has a valid lease, granting it the
+ * right to consider itself available for causal reads.
+ */
+bool
+WalRcvCausalReadsAvailable(void)
+{
+	WalRcvData *walrcv = WalRcv;
+	TimestampTz now = GetCurrentTimestamp();
+	bool result;
+
+	SpinLockAcquire(&walrcv->mutex);
+	result = walrcv->causalReadsLease != 0 && now <= walrcv->causalReadsLease;
+	SpinLockRelease(&walrcv->mutex);
+
+	return result;
+}
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index a3a2097..b64e93d 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -153,9 +153,20 @@ static StringInfoData tmpbuf;
  */
 static TimestampTz last_reply_timestamp = 0;
 
+static TimestampTz last_keepalive_timestamp = 0;
+
 /* Have we sent a heartbeat message asking for reply, since last reply? */
 static bool waiting_for_ping_response = false;
 
+/* How long do need to stay in JOINING state? */
+static XLogRecPtr causal_reads_joining_until = 0;
+
+/* The last causal reads lease sent to the standby. */
+static TimestampTz causal_reads_last_lease = 0;
+
+/* Is this WALSender listed in causal_reads_standby_names? */
+static bool am_potential_causal_reads_standby = false;
+
 /*
  * While streaming WAL in Copy mode, streamingDoneSending is set to true
  * after we have sent CopyDone. We should not send any more CopyData messages
@@ -241,6 +252,57 @@ InitWalSender(void)
 	SendPostmasterSignal(PMSIGNAL_ADVANCE_STATE_MACHINE);
 }
 
+ /*
+ * If we are exiting unexpectedly, we may need to communicate with concurrent
+ * causal_reads commits to maintain the causal consistency guarantee.
+ */
+static void
+PrepareUncleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * We've lost contact with the standby, but it may still be alive.  We
+		 * can't let any causal_reads transactions return until we've stalled
+		 * for long enough for a zombie standby to start raising errors
+		 * because its lease has expired.
+		 */
+		elog(LOG, "standby \"%s\" is lost (no longer available for causal reads)", application_name);
+		CausalReadsBeginStall(causal_reads_last_lease);
+
+		/*
+		 * We set the state to a lower level _after_ beginning the stall,
+		 * otherwise there would be a tiny window where commits could return
+		 * without observing the stall.
+		 */
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
+ * We are shutting down because we received a goodbye message from the
+ * walreceiver.
+ */
+static void
+PrepareCleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * The standby is shutting down, so it won't be running any more
+		 * transactions.  It is therefore safe to stop waiting for it, and no
+		 * stall is necessary.
+		 */
+		elog(LOG, "standby \"%s\" is leaving (no longer available for causal reads)", application_name);
+
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
 /*
  * Clean up after an error.
  *
@@ -264,7 +326,10 @@ WalSndErrorCleanup(void)
 
 	replication_active = false;
 	if (walsender_ready_to_stop)
+	{
+		PrepareUncleanExit();
 		proc_exit(0);
+	}
 
 	/* Revert back to startup state */
 	WalSndSetState(WALSNDSTATE_STARTUP);
@@ -276,6 +341,8 @@ WalSndErrorCleanup(void)
 static void
 WalSndShutdown(void)
 {
+	PrepareUncleanExit();
+
 	/*
 	 * Reset whereToSendOutput to prevent ereport from attempting to send any
 	 * more messages to the standby.
@@ -1386,6 +1453,7 @@ ProcessRepliesIfAny(void)
 		if (r < 0)
 		{
 			/* unexpected error or EOF */
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1402,6 +1470,7 @@ ProcessRepliesIfAny(void)
 		resetStringInfo(&reply_message);
 		if (pq_getmessage(&reply_message, 0))
 		{
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1451,6 +1520,7 @@ ProcessRepliesIfAny(void)
 				 * 'X' means that the standby is closing down the socket.
 				 */
 			case 'X':
+				PrepareCleanExit();
 				proc_exit(0);
 
 			default:
@@ -1582,6 +1652,83 @@ ProcessStandbyReplyMessage(void)
 	 */
 	{
 		WalSnd *walsnd = MyWalSnd;
+		WalSndCausalReadsState causal_reads_state = walsnd->causal_reads_state;
+		bool causal_reads_state_changed = false;
+		bool causal_reads_set_joining_until = false;
+
+		/*
+		 * Handle causal reads state transitions, if a causal_reads_timeout is
+		 * configured, this standby is listed in causal_reads_standby_names,
+		 * and we are a primary database (not a cascading standby).
+		 */
+		if (am_potential_causal_reads_standby &&
+			!am_cascading_walsender &&
+			applyLagUs >= 0)
+		{
+			if (applyLagUs / 1000 < causal_reads_timeout)
+			{
+				if (causal_reads_state == WALSNDCRSTATE_UNAVAILABLE)
+				{
+					/*
+					 * The standby is applying fast enough.  We can't grant a
+					 * lease yet though, we need to wait for everything that
+					 * was committed while this standby was unavailable to be
+					 * applied first.  We move to joining state while we wait
+					 * for the standby to catch up.
+					 */
+					causal_reads_state = WALSNDCRSTATE_JOINING;
+					causal_reads_set_joining_until = true;
+					causal_reads_state_changed = true;
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING &&
+						 applyPtr >= causal_reads_joining_until)
+				{
+					/*
+					 * The standby has applied everything committed before we
+					 * reached joining state, and has been waiting for remote
+					 * apply on this standby while it's been in joining state,
+					 * so it is safe to move to available state and send a
+					 * lease.
+					 */
+					causal_reads_state = WALSNDCRSTATE_AVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+			else
+			{
+				if (causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+					/*
+					 * We are dropping a causal reads available standby, so we
+					 * mustn't let any commit command that is waiting in
+					 * CausalReadsWaitForLSN return until we are sure that the
+					 * standby definitely knows that it's not available and
+					 * starts raising errors for causal_reads transactions.
+					 * TODO: We could just wait until the standby acks that
+					 * its lease has been cancelled, and start numbering
+					 * keepalives and sending the number back in replies, so
+					 * we know it's acking the right message; then lagging
+					 * standbys would be less disruptive, but for now we just
+					 * wait for the lease to expire, as we do when we lose
+					 * contact with a standby, for the sake of simplicity.
+					 */
+					CausalReadsBeginStall(causal_reads_last_lease);
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING)
+				{
+					/*
+					 * Dropping a joining standby doesn't require a stall,
+					 * because the standby doesn't think it's available, so
+					 * it's already raising the error for causal_reads
+					 * transactions.
+					 */
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+		}
 
 		SpinLockAcquire(&walsnd->mutex);
 		walsnd->write = writePtr;
@@ -1589,11 +1736,33 @@ ProcessStandbyReplyMessage(void)
 		walsnd->apply = applyPtr;
 		if (applyLagUs >= 0)
 			walsnd->applyLagUs = applyLagUs;
+		walsnd->causal_reads_state = causal_reads_state;
 		SpinLockRelease(&walsnd->mutex);
+
+		if (causal_reads_set_joining_until)
+		{
+			/*
+			 * Record the end of the primary's WAL at some arbitrary point
+			 * observed _after_ we moved to joining state (so that causal
+			 * reads commits start waiting, closing a race).  The standby
+			 * won't become available until it has replayed up to here.
+			 */
+			causal_reads_joining_until = GetFlushRecPtr();
+		}
+
+		if (causal_reads_state_changed)
+		{
+			WalSndKeepalive(true);
+			elog(LOG, "standby \"%s\" is %s", application_name,
+				 causal_reads_state == WALSNDCRSTATE_UNAVAILABLE ? "unavailable for causal reads" :
+				 causal_reads_state == WALSNDCRSTATE_JOINING ? "joining as a causal reads standby..." :
+				 causal_reads_state == WALSNDCRSTATE_AVAILABLE ? "available for causal reads" :
+				 "UNKNOWN");
+		}
 	}
 
 	if (!am_cascading_walsender)
-		SyncRepReleaseWaiters();
+		SyncRepReleaseWaiters(MyWalSnd->causal_reads_state >= WALSNDCRSTATE_JOINING);
 
 	/*
 	 * Advance our local xmin horizon when the client confirmed a flush.
@@ -1734,33 +1903,53 @@ ProcessStandbyHSFeedbackMessage(void)
  * If wal_sender_timeout is enabled we want to wake up in time to send
  * keepalives and to abort the connection if wal_sender_timeout has been
  * reached.
+ *
+ * But if causal_reads_timeout is enabled, we override that and send
+ * keepalives at a constant rate to replace expiring leases.
  */
 static long
 WalSndComputeSleeptime(TimestampTz now)
 {
 	long		sleeptime = 10000;		/* 10 s */
 
-	if (wal_sender_timeout > 0 && last_reply_timestamp > 0)
+	if ((wal_sender_timeout > 0 && last_reply_timestamp > 0) ||
+		am_potential_causal_reads_standby)
 	{
 		TimestampTz wakeup_time;
 		long		sec_to_timeout;
 		int			microsec_to_timeout;
 
-		/*
-		 * At the latest stop sleeping once wal_sender_timeout has been
-		 * reached.
-		 */
-		wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-												  wal_sender_timeout);
-
-		/*
-		 * If no ping has been sent yet, wakeup when it's time to do so.
-		 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
-		 * the timeout passed without a response.
-		 */
-		if (!waiting_for_ping_response)
+		if (am_potential_causal_reads_standby)
+		{
+			/*
+			 * Leases last for a period of between 50% and 100% of
+			 * causal_reads_timeout, depending on clock skew, assuming clock
+			 * skew is under the 25% of causal_reads_timeout.  We send new
+			 * leases every half a lease, so that there are no gaps between
+			 * leases.
+			 */
+			wakeup_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+													  causal_reads_timeout /
+													  CAUSAL_READS_KEEPALIVE_RATIO);
+		}
+		else
+		{
+			/*
+			 * At the latest stop sleeping once wal_sender_timeout has been
+			 * reached.
+			 */
 			wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-													  wal_sender_timeout / 2);
+													  wal_sender_timeout);
+
+			/*
+			 * If no ping has been sent yet, wakeup when it's time to do so.
+			 * WalSndKeepaliveIfNecessary() wants to send a keepalive once
+			 * half of the timeout passed without a response.
+			 */
+			if (!waiting_for_ping_response)
+				wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+														  wal_sender_timeout / 2);
+		}
 
 		/* Compute relative time until wakeup. */
 		TimestampDifference(now, wakeup_time,
@@ -1776,20 +1965,33 @@ WalSndComputeSleeptime(TimestampTz now)
 /*
  * Check whether there have been responses by the client within
  * wal_sender_timeout and shutdown if not.
+ *
+ * If causal_reads_timeout is configured we override that, so that
+ * unresponsive standbys are detected sooner.
  */
 static void
 WalSndCheckTimeOut(TimestampTz now)
 {
 	TimestampTz timeout;
+	int allowed_time;
 
 	/* don't bail out if we're doing something that doesn't require timeouts */
 	if (last_reply_timestamp <= 0)
 		return;
 
-	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
-										  wal_sender_timeout);
+	/*
+	 * If a causal_reads_timeout is configured, it is used instead of
+	 * wal_sender_timeout, to limit the time before an unresponsive causal
+	 * reads standby is dropped.
+	 */
+	if (am_potential_causal_reads_standby)
+		allowed_time = causal_reads_timeout;
+	else
+		allowed_time = wal_sender_timeout;
 
-	if (wal_sender_timeout > 0 && now >= timeout)
+	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
+										  allowed_time);
+	if (allowed_time > 0 && now >= timeout)
 	{
 		/*
 		 * Since typically expiration of replication timeout means
@@ -1822,6 +2024,9 @@ WalSndLoop(WalSndSendDataCallback send_data)
 	last_reply_timestamp = GetCurrentTimestamp();
 	waiting_for_ping_response = false;
 
+	/* Check if we are managing potential causal_reads standby. */
+	am_potential_causal_reads_standby = CausalReadsPotentialStandby();
+
 	/*
 	 * Loop until we reach the end of this timeline or the client requests to
 	 * stop streaming.
@@ -1982,6 +2187,7 @@ InitWalSenderSlot(void)
 			walsnd->flush = InvalidXLogRecPtr;
 			walsnd->apply = InvalidXLogRecPtr;
 			walsnd->state = WALSNDSTATE_STARTUP;
+			walsnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
 			walsnd->latch = &MyProc->procLatch;
 			SpinLockRelease(&walsnd->mutex);
 			/* don't need the lock anymore */
@@ -2751,6 +2957,24 @@ WalSndGetStateString(WalSndState state)
 	return "UNKNOWN";
 }
 
+/*
+ * Return a string constant representing the causal reads state. This is used
+ * in system views, and should *not* be translated.
+ */
+static const char *
+WalSndGetCausalReadsStateString(WalSndCausalReadsState causal_reads_state)
+{
+	switch (causal_reads_state)
+	{
+		case WALSNDCRSTATE_UNAVAILABLE:
+			return "unavailable";
+		case WALSNDCRSTATE_JOINING:
+			return "joining";
+		case WALSNDCRSTATE_AVAILABLE:
+			return "available";
+	}
+	return "UNKNOWN";
+}
 
 /*
  * Returns activity of walsenders, including pids and xlog locations sent to
@@ -2759,7 +2983,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	9
+#define PG_STAT_GET_WAL_SENDERS_COLS	10
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2810,6 +3034,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		int64		applyLagUs;
 		int			priority;
 		WalSndState state;
+		WalSndCausalReadsState causalReadsState;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
 		bool		nulls[PG_STAT_GET_WAL_SENDERS_COLS];
 
@@ -2819,6 +3044,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		SpinLockAcquire(&walsnd->mutex);
 		sentPtr = walsnd->sentPtr;
 		state = walsnd->state;
+		causalReadsState = walsnd->causal_reads_state;
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
@@ -2893,6 +3119,9 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				values[8] = CStringGetTextDatum("sync");
 			else
 				values[8] = CStringGetTextDatum("potential");
+
+			values[9] =
+				CStringGetTextDatum(WalSndGetCausalReadsStateString(causalReadsState));
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
@@ -2912,14 +3141,52 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 static void
 WalSndKeepalive(bool requestReply)
 {
+	TimestampTz now;
+	TimestampTz causal_reads_lease;
+
 	elog(DEBUG2, "sending replication keepalive");
 
+	/*
+	 * If the walsender currently deems the standby to be available for causal
+	 * reads, then it grants a causal reads lease.  The lease authorizes the
+	 * standby to consider itself available for causal reads until a short
+	 * time in the future.  The primary promises to uphold the causal reads
+	 * guarantee until that time, by stalling commits until the the lease has
+	 * expired if necessary.
+	 */
+	now = GetCurrentTimestamp();
+	if (MyWalSnd->causal_reads_state < WALSNDCRSTATE_AVAILABLE)
+		causal_reads_lease = 0; /* Not available, no lease granted. */
+	else
+	{
+		/*
+		 * Since this timestamp is being sent to the standby where it will be
+		 * compared against a time generated by the standby's system clock, we
+		 * must consider clock skew.  First, we decide on a maximum tolerable
+		 * difference between system clocks.  If the primary's clock is ahead
+		 * of the standby's by more than this, then all bets are off (the
+		 * standby could falsely believe it has a valid lease).  If the
+		 * primary's clock is behind the standby's by more than this, then the
+		 * standby will err the other way and generate spurious errors in
+		 * causal_reads mode.  Rather than having a separate GUC for this, we
+		 * derive it from causal_reads_timeout.
+		 */
+		int max_clock_skew = causal_reads_timeout / CAUSAL_READS_CLOCK_SKEW_RATIO;
+
+		/* Compute and remember the expiry time of the lease we're granting. */
+		causal_reads_last_lease = TimestampTzPlusMilliseconds(now, causal_reads_timeout);
+		/* The version we'll send to the standby is adjusted to tolerate clock skew. */
+		causal_reads_lease =
+			TimestampTzPlusMilliseconds(causal_reads_last_lease, -max_clock_skew);
+	}
+
 	/* construct the message... */
 	resetStringInfo(&output_message);
 	pq_sendbyte(&output_message, 'k');
 	pq_sendint64(&output_message, sentPtr);
-	pq_sendint64(&output_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(now));
 	pq_sendbyte(&output_message, requestReply ? 1 : 0);
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(causal_reads_lease));
 
 	/* ... and send it wrapped in CopyData */
 	pq_putmessage_noblock('d', output_message.data, output_message.len);
@@ -2937,23 +3204,35 @@ WalSndKeepaliveIfNecessary(TimestampTz now)
 	 * Don't send keepalive messages if timeouts are globally disabled or
 	 * we're doing something not partaking in timeouts.
 	 */
-	if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
-		return;
-
-	if (waiting_for_ping_response)
-		return;
+	if (!am_potential_causal_reads_standby)
+	{
+		if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
+			return;
+		if (waiting_for_ping_response)
+			return;
+	}
 
 	/*
 	 * If half of wal_sender_timeout has lapsed without receiving any reply
 	 * from the standby, send a keep-alive message to the standby requesting
 	 * an immediate reply.
+	 *
+	 * If causal_reads_timeout has been configured, use it to control
+	 * keepalive intervals rather than wal_sender_timeout, so that we can keep
+	 * replacing leases at the right frequency.
 	 */
-	ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-											wal_sender_timeout / 2);
+	if (am_potential_causal_reads_standby)
+		ping_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+												causal_reads_timeout /
+												CAUSAL_READS_KEEPALIVE_RATIO);
+	else
+		ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+												wal_sender_timeout / 2);
 	if (now >= ping_time)
 	{
 		WalSndKeepalive(true);
 		waiting_for_ping_response = true;
+		last_keepalive_timestamp = now;
 
 		/* Try to flush pending output to the client */
 		if (pq_flush_if_writable() != 0)
diff --git a/src/backend/utils/errcodes.txt b/src/backend/utils/errcodes.txt
index 04c9c00..d4bf0c0 100644
--- a/src/backend/utils/errcodes.txt
+++ b/src/backend/utils/errcodes.txt
@@ -302,6 +302,7 @@ Section: Class 40 - Transaction Rollback
 40001    E    ERRCODE_T_R_SERIALIZATION_FAILURE                              serialization_failure
 40003    E    ERRCODE_T_R_STATEMENT_COMPLETION_UNKNOWN                       statement_completion_unknown
 40P01    E    ERRCODE_T_R_DEADLOCK_DETECTED                                  deadlock_detected
+40P02    E    ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE                         causal_reads_not_available
 
 Section: Class 42 - Syntax Error or Access Rule Violation
 
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index acedccd..aaf8fae 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -1633,6 +1633,16 @@ static struct config_bool ConfigureNamesBool[] =
 		NULL, NULL, NULL
 	},
 
+	{
+		{"causal_reads", PGC_USERSET, REPLICATION_STANDBY,
+		 gettext_noop("Enables causal reads."),
+		 NULL
+		},
+		&causal_reads,
+		false,
+		NULL, NULL, NULL
+	},
+
 	/* End-of-list marker */
 	{
 		{NULL, 0, 0, NULL, NULL}, NULL, false, NULL, NULL, NULL
@@ -1791,6 +1801,17 @@ static struct config_int ConfigureNamesInt[] =
 	},
 
 	{
+		{"causal_reads_timeout", PGC_SIGHUP, REPLICATION_STANDBY,
+			gettext_noop("Sets the maximum apply lag before causal reads standbys are no longer available."),
+			NULL,
+			GUC_UNIT_MS
+		},
+		&causal_reads_timeout,
+		0, 0, INT_MAX,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"max_connections", PGC_POSTMASTER, CONN_AUTH_SETTINGS,
 			gettext_noop("Sets the maximum number of concurrent connections."),
 			NULL
@@ -3387,10 +3408,21 @@ static struct config_string ConfigureNamesString[] =
 		},
 		&SyncRepStandbyNames,
 		"",
-		check_synchronous_standby_names, NULL, NULL
+		check_standby_names, NULL, NULL
 	},
 
 	{
+		{"causal_reads_standby_names", PGC_SIGHUP, REPLICATION_MASTER,
+			gettext_noop("List of names of potential causal reads standbys."),
+			NULL,
+			GUC_LIST_INPUT
+		},
+		&causal_reads_standby_names,
+		"*",
+		check_standby_names, NULL, NULL
+ 	},
+
+	{
 		{"default_text_search_config", PGC_USERSET, CLIENT_CONN_LOCALE,
 			gettext_noop("Sets default text search configuration."),
 			NULL
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index 085099c..733b9e5 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -244,6 +244,15 @@
 				# from standby(s); '*' = all
 #vacuum_defer_cleanup_age = 0	# number of xacts by which cleanup is delayed
 
+#causal_reads_timeout = 0s      # maximum replication delay to tolerate from
+                                # standbys before dropping them from the set of
+				# available causal reads peers; 0 to disable
+				# causal reads
+
+#causal_reads_standy_names = '*'
+                                # standby servers that can potentially become
+				# available for causal reads; '*' = all
+
 # - Standby Servers -
 
 # These settings are ignored on a master server.
@@ -266,6 +275,14 @@
 #wal_retrieve_retry_interval = 5s	# time to wait before retrying to
 					# retrieve WAL after a failed attempt
 
+# - All Servers -
+
+#causal_reads = off                     # "on" in any pair of consecutive
+                                        # transactions guarantees that the second
+					# can see the first (even if the second
+					# is run on a standby), or will raise an
+					# error to report that the standby is
+					# unavailable for causal reads
 
 #------------------------------------------------------------------------------
 # QUERY TUNING
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index b88e012..6336240 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -46,8 +46,11 @@
 
 #include "access/transam.h"
 #include "access/xact.h"
+#include "access/xlog.h"
 #include "lib/pairingheap.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
+#include "replication/walreceiver.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
 #include "storage/procarray.h"
@@ -209,6 +212,16 @@ GetTransactionSnapshot(void)
 				 "cannot take query snapshot during a parallel operation");
 
 		/*
+		 * In causal_reads mode on a standby, check if we have definitely
+		 * applied WAL for any COMMIT that returned successfully on the
+		 * primary.
+		 */
+		if (causal_reads && RecoveryInProgress() && !WalRcvCausalReadsAvailable())
+			ereport(ERROR,
+					(errcode(ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE),
+					 errmsg("standby is not available for causal reads")));
+
+		/*
 		 * In transaction-snapshot mode, the first snapshot must live until
 		 * end of xact regardless of what the caller does with it, so we must
 		 * make a copy of it rather than returning CurrentSnapshotData
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 811f10c..dba207c 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2712,7 +2712,7 @@ DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f
 DESCR("statistics: information about currently active backends");
 DATA(insert OID = 3318 (  pg_stat_get_progress_info           PGNSP PGUID 12 1 100 0 0 f f f f f t s r 1 0 2249 "25" "{25,23,26,26,20,20,20,20,20,20,20,20,20,20}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{cmdtype,pid,datid,relid,param1,param2,param3,param4,param5,param6,param7,param8,param9,param10}" _null_ _null_ pg_stat_get_progress_info _null_ _null_ _null_ ));
 DESCR("statistics: information about progress of backends running maintenance command");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25}" "{o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25,25}" "{o,o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state,causal_reads_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index 28b68f6..253f831 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -24,14 +24,33 @@
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
 #define SYNC_REP_WAIT_APPLY		2
+#define SYNC_REP_WAIT_CAUSAL_READS 3
 
-#define NUM_SYNC_REP_WAIT_MODE	3
+#define NUM_SYNC_REP_WAIT_MODE	4
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
 #define SYNC_REP_WAITING			1
 #define SYNC_REP_WAIT_COMPLETE		2
 
+/*
+ * ratio of causal_read_timeout to max_clock_skew (4 means than the maximum
+ * tolerated clock difference between primary and standbys using causal_reads
+ * is 1/4 of causal_reads_timeout)
+ */
+#define CAUSAL_READS_CLOCK_SKEW_RATIO 4
+
+/*
+ * ratio of causal_reads_timeout to keepalive time (2 means that the effective
+ * keepalive time is 1/2 of the causal_reads_timeout GUC when it is non-zero)
+ */
+#define CAUSAL_READS_KEEPALIVE_RATIO 2
+
+/* GUC variables */
+extern int causal_reads_timeout;
+extern bool causal_reads;
+extern char *causal_reads_standby_names;
+
 /* user-settable parameters for synchronous replication */
 extern char *SyncRepStandbyNames;
 
@@ -43,16 +62,23 @@ extern void SyncRepCleanupAtProcExit(void);
 
 /* called by wal sender */
 extern void SyncRepInitConfig(void);
-extern void SyncRepReleaseWaiters(void);
+extern void SyncRepReleaseWaiters(bool walsender_cr_available_or_joining);
 
 /* called by checkpointer */
 extern void SyncRepUpdateSyncStandbysDefined(void);
 
+/* called by user backend (xact.c) */
+extern void CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN);
+
+/* called by wal sender */
+extern void CausalReadsBeginStall(TimestampTz lease_expiry_time);
+extern bool CausalReadsPotentialStandby(void);
+
 /* forward declaration to avoid pulling in walsender_private.h */
 struct WalSnd;
 extern struct WalSnd *SyncRepGetSynchronousStandby(void);
 
-extern bool check_synchronous_standby_names(char **newval, void **extra, GucSource source);
+extern bool check_standby_names(char **newval, void **extra, GucSource source);
 extern void assign_synchronous_commit(int newval, void *extra);
 
 #endif   /* _SYNCREP_H */
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 3294df9..7f83934 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -80,6 +80,13 @@ typedef struct
 	TimeLineID	receivedTLI;
 
 	/*
+	 * causalReadsLease is the time until which the primary has authorized
+	 * this standby to consider itself available for causal_reads mode, or 0
+	 * for not authorized.
+	 */
+	TimestampTz causalReadsLease;
+
+	/*
 	 * latestChunkStart is the starting byte position of the current "batch"
 	 * of received WAL.  It's actually the same as the previous value of
 	 * receivedUpto before the last flush to disk.  Startup process can use
@@ -164,4 +171,6 @@ extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
 extern void WalRcvWakeup(void);
 
+extern bool WalRcvCausalReadsAvailable(void);
+
 #endif   /* _WALRECEIVER_H */
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 4de43e8..f6e0e9e 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -27,6 +27,13 @@ typedef enum WalSndState
 	WALSNDSTATE_STREAMING
 } WalSndState;
 
+typedef enum WalSndCausalReadsState
+{
+	WALSNDCRSTATE_UNAVAILABLE = 0,
+	WALSNDCRSTATE_JOINING,
+	WALSNDCRSTATE_AVAILABLE
+} WalSndCausalReadsState;
+
 /*
  * Each walsender has a WalSnd struct in shared memory.
  */
@@ -34,6 +41,7 @@ typedef struct WalSnd
 {
 	pid_t		pid;			/* this walsender's process id, or 0 */
 	WalSndState state;			/* this walsender's state */
+	WalSndCausalReadsState causal_reads_state; /* the walsender's causal reads state */
 	XLogRecPtr	sentPtr;		/* WAL has been sent up to this point */
 	bool		needreload;		/* does currently-open file need to be
 								 * reloaded? */
@@ -89,6 +97,12 @@ typedef struct
 	 */
 	bool		sync_standbys_defined;
 
+	/*
+	 * Until when must commits in causal_reads stall?  This is used to wait
+	 * for causal reads leases to expire.
+	 */
+	TimestampTz	stall_causal_reads_until;
+
 	WalSnd		walsnds[FLEXIBLE_ARRAY_MEMBER];
 } WalSndCtlData;
 
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 6f80548..2727533 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1762,10 +1762,11 @@ pg_stat_replication| SELECT s.pid,
     w.replay_location,
     w.replay_lag,
     w.sync_priority,
-    w.sync_state
+    w.sync_state,
+    w.causal_reads_state
    FROM pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, waiting, xact_start, query_start, backend_start, state_change, client_addr, client_hostname, client_port, backend_xid, backend_xmin, ssl, sslversion, sslcipher, sslbits, sslcompression, sslclientdn),
     pg_authid u,
-    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state)
+    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state, causal_reads_state)
   WHERE ((s.usesysid = u.oid) AND (s.pid = w.pid));
 pg_stat_ssl| SELECT s.pid,
     s.ssl,
#48Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Thomas Munro (#47)
4 attachment(s)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Thu, Mar 10, 2016 at 8:35 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Thu, Mar 10, 2016 at 12:35 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

I looked into COMMIT PREPARED replay feedback and realised that it
doesn't need any special handling beyond what is already in
xact_redo_commit. However, I see now that I *do* need to do something
when replaying PREPARE TRANSACTION, as you said. Not for causal reads
though -- it doesn't care about an operation with no visible effect --
but for synchronous_commit = remote_apply. I am thinking about how to
fix that.

Ok, here is a new version with the following changes:

1. If you PREPARE TRANSACTION with synchronous_commit = remote_apply,
it just waits for remote WAL flush. When you eventually
COMMIT/ROLLBACK PREPARED, it waits for remote apply. Also, the
XACT_COMPLETION_SYNC_APPLY_FEEDBACK bit is now set in appropriate
abort records, and then handled in recovery, just as for commits, so
that ROLLBACK and ROLLBACK PREPARED return at the right time in
remote_apply.

2. I fixed a recently introduced stupid bug that caused causal reads
to be broken when sync rep wasn't also configured (I had lost a change
to stop SyncRepReleaseWaiters from leaving early, when I split up the
patch... oops).

3. I switched the pg_stat_replication.replay_lag accounting from
milliseconds to microseconds. The measured lag with fast machines on
local networks can sometimes be sub-millisecond or very low numbers of
milliseconds, so it's interesting to see more detail (depending on the
primary's clock resolution).

The last patches I posted don't apply today due to changes in master,
so here's a freshly merged patch series.

--
Thomas Munro
http://www.enterprisedb.com

Attachments:

0001-remote-apply-v4.patchapplication/octet-stream; name=0001-remote-apply-v4.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 6c73fb4..e8e5d5f 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -2149,7 +2149,7 @@ include_dir 'conf.d'
         Specifies whether transaction commit will wait for WAL records
         to be written to disk before the command returns a <quote>success</>
         indication to the client.  Valid values are <literal>on</>,
-        <literal>remote_write</>, <literal>local</>, and <literal>off</>.
+        <literal>remote_write</>, <literal> remote_apply</>, <literal>local</>, and <literal>off</>.
         The default, and safe, setting
         is <literal>on</>.  When <literal>off</>, there can be a delay between
         when success is reported to the client and when the transaction is
@@ -2183,6 +2183,10 @@ include_dir 'conf.d'
         ensure data preservation even if the standby instance of
         <productname>PostgreSQL</> were to crash, but not if the standby
         suffers an operating-system-level crash.
+        When set to <literal>remote_apply</>, commits will wait until a reply
+        from the current synchronous stanbyindicates it has received the
+        commit record of the transaction and applied it, so that it has become
+        visible to queries.
        </para>
        <para>
         When synchronous
diff --git a/doc/src/sgml/high-availability.sgml b/doc/src/sgml/high-availability.sgml
index 6cb690c..2600fba 100644
--- a/doc/src/sgml/high-availability.sgml
+++ b/doc/src/sgml/high-availability.sgml
@@ -1081,6 +1081,9 @@ primary_slot_name = 'node_a_slot'
     WAL record is then sent to the standby. The standby sends reply
     messages each time a new batch of WAL data is written to disk, unless
     <varname>wal_receiver_status_interval</> is set to zero on the standby.
+    In the case that <varname>synchronous_commit</> is set to
+    <literal>remote_apply</>, the standby sends reply messages when the commit
+    record is replayed, making the transaction visible.
     If the standby is the first matching standby, as specified in
     <varname>synchronous_standby_names</> on the primary, the reply
     messages from that standby will be used to wake users waiting for
@@ -1107,6 +1110,14 @@ primary_slot_name = 'node_a_slot'
    </para>
 
    <para>
+    Setting <varname>synchronous_commit</> to <literal>remote_apply</> will
+    cause each commit to wait until the current synchronous standby reports
+    that it has replayed the transaction, making it visible to user queries.
+    In simple cases, this allows for load balancing with causal consistency
+    on a single hot standby.
+   </para>
+
+   <para>
     Users will stop waiting if a fast shutdown is requested.  However, as
     when using asynchronous replication, the server will not fully
     shutdown until all outstanding WAL records are transferred to the currently
@@ -1160,9 +1171,10 @@ primary_slot_name = 'node_a_slot'
     <title>Planning for High Availability</title>
 
    <para>
-    Commits made when <varname>synchronous_commit</> is set to <literal>on</>
-    or <literal>remote_write</> will wait until the synchronous standby responds. The response
-    may never occur if the last, or only, standby should crash.
+    Commits made when <varname>synchronous_commit</> is set to <literal>on</>,
+    <literal>remote_write</> or <literal>remote_apply</> will wait until the
+    synchronous standby responds. The response may never occur if the last, or
+    only, standby should crash.
    </para>
 
    <para>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index c4fd9ef..80721e7 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -1107,7 +1107,20 @@ EndPrepare(GlobalTransaction gxact)
 	 * Note that at this stage we have marked the prepare, but still show as
 	 * running in the procarray (twice!) and continue to hold locks.
 	 */
-	SyncRepWaitForLSN(gxact->prepare_end_lsn);
+	{
+		/*
+		 * Don't wait for the prepare to be applied remotely in remote_apply
+		 * mode, just wait for it to be flushed to the WAL.  We will wait for
+		 * apply when the transaction eventuallly commits or aborts.
+		 */
+		if (synchronous_commit == SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+			assign_synchronous_commit(SYNCHRONOUS_COMMIT_REMOTE_FLUSH, NULL);
+
+		SyncRepWaitForLSN(gxact->prepare_end_lsn);
+
+		if (synchronous_commit == SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+			assign_synchronous_commit(SYNCHRONOUS_COMMIT_REMOTE_APPLY, NULL);
+	}
 
 	records.tail = records.head = NULL;
 	records.num_chunks = 0;
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 8a2cd45..002bf50 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -5123,6 +5123,13 @@ XactLogCommitRecord(TimestampTz commit_time,
 		xl_xinfo.xinfo |= XACT_COMPLETION_FORCE_SYNC_COMMIT;
 
 	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this commit is applied.
+	*/
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
+	/*
 	 * Relcache invalidations requires information about the current database
 	 * and so does logical decoding.
 	 */
@@ -5300,6 +5307,13 @@ XactLogAbortRecord(TimestampTz abort_time,
 	if (xl_xinfo.xinfo & XACT_XINFO_HAS_TWOPHASE)
 		XLogRegisterData((char *) (&xl_twophase), sizeof(xl_xact_twophase));
 
+	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this abort is applied.
+	*/
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
 	return XLogInsert(RM_XACT_ID, info);
 }
 
@@ -5458,6 +5472,13 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 	if (XactCompletionForceSyncCommit(parsed->xinfo))
 		XLogFlush(lsn);
 
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit = remote_apply), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 /*
@@ -5544,6 +5565,14 @@ xact_redo_abort(xl_xact_parsed_abort *parsed, TransactionId xid)
 		smgrdounlink(srel, true);
 		smgrclose(srel);
 	}
+
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit = remote_apply), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 void
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 5b1c361..5744429 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -346,6 +346,12 @@ static XLogRecPtr RedoRecPtr;
 static bool doPageWrites;
 
 /*
+ * doRequestWalReceiverReply is used by recovery code to ask the main recovery
+ * loop to trigger a walreceiver reply.
+ */
+static bool doRequestWalReceiverReply;
+
+/*
  * RedoStartLSN points to the checkpoint's REDO location which is specified
  * in a backup label file, backup history file or control file. In standby
  * mode, XLOG streaming usually starts from the position where an invalid
@@ -6879,6 +6885,19 @@ StartupXLOG(void)
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
 				SpinLockRelease(&XLogCtl->info_lck);
 
+				/*
+				 * If rm_redo reported that it applied a commit record that
+				 * the master is waiting for by calling
+				 * XLogRequestWalReceiverReply, then we wake up the receiver
+				 * so that it notices the updated lastReplayedEndRecPtr and
+				 * sends a reply to the master.
+				 */
+				if (doRequestWalReceiverReply)
+				{
+					doRequestWalReceiverReply = false;
+					WalRcvWakeup();
+				}
+
 				/* Remember this record as the last-applied one */
 				LastRec = ReadRecPtr;
 
@@ -11596,3 +11615,13 @@ SetWalWriterSleeping(bool sleeping)
 	XLogCtl->WalWriterSleeping = sleeping;
 	SpinLockRelease(&XLogCtl->info_lck);
 }
+
+/*
+ * Called by redo code to indicate that the xlog replay loop should wake up
+ * the walreceiver process so that a reply can be sent to the primary.
+ */
+void
+XLogRequestWalReceiverReply(void)
+{
+	doRequestWalReceiverReply = true;
+}
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 92faf4e..4565348 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -416,6 +416,7 @@ SyncRepReleaseWaiters(void)
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
+	int			numapply = 0;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
@@ -462,12 +463,18 @@ SyncRepReleaseWaiters(void)
 		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
 		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
 	}
+	if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+	{
+		walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+		numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY);
+	}
 
 	LWLockRelease(SyncRepLock);
 
-	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X",
+	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%x",
 		 numwrite, (uint32) (MyWalSnd->write >> 32), (uint32) MyWalSnd->write,
-	   numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush);
+		 numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush,
+		 numapply, (uint32) (MyWalSnd->apply >> 32), (uint32) MyWalSnd->apply);
 
 	/*
 	 * If we are managing the highest priority standby, though we weren't
@@ -728,6 +735,9 @@ assign_synchronous_commit(int newval, void *extra)
 		case SYNCHRONOUS_COMMIT_REMOTE_FLUSH:
 			SyncRepWaitMode = SYNC_REP_WAIT_FLUSH;
 			break;
+		case SYNCHRONOUS_COMMIT_REMOTE_APPLY:
+			SyncRepWaitMode = SYNC_REP_WAIT_APPLY;
+			break;
 		default:
 			SyncRepWaitMode = SYNC_REP_NO_WAIT;
 			break;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 7b36e02..c19842e 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -101,6 +101,7 @@ static uint32 recvOff = 0;
  */
 static volatile sig_atomic_t got_SIGHUP = false;
 static volatile sig_atomic_t got_SIGTERM = false;
+static volatile sig_atomic_t got_SIGUSR2 = false;
 
 /*
  * LogstreamResult indicates the byte positions that we have already
@@ -150,9 +151,27 @@ static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
 /* Signal handlers */
 static void WalRcvSigHupHandler(SIGNAL_ARGS);
 static void WalRcvSigUsr1Handler(SIGNAL_ARGS);
+static void WalRcvSigUsr2Handler(SIGNAL_ARGS);
 static void WalRcvShutdownHandler(SIGNAL_ARGS);
 static void WalRcvQuickDieHandler(SIGNAL_ARGS);
 
+static void WalRcvBlockSigUsr2(void)
+{
+	sigset_t mask;
+
+	sigemptyset(&mask);
+	sigaddset(&mask, SIGUSR2);
+	sigprocmask(SIG_BLOCK, &mask, NULL);
+}
+
+static void WalRcvUnblockSigUsr2(void)
+{
+	sigset_t mask;
+
+	sigemptyset(&mask);
+	sigaddset(&mask, SIGUSR2);
+	sigprocmask(SIG_UNBLOCK, &mask, NULL);
+}
 
 static void
 ProcessWalRcvInterrupts(void)
@@ -200,6 +219,7 @@ WalReceiverMain(void)
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz last_recv_timestamp;
 	bool		ping_sent;
+	bool		forceReply;
 
 	/*
 	 * WalRcv should be set up already (if we are a backend, we inherit this
@@ -268,7 +288,7 @@ WalReceiverMain(void)
 	pqsignal(SIGALRM, SIG_IGN);
 	pqsignal(SIGPIPE, SIG_IGN);
 	pqsignal(SIGUSR1, WalRcvSigUsr1Handler);
-	pqsignal(SIGUSR2, SIG_IGN);
+	pqsignal(SIGUSR2, WalRcvSigUsr2Handler);
 
 	/* Reset some signals that are accepted by postmaster but not here */
 	pqsignal(SIGCHLD, SIG_DFL);
@@ -299,6 +319,10 @@ WalReceiverMain(void)
 	/* Unblock signals (they were blocked when the postmaster forked us) */
 	PG_SETMASK(&UnBlockSig);
 
+	/* Block SIGUSR2 (we unblock it only during network waits). */
+	WalRcvBlockSigUsr2();
+	got_SIGUSR2 = false;
+
 	/* Establish the connection to the primary for XLOG streaming */
 	EnableWalRcvImmediateExit();
 	walrcv_connect(conninfo);
@@ -408,7 +432,9 @@ WalReceiverMain(void)
 				}
 
 				/* Wait a while for data to arrive */
+				WalRcvUnblockSigUsr2();
 				len = walrcv_receive(NAPTIME_PER_CYCLE, &buf);
+				WalRcvBlockSigUsr2();
 				if (len != 0)
 				{
 					/*
@@ -439,11 +465,21 @@ WalReceiverMain(void)
 							endofwal = true;
 							break;
 						}
+						WalRcvUnblockSigUsr2();
 						len = walrcv_receive(0, &buf);
+						WalRcvBlockSigUsr2();
+					}
+
+					if (got_SIGUSR2)
+					{
+						/* The recovery process asked us to force a reply. */
+						got_SIGUSR2 = false;
+						forceReply = true;
 					}
 
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(false, false);
+					XLogWalRcvSendReply(forceReply, false);
+					forceReply = false;
 
 					/*
 					 * If we've written some records, flush them to disk and
@@ -498,7 +534,14 @@ WalReceiverMain(void)
 						}
 					}
 
-					XLogWalRcvSendReply(requestReply, requestReply);
+					if (got_SIGUSR2)
+					{
+						/* The recovery process asked us to force a reply. */
+						got_SIGUSR2 = false;
+						forceReply = true;
+					}
+					XLogWalRcvSendReply(requestReply || forceReply, requestReply);
+					forceReply = false;
 					XLogWalRcvSendHSFeedback(false);
 				}
 			}
@@ -740,6 +783,13 @@ WalRcvSigUsr1Handler(SIGNAL_ARGS)
 	errno = save_errno;
 }
 
+/* SIGUSR2: used to receive wakeups from recovery */
+static void
+WalRcvSigUsr2Handler(SIGNAL_ARGS)
+{
+	got_SIGUSR2 = true;
+}
+
 /* SIGTERM: set flag for main loop, or shutdown immediately if safe */
 static void
 WalRcvShutdownHandler(SIGNAL_ARGS)
@@ -1222,6 +1272,22 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 }
 
 /*
+ * Wake up the walreceiver if it happens to be blocked in walrcv_receive,
+ * and tell it that a commit record has been applied.
+ *
+ * This is called by the startup process whenever interesting xlog records
+ * are applied, so that walreceiver can check if it needs to send an apply
+ * notification back to the master which may be waiting in a COMMIT with
+ * synchronous_commit = remote_apply.
+ */
+void
+WalRcvWakeup(void)
+{
+	if (WalRcv->pid != 0)
+		kill(WalRcv->pid, SIGUSR2);
+}
+
+/*
  * Return a string constant representing the state. This is used
  * in system functions and views, and should *not* be translated.
  */
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index edcafce..88c4624 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -345,12 +345,13 @@ static const struct config_enum_entry constraint_exclusion_options[] = {
 };
 
 /*
- * Although only "on", "off", "remote_write", and "local" are documented, we
- * accept all the likely variants of "on" and "off".
+ * Although only "on", "off", "remote_apply", "remote_write", and "local" are
+ * documented, we accept all the likely variants of "on" and "off".
  */
 static const struct config_enum_entry synchronous_commit_options[] = {
 	{"local", SYNCHRONOUS_COMMIT_LOCAL_FLUSH, false},
 	{"remote_write", SYNCHRONOUS_COMMIT_REMOTE_WRITE, false},
+	{"remote_apply", SYNCHRONOUS_COMMIT_REMOTE_APPLY, false},
 	{"on", SYNCHRONOUS_COMMIT_ON, false},
 	{"off", SYNCHRONOUS_COMMIT_OFF, false},
 	{"true", SYNCHRONOUS_COMMIT_ON, true},
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index ee3d378..085099c 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -177,7 +177,7 @@
 					# (change requires restart)
 #fsync = on				# turns forced synchronization on or off
 #synchronous_commit = on		# synchronization level;
-					# off, local, remote_write, or on
+					# off, local, remote_write, remote_apply, or on
 #wal_sync_method = fsync		# the default is the first option
 					# supported by the operating system:
 					#   open_datasync
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index ebeb582..ed8d22c 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -60,7 +60,9 @@ typedef enum
 	SYNCHRONOUS_COMMIT_LOCAL_FLUSH,		/* wait for local flush only */
 	SYNCHRONOUS_COMMIT_REMOTE_WRITE,	/* wait for local flush and remote
 										 * write */
-	SYNCHRONOUS_COMMIT_REMOTE_FLUSH		/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_FLUSH,	/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_APPLY		/* wait for local flush and remote
+										 * apply */
 }	SyncCommitLevel;
 
 /* Define the default setting for synchonous_commit */
@@ -144,10 +146,13 @@ typedef void (*SubXactCallback) (SubXactEvent event, SubTransactionId mySubid,
  * EOXact... routines which run at the end of the original transaction
  * completion.
  */
+#define XACT_COMPLETION_SYNC_APPLY_FEEDBACK		(1U << 29)
 #define XACT_COMPLETION_UPDATE_RELCACHE_FILE	(1U << 30)
 #define XACT_COMPLETION_FORCE_SYNC_COMMIT		(1U << 31)
 
 /* Access macros for above flags */
+#define XactCompletionSyncApplyFeedback(xinfo) \
+	(!!(xinfo & XACT_COMPLETION_SYNC_APPLY_FEEDBACK))
 #define XactCompletionRelcacheInitFileInval(xinfo) \
 	(!!(xinfo & XACT_COMPLETION_UPDATE_RELCACHE_FILE))
 #define XactCompletionForceSyncCommit(xinfo) \
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index ecd30ce..68e20e4 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -268,6 +268,8 @@ extern bool CheckPromoteSignal(void);
 extern void WakeupRecovery(void);
 extern void SetWalWriterSleeping(bool sleeping);
 
+extern void XLogRequestWalReceiverReply(void);
+
 extern void assign_max_wal_size(int newval, void *extra);
 extern void assign_checkpoint_completion_target(double newval, void *extra);
 
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index 96e059b..28b68f6 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -23,8 +23,9 @@
 #define SYNC_REP_NO_WAIT		-1
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
+#define SYNC_REP_WAIT_APPLY		2
 
-#define NUM_SYNC_REP_WAIT_MODE	2
+#define NUM_SYNC_REP_WAIT_MODE	3
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 6eacb09..3294df9 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -162,5 +162,6 @@ extern void RequestXLogStreaming(TimeLineID tli, XLogRecPtr recptr,
 extern XLogRecPtr GetWalRcvWriteRecPtr(XLogRecPtr *latestChunkStart, TimeLineID *receiveTLI);
 extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
+extern void WalRcvWakeup(void);
 
 #endif   /* _WALRECEIVER_H */
0002-replay-lag-v4.patchapplication/octet-stream; name=0002-replay-lag-v4.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index ec5328e..ac67f86 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1202,6 +1202,12 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
       standby server</entry>
     </row>
     <row>
+     <entry><structfield>replay_lag</></entry>
+     <entry><type>interval</></entry>
+     <entry>Estimated time taken for recent WAL records to be replayed on this
+      standby server</entry>
+    </row>
+    <row>
      <entry><structfield>sync_priority</></entry>
      <entry><type>integer</></entry>
      <entry>Priority of this standby server for being chosen as the
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 002bf50..60375e2 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -5473,6 +5473,12 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 		XLogFlush(lsn);
 
 	/*
+	 * Record the primary's timestamp for the commit record, so it can be used
+	 * for tracking replay lag.
+	 */
+	SetXLogReplayTimestamp(parsed->xact_time);
+
+	/*
 	 * If asked by the primary (because someone is waiting for a synchronous
 	 * commit = remote_apply), we will need to ask walreceiver to send a
 	 * reply immediately.
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 5744429..5abfc61 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -81,6 +81,8 @@ extern uint32 bootstrap_data_checksum_version;
 #define PROMOTE_SIGNAL_FILE		"promote"
 #define FALLBACK_PROMOTE_SIGNAL_FILE "fallback_promote"
 
+/* Size of the circular buffer of timestamped LSNs. */
+#define MAX_TIMESTAMPED_LSNS 8192
 
 /* User-settable parameters */
 int			max_wal_size = 64;	/* 1 GB */
@@ -363,6 +365,13 @@ static bool doRequestWalReceiverReply;
  */
 static XLogRecPtr RedoStartLSN = InvalidXLogRecPtr;
 
+/*
+ * LastReplayedTimestamp can be set by redo handlers when they apply a record
+ * that carries a timestamp, by calling SetXLogReplayedTimestamp.  The xlog
+ * apply loop can then update the value in shared memory.
+ */
+static TimestampTz LastReplayedTimestamp = 0;
+
 /*----------
  * Shared-memory data structures for XLOG control
  *
@@ -637,6 +646,21 @@ typedef struct XLogCtlData
 	/* current effective recovery target timeline */
 	TimeLineID	RecoveryTargetTLI;
 
+	/* timestamp from the most recently applied record associated with a timestamp. */
+	TimestampTz lastReplayedTimestamp;
+
+	/*
+	 * We maintain a circular buffer of LSNs and associated timestamps.
+	 * Walreceiver writes into it using information from timestamps, and the
+	 * startup recovery process reads from it and notifies walreceiver when
+	 * LSNs are replayed so that the timestamps can eventually be fed back to
+	 * the upstream server, to track lag.
+	 */
+	Index			timestampedLsnRead;
+	Index			timestampedLsnWrite;
+	XLogRecPtr		timestampedLsn[MAX_TIMESTAMPED_LSNS];
+	TimestampTz		timestampedLsnTime[MAX_TIMESTAMPED_LSNS];
+
 	/*
 	 * timestamp of when we started replaying the current chunk of WAL data,
 	 * only relevant for replication or archive recovery
@@ -6877,20 +6901,51 @@ StartupXLOG(void)
 				error_context_stack = errcallback.previous;
 
 				/*
-				 * Update lastReplayedEndRecPtr after this record has been
-				 * successfully replayed.
+				 * Update lastReplayedEndRecPtr and lastReplayedTimestamp
+				 * after this record has been successfully replayed.
 				 */
 				SpinLockAcquire(&XLogCtl->info_lck);
 				XLogCtl->lastReplayedEndRecPtr = EndRecPtr;
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
+				if (LastReplayedTimestamp != 0)
+				{
+					/* If replaying a record produced a timestamp, use that. */
+					XLogCtl->lastReplayedTimestamp = LastReplayedTimestamp;
+					LastReplayedTimestamp = 0;
+				}
+				else
+				{
+					/*
+					 * If we have applied LSNs associated with timestamps
+					 * received by walreceiver, then use the recorded
+					 * timestamp.  We consume from the read end of the
+					 * circular buffer.
+					 */
+					while (XLogCtl->timestampedLsnRead !=
+						   XLogCtl->timestampedLsnWrite &&
+						   XLogCtl->timestampedLsn[XLogCtl->timestampedLsnRead]
+						   <= EndRecPtr)
+					{
+						if (XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead] >
+							XLogCtl->lastReplayedTimestamp)
+						{
+							XLogCtl->lastReplayedTimestamp =
+								XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead];
+							doRequestWalReceiverReply = true;
+						}
+						XLogCtl->timestampedLsnRead =
+							(XLogCtl->timestampedLsnRead + 1) % MAX_TIMESTAMPED_LSNS;
+					}
+				}
 				SpinLockRelease(&XLogCtl->info_lck);
 
 				/*
 				 * If rm_redo reported that it applied a commit record that
 				 * the master is waiting for by calling
-				 * XLogRequestWalReceiverReply, then we wake up the receiver
-				 * so that it notices the updated lastReplayedEndRecPtr and
-				 * sends a reply to the master.
+				 * XLogRequestWalReceiverReply, or we encountered a WAL
+				 * location that was associated with a timestamp above, then
+				 * we wake up the receiver so that it notices the updated
+				 * lastReplayedEndRecPtr and sends a reply to the master.
 				 */
 				if (doRequestWalReceiverReply)
 				{
@@ -11625,3 +11680,91 @@ XLogRequestWalReceiverReply(void)
 {
 	doRequestWalReceiverReply = true;
 }
+
+/*
+ * Record the timestamp that is associated with a WAL position.
+ *
+ * This is called by walreceiver on standby servers when keepalive messages
+ * arrive, using timestamps generated on the primary server.  The timestamp
+ * will be sent back to the primary server when the standby had applied this
+ * WAL position.  The primary can use the elapsed time to estimate the replay
+ * lag.
+ */
+void
+SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn)
+{
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn == XLogCtl->lastReplayedEndRecPtr)
+	{
+		/*
+		 * That is the last replayed LSN: we are fully replayed, so we can
+		 * update the replay timestamp immediately.
+		 */
+		XLogCtl->lastReplayedTimestamp = timestamp;
+	}
+	else
+	{
+		/*
+		 * There is WAL still to be applied.  We will associate the timestamp
+		 * with this WAL position and wait for it to be replayed.  We add it
+		 * at the 'write' end of the circular buffer of LSN/timestamp
+		 * mappings, which the replay loop will eventually read.
+		 */
+		Index w = XLogCtl->timestampedLsnWrite;
+		Index r = XLogCtl->timestampedLsnRead;
+
+		XLogCtl->timestampedLsn[w] = lsn;
+		XLogCtl->timestampedLsnTime[w] = timestamp;
+
+		/* Advance the write point. */
+		w = (w + 1) % MAX_TIMESTAMPED_LSNS;
+		XLogCtl->timestampedLsnWrite = w;
+		if (w == r)
+		{
+			/*
+			 * The buffer is full.  Advance the read point (throwing away
+			 * oldest values; we will begin to overestimate replay lag, until
+			 * lag decreases to a size our buffer can manage, or the next
+			 * commit record is replayed).
+			 */
+			r = (r + 1) % MAX_TIMESTAMPED_LSNS;
+			XLogCtl->timestampedLsnRead = r;
+		}
+	}
+	SpinLockRelease(&XLogCtl->info_lck);
+}
+
+/*
+ * Set the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary.  This can be called by redo handlers that have
+ * an appropriate timestamp (currently only commit records).  Updating the
+ * shared memory value is deferred until after the redo handler returns.
+ */
+void
+SetXLogReplayTimestamp(TimestampTz timestamp)
+{
+	LastReplayedTimestamp = timestamp;
+}
+
+/*
+ * Get the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary, and also the most recently applied LSN.  (Note
+ * that the timestamp and the LSN don't necessarily relate to the same
+ * record.)
+ *
+ * This is similar to GetLatestXTime, except that it is not only advanced by
+ * commit records (see SetXLogReplayTimestampAtLsn).
+ */
+TimestampTz
+GetXLogReplayTimestamp(XLogRecPtr *lsn)
+{
+	TimestampTz result;
+
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn)
+		*lsn = XLogCtl->lastReplayedEndRecPtr;
+	result = XLogCtl->lastReplayedTimestamp;
+	SpinLockRelease(&XLogCtl->info_lck);
+
+	return result;
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 84aa061..1b24fe5 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -662,6 +662,7 @@ CREATE VIEW pg_stat_replication AS
             W.write_location,
             W.flush_location,
             W.replay_location,
+	    W.replay_lag,
             W.sync_priority,
             W.sync_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index c19842e..d061dfe 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -85,6 +85,8 @@ walrcv_disconnect_type walrcv_disconnect = NULL;
 
 #define NAPTIME_PER_CYCLE 100	/* max sleep time between cycles (100ms) */
 
+#define MIN_TIME_BETWEEN_TIMESTAMPED_LSNS 1000 /* 1s */
+
 /*
  * These variables are used similarly to openLogFile/SegNo/Off,
  * but for walreceiver to write the XLOG. recvFileTLI is the TimeLineID
@@ -103,6 +105,8 @@ static volatile sig_atomic_t got_SIGHUP = false;
 static volatile sig_atomic_t got_SIGTERM = false;
 static volatile sig_atomic_t got_SIGUSR2 = false;
 
+static bool reply_sent_on_recovery_activity = false;
+
 /*
  * LogstreamResult indicates the byte positions that we have already
  * written/fsynced.
@@ -144,7 +148,7 @@ static void WalRcvDie(int code, Datum arg);
 static void XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len);
 static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
-static void XLogWalRcvSendReply(bool force, bool requestReply);
+static void XLogWalRcvSendReply(bool force, bool requestReply, bool includeApplyTimestamp);
 static void XLogWalRcvSendHSFeedback(bool immed);
 static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
 
@@ -219,7 +223,7 @@ WalReceiverMain(void)
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz last_recv_timestamp;
 	bool		ping_sent;
-	bool		forceReply;
+	bool		timestampedWalApplied;
 
 	/*
 	 * WalRcv should be set up already (if we are a backend, we inherit this
@@ -472,14 +476,15 @@ WalReceiverMain(void)
 
 					if (got_SIGUSR2)
 					{
-						/* The recovery process asked us to force a reply. */
+						/* The recovery process asked us to report an applied timestamp. */
 						got_SIGUSR2 = false;
-						forceReply = true;
+						timestampedWalApplied = true;
+						reply_sent_on_recovery_activity = true;
 					}
 
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(forceReply, false);
-					forceReply = false;
+					XLogWalRcvSendReply(timestampedWalApplied, false, timestampedWalApplied);
+					timestampedWalApplied = false;
 
 					/*
 					 * If we've written some records, flush them to disk and
@@ -536,12 +541,14 @@ WalReceiverMain(void)
 
 					if (got_SIGUSR2)
 					{
-						/* The recovery process asked us to force a reply. */
+						/* The recovery process asked us to report an apply timestamp. */
 						got_SIGUSR2 = false;
-						forceReply = true;
+						timestampedWalApplied = true;
+						reply_sent_on_recovery_activity = true;
 					}
-					XLogWalRcvSendReply(requestReply || forceReply, requestReply);
-					forceReply = false;
+					XLogWalRcvSendReply(requestReply || timestampedWalApplied, requestReply,
+										timestampedWalApplied);
+					timestampedWalApplied = false;
 					XLogWalRcvSendHSFeedback(false);
 				}
 			}
@@ -879,6 +886,8 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 			}
 		case 'k':				/* Keepalive */
 			{
+				bool reportApplyTimestamp = false;
+
 				/* copy message to StringInfo */
 				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
 				if (len != hdrlen)
@@ -895,9 +904,22 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 
 				ProcessWalSndrMessage(walEnd, sendTime);
 
-				/* If the primary requested a reply, send one immediately */
-				if (replyRequested)
-					XLogWalRcvSendReply(true, false);
+				/*
+				 * If no apply timestamps have been sent at the request of the
+				 * recovery process since we last received a keepalive, then
+				 * we will send one now.  This allows us to feed back
+				 * timestamps in response to pings if we are idle or if the
+				 * recovery process is somehow blocked, but we don't want to
+				 * do that if it's actively applying and periodically waking
+				 * us up.
+				 */
+				if (!reply_sent_on_recovery_activity)
+					reportApplyTimestamp = true;
+				reply_sent_on_recovery_activity = false;
+
+				/* If the primary requested a reply, send one immediately. */
+				if (replyRequested || reportApplyTimestamp)
+					XLogWalRcvSendReply(true, false, reportApplyTimestamp);
 				break;
 			}
 		default:
@@ -1060,7 +1082,7 @@ XLogWalRcvFlush(bool dying)
 		/* Also let the master know that we made some progress */
 		if (!dying)
 		{
-			XLogWalRcvSendReply(false, false);
+			XLogWalRcvSendReply(false, false, false);
 			XLogWalRcvSendHSFeedback(false);
 		}
 	}
@@ -1078,15 +1100,20 @@ XLogWalRcvFlush(bool dying)
  * If 'requestReply' is true, requests the server to reply immediately upon
  * receiving this message. This is used for heartbearts, when approaching
  * wal_receiver_timeout.
+ *
+ * If 'reportApplyTimestamp' is true, the latest apply timestamp is included.
+ * This is set to true only when this function is called after the recovery
+ * process has replayed a record with an associated timestamp.
  */
 static void
-XLogWalRcvSendReply(bool force, bool requestReply)
+XLogWalRcvSendReply(bool force, bool requestReply, bool reportApplyTimestamp)
 {
 	static XLogRecPtr writePtr = 0;
 	static XLogRecPtr flushPtr = 0;
 	XLogRecPtr	applyPtr;
 	static TimestampTz sendTime = 0;
 	TimestampTz now;
+	TimestampTz applyTimestamp = 0;
 
 	/*
 	 * If the user doesn't want status to be reported to the master, be sure
@@ -1102,10 +1129,8 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	 * We can compare the write and flush positions to the last message we
 	 * sent without taking any lock, but the apply position requires a spin
 	 * lock, so we don't check that unless something else has changed or 10
-	 * seconds have passed.  This means that the apply log position will
-	 * appear, from the master's point of view, to lag slightly, but since
-	 * this is only for reporting purposes and only on idle systems, that's
-	 * probably OK.
+	 * seconds have passed, or the force flag has been set (which happens when
+	 * apply feedback has been requested by the primary).
 	 */
 	if (!force
 		&& writePtr == LogstreamResult.Write
@@ -1118,7 +1143,10 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	/* Construct a new message */
 	writePtr = LogstreamResult.Write;
 	flushPtr = LogstreamResult.Flush;
-	applyPtr = GetXLogReplayRecPtr(NULL);
+	if (reportApplyTimestamp)
+		applyTimestamp = GetXLogReplayTimestamp(&applyPtr);
+	else
+		applyPtr = GetXLogReplayRecPtr(NULL);
 
 	resetStringInfo(&reply_message);
 	pq_sendbyte(&reply_message, 'r');
@@ -1126,6 +1154,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	pq_sendint64(&reply_message, flushPtr);
 	pq_sendint64(&reply_message, applyPtr);
 	pq_sendint64(&reply_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&reply_message, TimestampTzToIntegerTimestamp(applyTimestamp));
 	pq_sendbyte(&reply_message, requestReply ? 1 : 0);
 
 	/* Send it */
@@ -1230,8 +1259,8 @@ static void
 ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 {
 	WalRcvData *walrcv = WalRcv;
-
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
+	static TimestampTz lastRecordedTimestamp = 0;
 
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
@@ -1242,6 +1271,18 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
 	SpinLockRelease(&walrcv->mutex);
 
+	/*
+	 * Remember primary's timestamp at this WAL location.  We throw away
+	 * samples if they are coming too fast because we don't want to fill up
+	 * the finite circular buffer and have to throw away older samples.
+	 */
+	if (lastRecordedTimestamp < TimestampTzPlusMilliseconds(sendTime,
+															-MIN_TIME_BETWEEN_TIMESTAMPED_LSNS))
+	{
+		SetXLogReplayTimestampAtLsn(sendTime, walEnd);
+		lastRecordedTimestamp = sendTime;
+	}
+
 	if (log_min_messages <= DEBUG2)
 	{
 		char	   *sendtime;
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index f98475c..16d7abc 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1545,15 +1545,29 @@ ProcessStandbyReplyMessage(void)
 	XLogRecPtr	writePtr,
 				flushPtr,
 				applyPtr;
+	int64		applyLagUs;
 	bool		replyRequested;
+	TimestampTz now = GetCurrentTimestamp();
+	TimestampTz applyTimestamp;
 
 	/* the caller already consumed the msgtype byte */
 	writePtr = pq_getmsgint64(&reply_message);
 	flushPtr = pq_getmsgint64(&reply_message);
 	applyPtr = pq_getmsgint64(&reply_message);
 	(void) pq_getmsgint64(&reply_message);		/* sendTime; not used ATM */
+	applyTimestamp = IntegerTimestampToTimestampTz(pq_getmsgint64(&reply_message));
 	replyRequested = pq_getmsgbyte(&reply_message);
 
+	/* Compute the apply lag in milliseconds. */
+	if (applyTimestamp == 0)
+		applyLagUs = -1;
+	else
+#ifdef HAVE_INT64_TIMESTAMP
+		applyLagUs = now - applyTimestamp;
+#else
+		applyLagUs = (now - applyTimestamp) * 1000000;
+#endif
+
 	elog(DEBUG2, "write %X/%X flush %X/%X apply %X/%X%s",
 		 (uint32) (writePtr >> 32), (uint32) writePtr,
 		 (uint32) (flushPtr >> 32), (uint32) flushPtr,
@@ -1575,6 +1589,8 @@ ProcessStandbyReplyMessage(void)
 		walsnd->write = writePtr;
 		walsnd->flush = flushPtr;
 		walsnd->apply = applyPtr;
+		if (applyLagUs >= 0)
+			walsnd->applyLagUs = applyLagUs;
 		SpinLockRelease(&walsnd->mutex);
 	}
 
@@ -2745,7 +2761,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	8
+#define PG_STAT_GET_WAL_SENDERS_COLS	9
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2793,6 +2809,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		XLogRecPtr	write;
 		XLogRecPtr	flush;
 		XLogRecPtr	apply;
+		int64		applyLagUs;
 		int			priority;
 		WalSndState state;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
@@ -2807,6 +2824,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
+		applyLagUs = walsnd->applyLagUs;
 		priority = walsnd->sync_standby_priority;
 		SpinLockRelease(&walsnd->mutex);
 
@@ -2841,6 +2859,23 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				nulls[5] = true;
 			values[5] = LSNGetDatum(apply);
 
+			if (applyLagUs < 0)
+				nulls[6] = true;
+			else
+			{
+				Interval *applyLagInterval = palloc(sizeof(Interval));
+
+				applyLagInterval->month = 0;
+				applyLagInterval->day = 0;
+#ifdef HAVE_INT64_TIMESTAMP
+				applyLagInterval->time = applyLagUs;
+#else
+				applyLagInterval->time = applyLagUs / 1000000.0;
+#endif
+				nulls[6] = false;
+				values[6] = IntervalPGetDatum(applyLagInterval);
+			}
+
 			/*
 			 * Treat a standby such as a pg_basebackup background process
 			 * which always returns an invalid flush location, as an
@@ -2848,18 +2883,18 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 			 */
 			priority = XLogRecPtrIsInvalid(walsnd->flush) ? 0 : priority;
 
-			values[6] = Int32GetDatum(priority);
+			values[7] = Int32GetDatum(priority);
 
 			/*
 			 * More easily understood version of standby state. This is purely
 			 * informational, not different from priority.
 			 */
 			if (priority == 0)
-				values[7] = CStringGetTextDatum("async");
+				values[8] = CStringGetTextDatum("async");
 			else if (walsnd == sync_standby)
-				values[7] = CStringGetTextDatum("sync");
+				values[8] = CStringGetTextDatum("sync");
 			else
-				values[7] = CStringGetTextDatum("potential");
+				values[8] = CStringGetTextDatum("potential");
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
diff --git a/src/backend/utils/adt/timestamp.c b/src/backend/utils/adt/timestamp.c
index c4f556a..2032f61 100644
--- a/src/backend/utils/adt/timestamp.c
+++ b/src/backend/utils/adt/timestamp.c
@@ -1611,6 +1611,20 @@ IntegerTimestampToTimestampTz(int64 timestamp)
 #endif
 
 /*
+ * TimestampTzToIntegerTimestamp -- convert a native timestamp to int64 format
+ *
+ * When compiled with --enable-integer-datetimes, this is implemented as a
+ * no-op macro.
+ */
+#ifndef HAVE_INT64_TIMESTAMP
+int64
+TimestampTzToIntegerTimestamp(TimestampTz timestamp)
+{
+	return timestamp * 1000000;
+}
+#endif
+
+/*
  * TimestampDifference -- convert the difference between two timestamps
  *		into integer seconds and microseconds
  *
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 68e20e4..efb9719 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -236,6 +236,9 @@ extern void GetXLogReceiptTime(TimestampTz *rtime, bool *fromStream);
 extern XLogRecPtr GetXLogReplayRecPtr(TimeLineID *replayTLI);
 extern XLogRecPtr GetXLogInsertRecPtr(void);
 extern XLogRecPtr GetXLogWriteRecPtr(void);
+extern void SetXLogReplayTimestamp(TimestampTz timestamp);
+extern void SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn);
+extern TimestampTz GetXLogReplayTimestamp(XLogRecPtr *lsn);
 extern bool RecoveryIsPaused(void);
 extern void SetRecoveryPause(bool recoveryPause);
 extern TimestampTz GetLatestXTime(void);
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 5c71bce..b42fc68 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2712,7 +2712,7 @@ DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f
 DESCR("statistics: information about currently active backends");
 DATA(insert OID = 3318 (  pg_stat_get_progress_info           PGNSP PGUID 12 1 100 0 0 f f f f f t s r 1 0 2249 "25" "{25,23,26,26,20,20,20,20,20,20,20,20,20,20}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{cmdtype,pid,datid,relid,param1,param2,param3,param4,param5,param6,param7,param8,param9,param10}" _null_ _null_ pg_stat_get_progress_info _null_ _null_ _null_ ));
 DESCR("statistics: information about progress of backends running maintenance command");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,23,25}" "{o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25}" "{o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 7794aa5..4de43e8 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -46,6 +46,7 @@ typedef struct WalSnd
 	XLogRecPtr	write;
 	XLogRecPtr	flush;
 	XLogRecPtr	apply;
+	int64		applyLagUs;
 
 	/* Protects shared variables shown above. */
 	slock_t		mutex;
diff --git a/src/include/utils/timestamp.h b/src/include/utils/timestamp.h
index fbead3a..297e151 100644
--- a/src/include/utils/timestamp.h
+++ b/src/include/utils/timestamp.h
@@ -227,9 +227,11 @@ extern bool TimestampDifferenceExceeds(TimestampTz start_time,
 #ifndef HAVE_INT64_TIMESTAMP
 extern int64 GetCurrentIntegerTimestamp(void);
 extern TimestampTz IntegerTimestampToTimestampTz(int64 timestamp);
+extern int64 TimestampTzToIntegerTimestamp(TimestampTz timestamp);
 #else
 #define GetCurrentIntegerTimestamp()	GetCurrentTimestamp()
 #define IntegerTimestampToTimestampTz(timestamp) (timestamp)
+#define TimestampTzToIntegerTimestamp(timestamp) (timestamp)
 #endif
 
 extern TimestampTz time_t_to_timestamptz(pg_time_t tm);
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 22ea06c..f02a60c 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1761,11 +1761,12 @@ pg_stat_replication| SELECT s.pid,
     w.write_location,
     w.flush_location,
     w.replay_location,
+    w.replay_lag,
     w.sync_priority,
     w.sync_state
    FROM pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, wait_event_type, wait_event, xact_start, query_start, backend_start, state_change, client_addr, client_hostname, client_port, backend_xid, backend_xmin, ssl, sslversion, sslcipher, sslbits, sslcompression, sslclientdn),
     pg_authid u,
-    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, sync_priority, sync_state)
+    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state)
   WHERE ((s.usesysid = u.oid) AND (s.pid = w.pid));
 pg_stat_ssl| SELECT s.pid,
     s.ssl,
0003-refactor-syncrep-exit-v4.patchapplication/octet-stream; name=0003-refactor-syncrep-exit-v4.patchDownload
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 4565348..bbc28a7 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -83,6 +83,64 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * ===========================================================
  */
 
+static bool
+SyncRepCheckEarlyExit(void)
+{
+	/*
+	 * If a wait for synchronous replication is pending, we can neither
+	 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
+	 * lead the client to believe that the transaction aborted, which
+	 * is not true: it's already committed locally. The former is no good
+	 * either: the client has requested synchronous replication, and is
+	 * entitled to assume that an acknowledged commit is also replicated,
+	 * which might not be true. So in this case we issue a WARNING (which
+	 * some clients may be able to interpret) and shut off further output.
+	 * We do NOT reset ProcDiePending, so that the process will die after
+	 * the commit is cleaned up.
+	 */
+	if (ProcDiePending)
+	{
+		ereport(WARNING,
+				(errcode(ERRCODE_ADMIN_SHUTDOWN),
+				 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * It's unclear what to do if a query cancel interrupt arrives.  We
+	 * can't actually abort at this point, but ignoring the interrupt
+	 * altogether is not helpful, so we just terminate the wait with a
+	 * suitable warning.
+	 */
+	if (QueryCancelPending)
+	{
+		QueryCancelPending = false;
+		ereport(WARNING,
+				(errmsg("canceling wait for synchronous replication due to user request"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * If the postmaster dies, we'll probably never get an
+	 * acknowledgement, because all the wal sender processes will exit. So
+	 * just bail out.
+	 */
+	if (!PostmasterIsAlive())
+	{
+		ProcDiePending = true;
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	return false;
+}
+
 /*
  * Wait for synchronous replication, if requested by user.
  *
@@ -180,57 +238,9 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 		if (syncRepState == SYNC_REP_WAIT_COMPLETE)
 			break;
 
-		/*
-		 * If a wait for synchronous replication is pending, we can neither
-		 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
-		 * lead the client to believe that the transaction aborted, which
-		 * is not true: it's already committed locally. The former is no good
-		 * either: the client has requested synchronous replication, and is
-		 * entitled to assume that an acknowledged commit is also replicated,
-		 * which might not be true. So in this case we issue a WARNING (which
-		 * some clients may be able to interpret) and shut off further output.
-		 * We do NOT reset ProcDiePending, so that the process will die after
-		 * the commit is cleaned up.
-		 */
-		if (ProcDiePending)
-		{
-			ereport(WARNING,
-					(errcode(ERRCODE_ADMIN_SHUTDOWN),
-					 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit())
 			break;
-		}
-
-		/*
-		 * It's unclear what to do if a query cancel interrupt arrives.  We
-		 * can't actually abort at this point, but ignoring the interrupt
-		 * altogether is not helpful, so we just terminate the wait with a
-		 * suitable warning.
-		 */
-		if (QueryCancelPending)
-		{
-			QueryCancelPending = false;
-			ereport(WARNING,
-					(errmsg("canceling wait for synchronous replication due to user request"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * If the postmaster dies, we'll probably never get an
-		 * acknowledgement, because all the wal sender processes will exit. So
-		 * just bail out.
-		 */
-		if (!PostmasterIsAlive())
-		{
-			ProcDiePending = true;
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
-			break;
-		}
 
 		/*
 		 * Wait on latch.  Any condition that should wake us up will set the
0004-causal-reads-v4.patchapplication/octet-stream; name=0004-causal-reads-v4.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index e8e5d5f..31e3cd6 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -2755,6 +2755,35 @@ include_dir 'conf.d'
      across the cluster without problems if that is required.
     </para>
 
+    <sect2 id="runtime-config-replication-all">
+     <title>All Servers</title>
+     <para>
+      These parameters can be set on the primary or any standby.
+     </para>
+     <variablelist>
+      <varlistentry id="guc-causal-reads" xreflabel="causal_reads">
+       <term><varname>causal_reads</varname> (<type>boolean</type>)
+       <indexterm>
+        <primary><varname>causal_reads</> configuration parameter</primary>
+       </indexterm>
+       </term>
+       <listitem>
+        <para>
+         Enables causal consistency between transactions run on different
+         servers.  A transaction that is run on a standby
+         with <varname>causal_reads</> set to <literal>on</> is guaranteed
+         either to see the effects of all completed transactions run on the
+         primary with the setting on, or to receive an error "standby is not
+         available for causal reads".  Note that both transactions involved in
+         a causal dependency (a write on the primary followed by a read on any
+         server which must see the write) must be run with the setting on.
+         See <xref linkend="causal-reads"> for more details.
+        </para>
+       </listitem>
+      </varlistentry>
+     </variablelist>     
+    </sect2>
+
     <sect2 id="runtime-config-replication-sender">
      <title>Sending Server(s)</title>
 
@@ -2986,6 +3015,48 @@ include_dir 'conf.d'
       </listitem>
      </varlistentry>
 
+     <varlistentry>
+      <term><varname>causal_reads_timeout</varname> (<type>integer</type>)
+       <indexterm>
+        <primary><varname>causal_reads_timeout</> configuration parameter</primary>
+       </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies the maximum replay lag the primary will tolerate from a
+        standby before dropping it from the set of standbys available for
+        causal reads.
+       </para>
+       <para>
+        This setting is also used to control the <firstterm>leases</> used to
+        maintain the causal reads guarantee.  It must be set to a value which
+        is at least 4 times the maximum possible difference in system clocks
+        between the primary and standby servers, as described
+        in <xref linkend="causal-reads">.
+       </para>
+      </listitem>
+     </varlistentry>
+
+     <varlistentry id="guc-causal-reads-standby-names" xreflabel="causal-reads-standby-names">
+      <term><varname>causal_reads_standby_names</varname> (<type>string</type>)
+      <indexterm>
+       <primary><varname>causal_reads_standby_names</> configuration parameter</primary>
+      </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies a comma-separated list of standby names that can support
+        <firstterm>causal reads</>, as described in
+        <xref linkend="causal-reads">.  Follows the same convention
+        as <link linkend="guc-synchronous-standby-names"><literal>synchronous_standby_name</></>.
+        The default is <literal>*</>, matching all standbys.
+       </para>
+       <para>
+        This setting has no effect if <varname>causal_reads_timeout</> is not set.
+       </para>
+      </listitem>
+     </varlistentry>
+
      </variablelist>
     </sect2>
 
diff --git a/doc/src/sgml/high-availability.sgml b/doc/src/sgml/high-availability.sgml
index 2600fba..a01df0d 100644
--- a/doc/src/sgml/high-availability.sgml
+++ b/doc/src/sgml/high-availability.sgml
@@ -1115,6 +1115,9 @@ primary_slot_name = 'node_a_slot'
     that it has replayed the transaction, making it visible to user queries.
     In simple cases, this allows for load balancing with causal consistency
     on a single hot standby.
+    (See also
+    <xref linkend="causal-reads"> which deals with multiple standbys and
+    standby failure.)
    </para>
 
    <para>
@@ -1233,6 +1236,119 @@ primary_slot_name = 'node_a_slot'
    </sect3>
   </sect2>
 
+  <sect2 id="causal-reads">
+   <title>Causal reads</title>
+   <indexterm>
+    <primary>causal reads</primary>
+    <secondary>in standby</secondary>
+   </indexterm>
+
+   <para>
+    The causal reads feature allows read-only queries to run on hot standby
+    servers without exposing stale data to the client, providing a form of
+    causal consistency.  Transactions can run on any standby with the
+    following guarantee about the visibility of preceding transactions: If you
+    set <varname>causal_reads</> to <literal>on</> in any pair of consecutive
+    transactions tx1, tx2 where tx2 begins after tx1 successfully returns,
+    then tx2 will either see tx1 or fail with a new error "standby is not
+    available for causal reads", no matter which server it runs on.  Although
+    the guarantee is expressed in terms of two individual transactions, the
+    GUC can also be set at session, role or system level to make the guarantee
+    generally, allowing for load balancing of applications that were not
+    designed with load balancing in mind.
+   </para>
+
+   <para>
+    In order to enable the feature, <varname>causal_reads_timeout</> must be
+    set to a non-zero value on the primary server.  The
+    GUC <varname>causal_reads_standby_names</> can be used to limit the set of
+    standbys that can join the dynamic set of causal reads standbys by
+    providing a comma-separated list of application names.  By default, all
+    standbys are candidates, if the feature is enabled.
+   </para>
+
+   <para>
+    The current set of servers that the primary considers to be available for
+    causal reads can be seen in
+    the <link linkend="monitoring-stats-views-table"> <literal>pg_stat_replication</></>
+    view.  Administrators, applications and load balancing middleware can use
+    this view to discover standbys that can currently handle causal reads
+    transactions without raising the error.  Since that information is only an
+    instantantaneous snapshot, clients should still be prepared for the error
+    to be raised at any time, and consider redirecting transactions to another
+    standby.
+   </para>
+
+   <para>
+    The advantages of the causal reads feature over simply
+    setting <varname>synchronous_commit</> to <literal>remote_apply</> are:
+    <orderedlist>
+      <listitem>
+       <para>
+        It allows the primary to wait for multiple standbys to replay
+        transactions.
+       </para>
+      </listitem>
+      <listitem>
+       <para>
+        It places a configurable limit on how much replay lag (and therefore
+        delay at commit time) the primary tolerates from standbys before it
+        drops them from the dynamic set of standbys it waits for.
+       </para>   
+      </listitem>
+      <listitem>
+       <para>
+        It upholds the causal reads guarantee during the transitions that
+        occur when new standbys are added or removed from the set of standbys,
+        including scenarios where contact has been lost between the primary
+        and standbys but the standby is still alive and running client
+        queries.
+       </para>
+      </listitem>
+    </orderedlist>
+   </para>
+
+   <para>
+    The protocol used to uphold the guarantee even in the case of network
+    failure depends on the system clocks of the primary and standby servers
+    being synchronized, with an allowance for a difference up to one quarter
+    of <varname>causal_reads_timeout</>.  For example,
+    if <varname>causal_reads_timeout</> is set to <literal>4s</>, then the
+    clocks must not be further than 1 second apart for the guarantee to be
+    upheld reliably during transitions.  The ubiquity of the Network Time
+    Protocol (NTP) on modern operating systems and availability of high
+    quality time servers makes it possible to choose a tolerance significantly
+    higher than the maximum expected clock difference.  An effort is
+    nevertheless made to detect and report misconfigured and faulty systems
+    with clock differences greater than the configured tolerance.
+   </para>
+
+   <note>
+    <para>
+     Current hardware clocks, NTP implementations and public time servers are
+     unlikely to allow the system clocks to differ more than tens or hundreds
+     of milliseconds, and systems synchronized with dedicated local time
+     servers may be considerably more accurate, but you should only consider
+     setting <varname>causal_reads_timeout</> below 4 seconds (allowing up to
+     1 second of clock difference) after researching your time synchronization
+     infrastructure thoroughly.
+    </para>  
+   </note>
+
+   <note>
+    <para>
+      While similar to synchronous replication in the sense that both involve
+      the primary server waiting for responses from standby servers, the
+      causal reads feature is not concerned with avoiding data loss.  A
+      primary configured for causal reads will drop all standbys that stop
+      responding or replay too slowly from the dynamic set that it waits for,
+      so you should consider configuring both synchronous replication and
+      causal reads if you need data loss avoidance guarantees and causal
+      consistency guarantees for load balancing.
+    </para>
+   </note>
+  </sect2>
+
   <sect2 id="continuous-archiving-in-standby">
    <title>Continuous archiving in standby</title>
 
@@ -1581,7 +1697,16 @@ if (!triggered)
     so there will be a measurable delay between primary and standby. Running the
     same query nearly simultaneously on both primary and standby might therefore
     return differing results. We say that data on the standby is
-    <firstterm>eventually consistent</firstterm> with the primary.  Once the
+    <firstterm>eventually consistent</firstterm> with the primary by default.
+    The data visible to a transaction running on a standby can be
+    made <firstterm>causally consistent</> with respect to a transaction that
+    has completed on the primary by setting <varname>causal_reads</>
+    to <literal>on</> in both transactions.  For more details,
+    see <xref linkend="causal-reads">.
+   </para>
+
+   <para>
+    Once the    
     commit record for a transaction is replayed on the standby, the changes
     made by that transaction will be visible to any new snapshots taken on
     the standby.  Snapshots may be taken at the start of each query or at the
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index ac67f86..5868b2a 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1218,6 +1218,17 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
      <entry><type>text</></entry>
      <entry>Synchronous state of this standby server</entry>
     </row>
+    <row>
+     <entry><structfield>causal_reads_state</></entry>
+     <entry><type>text</></entry>
+     <entry>Causal reads state of this standby server.  This field will be
+     non-null only if <varname>cause_reads_timeout</> is set.  If a standby is
+     in <literal>available</> state, then it can currently serve causal reads
+     queries.  If it is not replaying fast enough or not responding to
+     keepalive messages, it will be in <literal>unavailable</> state, and if
+     it is currently transitioning to availability it will be
+     in <literal>joining</> state for a short time.</entry>
+    </row>
    </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 80721e7..95c622d 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -2111,11 +2111,12 @@ RecordTransactionCommitPrepared(TransactionId xid,
 	END_CRIT_SECTION();
 
 	/*
-	 * Wait for synchronous replication, if required.
+	 * Wait for causal reads and synchronous replication, if required.
 	 *
 	 * Note that at this stage we have marked clog, but still show as running
 	 * in the procarray and continue to hold locks.
 	 */
+	CausalReadsWaitForLSN(recptr);
 	SyncRepWaitForLSN(recptr);
 }
 
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 60375e2..cb50166 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,10 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
+	{
+		CausalReadsWaitForLSN(XactLastRecEnd);
 		SyncRepWaitForLSN(XactLastRecEnd);
+	}
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5126,7 +5129,7 @@ XactLogCommitRecord(TimestampTz commit_time,
 	 * Check if the caller would like to ask standbys for immediate feedback
 	 * once this commit is applied.
 	*/
-	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY || causal_reads)
 		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
 
 	/*
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 1b24fe5..07f1da9 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -664,7 +664,8 @@ CREATE VIEW pg_stat_replication AS
             W.replay_location,
 	    W.replay_lag,
             W.sync_priority,
-            W.sync_state
+            W.sync_state,
+	    W.causal_reads_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
             pg_stat_get_wal_senders() AS W
     WHERE S.usesysid = U.oid AND
diff --git a/src/backend/replication/README.causal_reads b/src/backend/replication/README.causal_reads
new file mode 100644
index 0000000..1fddd62
--- /dev/null
+++ b/src/backend/replication/README.causal_reads
@@ -0,0 +1,193 @@
+The causal reads guarantee says: If you run any two consecutive
+transactions tx1, tx2 where tx1 completes before tx2 begins, with
+causal_reads set to "on" in both transactions, tx2 will see tx1 or
+raise an error to complain that it can't guarantee causal consistency,
+no matter which servers (primary or any standby) you run each
+transaction on.
+
+When both transactions run on the primary, the guarantee is trivially
+upheld.
+
+To deal with read-only physical streaming standbys, the primary keeps
+track of a set of standbys that it considers to be currently
+"available" for causal reads, and sends a stream of "leases" to those
+standbys granting them the right to handle causal reads transactions
+for a short time without any further communication with the primary.
+
+In general, the primary provides the guarantee by waiting for all of
+the "available" standbys to report that they have applied a
+transaction.  However, the set of available standbys is dynamic, and
+things get more complicated during state transitions.  There are two
+types of transitions to consider:
+
+1.  unavailable->joining->available
+
+Standbys start out as "unavailable".  If a standby is unavailable and
+is applying fast enough and matches causal_reads_standby_names, the
+primary transitions it to "available", but first it sets it to
+"joining" until it is sure that any transaction committed while it was
+unavailable has definitely been applied on the standby.  This closes a
+race that would otherwise exist if we moved directly to available
+state: tx1 might not wait for a given standby because it's
+unavailable, then a lease might be granted, and then tx2 might run a
+causal reads transaction without error but see stale data.  The
+joining state acts as an airlock: while in joining state, the primary
+waits for that standby to replay causal reads transactions in
+anticipation of the move to available, but it doesn't progress to
+available state and grant a lease to the standby until everything
+preceding joining state has also been applied.
+
+2.  available->unavailable
+
+If a standby is not applying fast enough or not responding to
+keepalive messages, then the primary kicks that standby out of the
+dynamic set of available standbys, that is, marks it as "unavailable".
+In order to make sure that the standby has started rejecting causal
+reads transactions, it needs to revoke the lease it most recently
+granted.  It does that by waiting for the lease to expire before
+allowing any causal reads commits to return.  (In future there could
+be a fast-path revocation message which waits for a serial-numbered
+acknowledgement to reduce waiting in the case where the standby is
+lagging but still reachable and responding).
+
+The rest of this document illustrates how clock skew affects the
+available->unavailable transition.
+
+The following 4 variables are derived from a single GUC, and these
+values will be used in the following illustrations:
+
+causal_reads_timeout = 4s
+lease_time           = 4s (= causal_reads_timeout)
+keepalive_time       = 2s (= lease_time / 2)
+max_clock_skew       = 1s (= lease_time / 4)
+
+Every keepalive_time, the primary transmits a lease that expires at
+local_clock_time + lease_time - max_clock_skew, shown in the following
+diagram as 't' for transmission time and '|' for expiry time.  If
+contact is lost with a standby, the primary will wait until sent_time
++ lease_time for the most recently granted lease to expire, shown on
+the following diagram 'x', to be sure that the standby's clock has
+reached the expiry time even if its clock differs by up to
+max_clock_skew.  In other words, the primary tells the standby that
+the expiry time is at one time, but it trusts that the standby will
+surely agree if it gives it some extra time.  The extra time is
+max_clock_skew.  If the clocks differ by more than max_clock_skew, all
+bets are off (but see below for attempt to detect obvious cases).
+
+0     1     2     3     4     5     6     7     8     9
+t-----------------|-----x
+            t-----------------|-----x
+                        t-----------------|-----x
+                                    t-----------------|...
+                                                t------...
+
+A standby whose clock is 2 seconds ahead of the primary's clock
+perceives gaps in the stream of leases, and will reject causal_reads
+transactions in those intervals.  The causal reads guarantee is
+upheld, but spurious errors are raised between leases, as a
+consequence of the clock skew being greater than max_clock_skew.  In
+the following diagram 'r' shows reception time, and the timeline along
+the top shows the standby's local clock time.
+
+2     3     4     5     6     7     8     9    10    11
+r-----|
+            r-----|
+                        r-----|
+                                    r-----|
+                                                r-----|
+
+If there were no network latency, a standby whose clock is exactly 1
+second ahead of the primary's clock would perceive the stream of
+leases as being replaced just in time, so there is no gap.  Since in
+reality the time of receipt is some time after the time of
+transmission due to network latency, if the standby's clock is exactly
+1 second behind, then there will be small network-latency-sized gaps
+before the next lease arrives, but still no correctness problem with
+respect to the causal reads guarantee.
+
+1     2     3     4     5     6     7     8     9    10
+r-----------|
+            r-----------|
+                        r-----------|
+                                    r-----------|
+                                                r------...
+
+A standby whose clock is perfectly in sync with the primary's
+perceives the stream of leases overlapping (this matches the primary's
+perception of the leases it sent):
+
+0     1     2     3     4     5     6     7     8     9
+r-----------------|
+            r-----------------|
+                        r-----------------|
+                                    r-----------------|
+                                                r------...
+
+A standby whose clock is exactly 1 second behind the primary's
+perceives the stream of leases as overlapping even more, but the time
+of expiry as judged by the standby is no later than the time the
+primary will wait for if required ('x').  That is, if contact is lost
+with the standby, the primary can still reliably hold up causal reads
+commits until the standby has started raising the error in
+causal_reads transactions.
+
+-1    0     1     2     3     4     5     6     7     8
+r-----------------------|
+            r-----------------------|
+                        r-----------------------|
+                                    r------------------...
+                                                r------...
+
+
+A standby whose clock is 2 seconds behind the primary's would perceive
+the stream of leases overlapping even more, and the primary would no
+longer be able to wait for a lease to expire if it wanted to revoke
+it.  But because the expiry time is after local_clock_time +
+lease_time, the standby can immediately see that its own clock must be
+more than 1 second behind the primary's, so it ignores the lease and
+logs a clock skew warning.  In the following diagram a lease expiry
+time that is obviously generated by a primary with a clock set too far
+in the future compared to the local clock is shown with a '!'.
+
+-2    -1    0     1     2     3     4     5     6     7
+r-----------------------------!
+            r-----------------------------!
+                        r-----------------------------!
+                                    r------------------...
+                                                r------...
+
+A danger window exists when the standby's clock is more than
+max_clock_skew behind the primary's clock, but not more than
+max_clock_skew + network latency time behind.  If the clock difference
+is in that range, then the algorithm presented above which is based on
+time of receipt cannot detect that the local clock is too far behind.
+The consequence of this problem could be as follows:
+
+1.  The standby loses contact with the primary due to a network fault.
+
+2.  The primary decides to drop the standby from the set of available
+    causal reads standbys due to lack of keepalive responses or
+    excessive lag, which necessitates holding up commits of causal
+    reads transactions until the most recently sent lease expires, in
+    the belief that the standby will definitely have started raising
+    the 'causal reads unavailable' error in causal reads transactions
+    by that time, if it is still alive and servicing requests.
+
+3.  The standby still has clients connected and running queries.
+
+4.  Due to clock skew in the problematic range, in the standby's
+    opinion the lease lasts slightly longer than the primary waits.
+
+5.  For a short window at most the duration of the network latency
+    time, clients running causal reads transactions are allowed to see
+    potentially stale data.
+
+For this reason we say that the causal reads guarantee only holds as
+long as the absolute difference between the system clocks of the
+machines is no more than max_clock_skew.  The theory is that NTP makes
+it possible to reason about the maximum possible clock difference
+between machines and choose a value that allows for a much larger
+difference.  However, we do make a best effort attempt to detect
+wildly divergent systems as described above, to catch the case of
+servers not running a correctly configured ntp daemon, or with a clock
+so far out of whack that ntp refuses to fix it.
\ No newline at end of file
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index bbc28a7..80502f3 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -57,6 +57,11 @@
 #include "utils/builtins.h"
 #include "utils/ps_status.h"
 
+/* GUC variables */
+int causal_reads_timeout;
+bool causal_reads;
+char *causal_reads_standby_names;
+
 /* User-settable parameters for sync rep */
 char	   *SyncRepStandbyNames;
 
@@ -69,7 +74,7 @@ static int	SyncRepWaitMode = SYNC_REP_NO_WAIT;
 
 static void SyncRepQueueInsert(int mode);
 static void SyncRepCancelWait(void);
-static int	SyncRepWakeQueue(bool all, int mode);
+static int	SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn);
 
 static int	SyncRepGetStandbyPriority(void);
 
@@ -142,6 +147,198 @@ SyncRepCheckEarlyExit(void)
 }
 
 /*
+ * Check if we can stop waiting for causal consistency.  We can stop waiting
+ * when the following conditions are met:
+ *
+ * 1.  All walsenders currently in 'joining' or 'available' state have
+ * applied the target LSN.
+ *
+ * 2.  Any stall periods caused by standbys dropping out of 'available' state
+ * have passed, so that we can be sure that their leases have expired and they
+ * have started rejecting causal reads transactions.
+ *
+ * The output parameter 'waitingFor' is set to the number of nodes we are
+ * currently waiting for.  The output parameters 'stallTimeMillis' is set to
+ * the number of milliseconds we need to wait for to observe any current
+ * commit stall.
+ *
+ * Returns true if commit can return control, because every standby has either
+ * applied the LSN or started rejecting causal_reads transactions.
+ */
+static bool
+CausalReadsCommitCanReturn(XLogRecPtr XactCommitLSN,
+						   int *waitingFor,
+						   long *stallTimeMillis)
+{
+	int i;
+	TimestampTz now;
+
+	/* Count how many joining/available nodes we are waiting for. */
+	*waitingFor = 0;
+	for (i = 0; i < max_wal_senders; ++i)
+	{
+		WalSnd *walsnd = &WalSndCtl->walsnds[i];
+
+		/*
+		 * Assuming atomic read of pid_t, we can check walsnd->pid without
+		 * acquiring the spinlock to avoid memory synchronization costs for
+		 * unused walsender slots.  We see a value that existed sometime at
+		 * least as recently as the last memory barrier.
+		 */
+		if (walsnd->pid != 0)
+		{
+			/*
+			 * We need to hold the spinlock to read LSNs, because we can't be
+			 * sure they can be read atomically.
+			 */
+			SpinLockAcquire(&walsnd->mutex);
+			if (walsnd->pid != 0 && walsnd->causal_reads_state >= WALSNDCRSTATE_JOINING)
+			{
+				if (walsnd->apply < XactCommitLSN)
+					++*waitingFor;
+			}
+			SpinLockRelease(&walsnd->mutex);
+		}
+	}
+
+	/* Check if there is a stall in progress that we need to observe. */
+	now = GetCurrentTimestamp();
+	LWLockAcquire(SyncRepLock, LW_SHARED);
+	if (WalSndCtl->stall_causal_reads_until > now)
+	{
+		long seconds;
+		int usecs;
+
+		/* Compute how long we have to wait, rounded up to nearest ms. */
+		TimestampDifference(now, WalSndCtl->stall_causal_reads_until,
+							&seconds, &usecs);
+		*stallTimeMillis = seconds * 1000 + (usecs + 999) / 1000;
+	}
+	else
+		*stallTimeMillis = 0;
+	LWLockRelease(SyncRepLock);
+
+	/* We are done if we are not waiting for any nodes or stalls. */
+	return *waitingFor == 0 && *stallTimeMillis == 0;
+}
+
+/*
+ * Wait for causal consistency in causal_reads mode, if requested by user.
+ */
+void
+CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN)
+{
+	long stallTimeMillis;
+	int waitingFor;
+	char *ps_display_buffer = NULL;
+
+	/* Leave if we aren't in causal_reads mode. */
+	if (!causal_reads)
+		return;
+
+	for (;;)
+	{
+		/* Reset latch before checking state. */
+		ResetLatch(MyLatch);
+
+		/*
+		 * Join the queue to be woken up if any causal reads joining/available
+		 * standby applies XactCommitLSN or the set of causal reads standbys
+		 * changes (if we aren't already in the queue).  We don't actually know
+		 * if we need to wait for any peers to reach the target LSN yet, but
+		 * we have to register just in case before checking the walsenders'
+		 * state to avoid a race condition that could occur if we did it after
+		 * calling CausalReadsCommitCanReturn.  (SyncRepWaitForLSN doesn't
+		 * have to do this because it can check the highest-seen LSN in
+		 * walsndctl->lsn[mode] which is protected by SyncRepLock, the same
+		 * lock as the queues.  We can't do that here, because there is no
+		 * single highest-seen LSN that is useful.  We must check
+		 * walsnd->apply for all relevant walsenders.  Therefore we must
+		 * register for notifications first, so that we can be notified via
+		 * our latch of any standby applying the LSN we're interested in after
+		 * we check but before we start waiting, or we could wait forever for
+		 * something that has already happened.)
+		 */
+		LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+		if (MyProc->syncRepState != SYNC_REP_WAITING)
+		{
+			MyProc->waitLSN = XactCommitLSN;
+			MyProc->syncRepState = SYNC_REP_WAITING;
+			SyncRepQueueInsert(SYNC_REP_WAIT_CAUSAL_READS);
+			Assert(SyncRepQueueIsOrderedByLSN(SYNC_REP_WAIT_CAUSAL_READS));
+		}
+		LWLockRelease(SyncRepLock);
+
+		/* Check if we're done. */
+		if (CausalReadsCommitCanReturn(XactCommitLSN, &waitingFor, &stallTimeMillis))
+		{
+			SyncRepCancelWait();
+			break;
+		}
+
+		Assert(waitingFor > 0 || stallTimeMillis > 0);
+
+		/* If we aren't actually waiting for any standbys, leave the queue. */
+		if (waitingFor == 0)
+			SyncRepCancelWait();
+
+		/* Update the ps title. */
+		if (update_process_title)
+		{
+			char buffer[80];
+
+			/* Remember the old value if this is our first update. */
+			if (ps_display_buffer == NULL)
+			{
+				int len;
+				const char *ps_display = get_ps_display(&len);
+
+				ps_display_buffer = palloc(len + 1);
+				memcpy(ps_display_buffer, ps_display, len);
+				ps_display_buffer[len] = '\0';
+			}
+
+			snprintf(buffer, sizeof(buffer),
+					 "waiting for %d peer(s) to apply %X/%X%s",
+					 waitingFor,
+					 (uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN,
+					 stallTimeMillis > 0 ? " (stalling)" : "");
+			set_ps_display(buffer, false);
+		}
+
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit()) /* Calls SyncRepCancelWait() if true. */
+			break;
+
+		/*
+		 * If are still waiting for peers, then we wait for any joining or
+		 * available peer to reach the LSN (or possibly stop being in one of
+		 * those states or go away).
+		 *
+		 * If not, there must be a non-zero stall time, so we wait for that to
+		 * elapse.
+		 */
+		if (waitingFor > 0)
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH, -1);
+		else
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH | WL_TIMEOUT,
+					  stallTimeMillis);
+	}
+
+	/* There is no way out of the loop that could leave us in the queue. */
+	Assert(SHMQueueIsDetached(&(MyProc->syncRepLinks)));
+	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
+	MyProc->waitLSN = 0;
+
+	/* Restore the ps display. */
+	if (ps_display_buffer != NULL)
+	{
+		set_ps_display(ps_display_buffer, false);
+		pfree(ps_display_buffer);
+	}
+}
+
+/*
  * Wait for synchronous replication, if requested by user.
  *
  * Initially backends start in state SYNC_REP_NOT_WAITING and then
@@ -413,6 +610,53 @@ SyncRepGetSynchronousStandby(void)
 }
 
 /*
+ * Check if the current WALSender process's application_name matches a name in
+ * causal_reads_standby_names (including '*' for wildcard).
+ */
+bool
+CausalReadsPotentialStandby(void)
+{
+	char *rawstring;
+	List	   *elemlist;
+	ListCell   *l;
+	bool		found = false;
+
+	/* If the feature is disable, then no. */
+	if (causal_reads_timeout == 0)
+		return false;
+
+	/* Need a modifiable copy of string */
+	rawstring = pstrdup(causal_reads_standby_names);
+
+	/* Parse string into list of identifiers */
+	if (!SplitIdentifierString(rawstring, ',', &elemlist))
+	{
+		/* syntax error in list */
+		pfree(rawstring);
+		list_free(elemlist);
+		/* GUC machinery will have already complained - no need to do again */
+		return 0;
+	}
+
+	foreach(l, elemlist)
+	{
+		char	   *standby_name = (char *) lfirst(l);
+
+		if (pg_strcasecmp(standby_name, application_name) == 0 ||
+			pg_strcasecmp(standby_name, "*") == 0)
+		{
+			found = true;
+			break;
+		}
+	}
+
+	pfree(rawstring);
+	list_free(elemlist);
+
+	return found;
+}
+
+/*
  * Update the LSNs on each queue based upon our latest state. This
  * implements a simple policy of first-valid-standby-releases-waiter.
  *
@@ -420,23 +664,27 @@ SyncRepGetSynchronousStandby(void)
  * perhaps also which information we store as well.
  */
 void
-SyncRepReleaseWaiters(void)
+SyncRepReleaseWaiters(bool walsender_cr_available_or_joining)
 {
 	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
 	int			numapply = 0;
+	int			numcausalreadsapply = 0;
+	bool		is_highest_priority_sync_standby;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
-	 * potential sync standbys then we have nothing to do. If we are still
-	 * starting up, still running base backup or the current flush position
-	 * is still invalid, then leave quickly also.
+	 * potential sync standbys and not in a state that causal_reads waits for,
+	 * then we have nothing to do. If we are still starting up, still running
+	 * base backup or the current flush position is still invalid, then leave
+	 * quickly also.
 	 */
-	if (MyWalSnd->sync_standby_priority == 0 ||
-		MyWalSnd->state < WALSNDSTATE_STREAMING ||
-		XLogRecPtrIsInvalid(MyWalSnd->flush))
+	if (!walsender_cr_available_or_joining &&
+		(MyWalSnd->sync_standby_priority == 0 ||
+		 MyWalSnd->state < WALSNDSTATE_STREAMING ||
+		 XLogRecPtrIsInvalid(MyWalSnd->flush)))
 		return;
 
 	/*
@@ -446,13 +694,19 @@ SyncRepReleaseWaiters(void)
 	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
 	syncWalSnd = SyncRepGetSynchronousStandby();
 
-	/* We should have found ourselves at least */
-	Assert(syncWalSnd != NULL);
+	/*
+	 * If we aren't managing the highest priority standby then make a note of
+	 * that so we can announce a takeover in the log if we ever get that job.
+	 */
+	is_highest_priority_sync_standby = syncWalSnd == MyWalSnd;
+	if (!is_highest_priority_sync_standby)
+		announce_next_takeover = true;
 
 	/*
-	 * If we aren't managing the highest priority standby then just leave.
+	 * If we aren't managing the highest priority standby or a standby in
+	 * causal reads 'joining' or 'available' state, then just leave.
 	 */
-	if (syncWalSnd != MyWalSnd)
+	if (!is_highest_priority_sync_standby && !walsender_cr_available_or_joining)
 	{
 		LWLockRelease(SyncRepLock);
 		announce_next_takeover = true;
@@ -461,24 +715,45 @@ SyncRepReleaseWaiters(void)
 
 	/*
 	 * Set the lsn first so that when we wake backends they will release up to
-	 * this location.
+	 * this location.  For the single-standby synchronous commit levels, we
+	 * only do this if we are the current synchronous standby and we are
+	 * advancing the LSN further than it has been advanced before, so that
+	 * SyncRepWaitForLSN can skip waiting in some cases.
 	 */
-	if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
-		numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
-		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+	if (is_highest_priority_sync_standby)
 	{
-		walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
-		numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY);
+		if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
+			numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE,
+										MyWalSnd->write);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
+			numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH,
+										MyWalSnd->flush);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+			numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY,
+										MyWalSnd->apply);
+		}
 	}
 
+	/*
+	 * For causal_reads, all walsenders currently in available or joining
+	 * state must reach the LSN on their own, and standbys will reach LSNs in
+	 * any order.  It doesn't make sense to keep the highest seen LSN in a
+	 * single walsndctl->lsn element.  (CausalReadsWaitForLSN has handling for
+	 * LSNs that have already been reached).
+	 */
+	if (walsender_cr_available_or_joining)
+		numcausalreadsapply =
+			SyncRepWakeQueue(false, SYNC_REP_WAIT_CAUSAL_READS,
+							 MyWalSnd->apply);
+
 	LWLockRelease(SyncRepLock);
 
 	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%x",
@@ -490,7 +765,7 @@ SyncRepReleaseWaiters(void)
 	 * If we are managing the highest priority standby, though we weren't
 	 * prior to this, then announce we are now the sync standby.
 	 */
-	if (announce_next_takeover)
+	if (is_highest_priority_sync_standby && announce_next_takeover)
 	{
 		announce_next_takeover = false;
 		ereport(LOG,
@@ -565,9 +840,8 @@ SyncRepGetStandbyPriority(void)
  * Must hold SyncRepLock.
  */
 static int
-SyncRepWakeQueue(bool all, int mode)
+SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn)
 {
-	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	PGPROC	   *proc = NULL;
 	PGPROC	   *thisproc = NULL;
 	int			numprocs = 0;
@@ -584,7 +858,7 @@ SyncRepWakeQueue(bool all, int mode)
 		/*
 		 * Assume the queue is ordered by LSN
 		 */
-		if (!all && walsndctl->lsn[mode] < proc->waitLSN)
+		if (!all && lsn < proc->waitLSN)
 			return numprocs;
 
 		/*
@@ -644,7 +918,7 @@ SyncRepUpdateSyncStandbysDefined(void)
 			int			i;
 
 			for (i = 0; i < NUM_SYNC_REP_WAIT_MODE; i++)
-				SyncRepWakeQueue(true, i);
+				SyncRepWakeQueue(true, i, InvalidXLogRecPtr);
 		}
 
 		/*
@@ -696,13 +970,31 @@ SyncRepQueueIsOrderedByLSN(int mode)
 #endif
 
 /*
+ * Make sure that CausalReadsWaitForLSN can't return until after the given
+ * lease expiry time has been reached.  In other words, revoke the lease.
+ *
+ * Wake up all backends waiting in CausalReadsWaitForLSN, because the set of
+ * available/joining peers has changed, and there is a new stall time they
+ * need to observe.
+ */
+void
+CausalReadsBeginStall(TimestampTz lease_expiry_time)
+{
+	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+	WalSndCtl->stall_causal_reads_until =
+		Max(WalSndCtl->stall_causal_reads_until, lease_expiry_time);
+	SyncRepWakeQueue(true, SYNC_REP_WAIT_CAUSAL_READS, InvalidXLogRecPtr);
+	LWLockRelease(SyncRepLock);
+}
+
+/*
  * ===========================================================
  * Synchronous Replication functions executed by any process
  * ===========================================================
  */
 
 bool
-check_synchronous_standby_names(char **newval, void **extra, GucSource source)
+check_standby_names(char **newval, void **extra, GucSource source)
 {
 	char	   *rawstring;
 	List	   *elemlist;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index d061dfe..e68384c 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -55,6 +55,7 @@
 #include "libpq/pqformat.h"
 #include "libpq/pqsignal.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "replication/walsender.h"
 #include "storage/ipc.h"
@@ -150,7 +151,8 @@ static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
 static void XLogWalRcvSendReply(bool force, bool requestReply, bool includeApplyTimestamp);
 static void XLogWalRcvSendHSFeedback(bool immed);
-static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
+static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+								  TimestampTz *causalReadsUntil);
 
 /* Signal handlers */
 static void WalRcvSigHupHandler(SIGNAL_ARGS);
@@ -857,6 +859,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 	XLogRecPtr	walEnd;
 	TimestampTz sendTime;
 	bool		replyRequested;
+	TimestampTz causalReadsLease;
 
 	resetStringInfo(&incoming_message);
 
@@ -877,7 +880,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				walEnd = pq_getmsgint64(&incoming_message);
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, NULL);
 
 				buf += hdrlen;
 				len -= hdrlen;
@@ -889,7 +892,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				bool reportApplyTimestamp = false;
 
 				/* copy message to StringInfo */
-				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
+				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char) + sizeof(int64);
 				if (len != hdrlen)
 					ereport(ERROR,
 							(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -901,8 +904,10 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
 				replyRequested = pq_getmsgbyte(&incoming_message);
+				causalReadsLease = IntegerTimestampToTimestampTz(
+					pq_getmsgint64(&incoming_message));
 
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, &causalReadsLease);
 
 				/*
 				 * If no apply timestamps have been sent at the request of the
@@ -1253,15 +1258,52 @@ XLogWalRcvSendHSFeedback(bool immed)
  * Update shared memory status upon receiving a message from primary.
  *
  * 'walEnd' and 'sendTime' are the end-of-WAL and timestamp of the latest
- * message, reported by primary.
+ * message, reported by primary.  'causalReadsLease' is a pointer to
+ * the time the primary promises that this standby can safely claim to be
+ * causally consistent, to 0 if it cannot, or a NULL pointer for no change.
  */
 static void
-ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
+ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+					  TimestampTz *causalReadsLease)
 {
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
 	static TimestampTz lastRecordedTimestamp = 0;
 
+	/* Sanity check for the causalReadsLease time. */
+	if (causalReadsLease != NULL && *causalReadsLease != 0)
+	{
+		/* Deduce max_clock_skew from the causalReadsLease and sendTime. */
+#ifdef HAVE_INT64_TIMESTAMP
+		int64 diffMillis = (*causalReadsLease - sendTime) / 1000;
+#else
+		int64 diffMillis = (*causalReadsLease - sendTime) * 1000;
+#endif
+		int64 max_clock_skew = diffMillis / (CAUSAL_READS_CLOCK_SKEW_RATIO - 1);
+
+		if (sendTime > TimestampTzPlusMilliseconds(lastMsgReceiptTime, max_clock_skew))
+		{
+			/*
+			 * The primary's clock is more than max_clock_skew + network
+			 * latency ahead of the standby's clock.  (If the primary's clock
+			 * is more than max_clock_skew ahead of the standby's clock, but
+			 * by less than the network latency, then there isn't much we can
+			 * do to detect that; but it still seems useful to have this basic
+			 * sanity check for wildly misconfigured servers.)
+			 */
+			elog(LOG, "the primary server's clock time is too far ahead");
+			causalReadsLease = NULL;
+		}
+		/*
+		 * We could also try to detect cases where sendTime is more than
+		 * max_clock_skew in the past according to the standby's clock, but
+		 * that is indistinguishable from network latency/buffering, so we
+		 * could produce misleading error messages; if we do nothing, the
+		 * consequence is 'standby is not available for causal reads' errors
+		 * which should cause the user to investigate.
+		 */
+	}
+
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
 	if (walrcv->latestWalEnd < walEnd)
@@ -1269,6 +1311,8 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->latestWalEnd = walEnd;
 	walrcv->lastMsgSendTime = sendTime;
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
+	if (causalReadsLease != NULL)
+		walrcv->causalReadsLease = *causalReadsLease;
 	SpinLockRelease(&walrcv->mutex);
 
 	/*
diff --git a/src/backend/replication/walreceiverfuncs.c b/src/backend/replication/walreceiverfuncs.c
index 5f6e423..e502f74 100644
--- a/src/backend/replication/walreceiverfuncs.c
+++ b/src/backend/replication/walreceiverfuncs.c
@@ -28,6 +28,7 @@
 #include "replication/walreceiver.h"
 #include "storage/pmsignal.h"
 #include "storage/shmem.h"
+#include "utils/guc.h"
 #include "utils/timestamp.h"
 
 WalRcvData *WalRcv = NULL;
@@ -374,3 +375,21 @@ GetReplicationTransferLatency(void)
 
 	return ms;
 }
+
+/*
+ * Used by snapmgr to check if this standby has a valid lease, granting it the
+ * right to consider itself available for causal reads.
+ */
+bool
+WalRcvCausalReadsAvailable(void)
+{
+	WalRcvData *walrcv = WalRcv;
+	TimestampTz now = GetCurrentTimestamp();
+	bool result;
+
+	SpinLockAcquire(&walrcv->mutex);
+	result = walrcv->causalReadsLease != 0 && now <= walrcv->causalReadsLease;
+	SpinLockRelease(&walrcv->mutex);
+
+	return result;
+}
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 16d7abc..b4dad72 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -154,9 +154,20 @@ static StringInfoData tmpbuf;
  */
 static TimestampTz last_reply_timestamp = 0;
 
+static TimestampTz last_keepalive_timestamp = 0;
+
 /* Have we sent a heartbeat message asking for reply, since last reply? */
 static bool waiting_for_ping_response = false;
 
+/* How long do need to stay in JOINING state? */
+static XLogRecPtr causal_reads_joining_until = 0;
+
+/* The last causal reads lease sent to the standby. */
+static TimestampTz causal_reads_last_lease = 0;
+
+/* Is this WALSender listed in causal_reads_standby_names? */
+static bool am_potential_causal_reads_standby = false;
+
 /*
  * While streaming WAL in Copy mode, streamingDoneSending is set to true
  * after we have sent CopyDone. We should not send any more CopyData messages
@@ -242,6 +253,57 @@ InitWalSender(void)
 	SendPostmasterSignal(PMSIGNAL_ADVANCE_STATE_MACHINE);
 }
 
+ /*
+ * If we are exiting unexpectedly, we may need to communicate with concurrent
+ * causal_reads commits to maintain the causal consistency guarantee.
+ */
+static void
+PrepareUncleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * We've lost contact with the standby, but it may still be alive.  We
+		 * can't let any causal_reads transactions return until we've stalled
+		 * for long enough for a zombie standby to start raising errors
+		 * because its lease has expired.
+		 */
+		elog(LOG, "standby \"%s\" is lost (no longer available for causal reads)", application_name);
+		CausalReadsBeginStall(causal_reads_last_lease);
+
+		/*
+		 * We set the state to a lower level _after_ beginning the stall,
+		 * otherwise there would be a tiny window where commits could return
+		 * without observing the stall.
+		 */
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
+ * We are shutting down because we received a goodbye message from the
+ * walreceiver.
+ */
+static void
+PrepareCleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * The standby is shutting down, so it won't be running any more
+		 * transactions.  It is therefore safe to stop waiting for it, and no
+		 * stall is necessary.
+		 */
+		elog(LOG, "standby \"%s\" is leaving (no longer available for causal reads)", application_name);
+
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
 /*
  * Clean up after an error.
  *
@@ -266,7 +328,10 @@ WalSndErrorCleanup(void)
 
 	replication_active = false;
 	if (walsender_ready_to_stop)
+	{
+		PrepareUncleanExit();
 		proc_exit(0);
+	}
 
 	/* Revert back to startup state */
 	WalSndSetState(WALSNDSTATE_STARTUP);
@@ -278,6 +343,8 @@ WalSndErrorCleanup(void)
 static void
 WalSndShutdown(void)
 {
+	PrepareUncleanExit();
+
 	/*
 	 * Reset whereToSendOutput to prevent ereport from attempting to send any
 	 * more messages to the standby.
@@ -1388,6 +1455,7 @@ ProcessRepliesIfAny(void)
 		if (r < 0)
 		{
 			/* unexpected error or EOF */
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1404,6 +1472,7 @@ ProcessRepliesIfAny(void)
 		resetStringInfo(&reply_message);
 		if (pq_getmessage(&reply_message, 0))
 		{
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1453,6 +1522,7 @@ ProcessRepliesIfAny(void)
 				 * 'X' means that the standby is closing down the socket.
 				 */
 			case 'X':
+				PrepareCleanExit();
 				proc_exit(0);
 
 			default:
@@ -1584,6 +1654,83 @@ ProcessStandbyReplyMessage(void)
 	 */
 	{
 		WalSnd *walsnd = MyWalSnd;
+		WalSndCausalReadsState causal_reads_state = walsnd->causal_reads_state;
+		bool causal_reads_state_changed = false;
+		bool causal_reads_set_joining_until = false;
+
+		/*
+		 * Handle causal reads state transitions, if a causal_reads_timeout is
+		 * configured, this standby is listed in causal_reads_standby_names,
+		 * and we are a primary database (not a cascading standby).
+		 */
+		if (am_potential_causal_reads_standby &&
+			!am_cascading_walsender &&
+			applyLagUs >= 0)
+		{
+			if (applyLagUs / 1000 < causal_reads_timeout)
+			{
+				if (causal_reads_state == WALSNDCRSTATE_UNAVAILABLE)
+				{
+					/*
+					 * The standby is applying fast enough.  We can't grant a
+					 * lease yet though, we need to wait for everything that
+					 * was committed while this standby was unavailable to be
+					 * applied first.  We move to joining state while we wait
+					 * for the standby to catch up.
+					 */
+					causal_reads_state = WALSNDCRSTATE_JOINING;
+					causal_reads_set_joining_until = true;
+					causal_reads_state_changed = true;
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING &&
+						 applyPtr >= causal_reads_joining_until)
+				{
+					/*
+					 * The standby has applied everything committed before we
+					 * reached joining state, and has been waiting for remote
+					 * apply on this standby while it's been in joining state,
+					 * so it is safe to move to available state and send a
+					 * lease.
+					 */
+					causal_reads_state = WALSNDCRSTATE_AVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+			else
+			{
+				if (causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+					/*
+					 * We are dropping a causal reads available standby, so we
+					 * mustn't let any commit command that is waiting in
+					 * CausalReadsWaitForLSN return until we are sure that the
+					 * standby definitely knows that it's not available and
+					 * starts raising errors for causal_reads transactions.
+					 * TODO: We could just wait until the standby acks that
+					 * its lease has been cancelled, and start numbering
+					 * keepalives and sending the number back in replies, so
+					 * we know it's acking the right message; then lagging
+					 * standbys would be less disruptive, but for now we just
+					 * wait for the lease to expire, as we do when we lose
+					 * contact with a standby, for the sake of simplicity.
+					 */
+					CausalReadsBeginStall(causal_reads_last_lease);
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING)
+				{
+					/*
+					 * Dropping a joining standby doesn't require a stall,
+					 * because the standby doesn't think it's available, so
+					 * it's already raising the error for causal_reads
+					 * transactions.
+					 */
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+		}
 
 		SpinLockAcquire(&walsnd->mutex);
 		walsnd->write = writePtr;
@@ -1591,11 +1738,33 @@ ProcessStandbyReplyMessage(void)
 		walsnd->apply = applyPtr;
 		if (applyLagUs >= 0)
 			walsnd->applyLagUs = applyLagUs;
+		walsnd->causal_reads_state = causal_reads_state;
 		SpinLockRelease(&walsnd->mutex);
+
+		if (causal_reads_set_joining_until)
+		{
+			/*
+			 * Record the end of the primary's WAL at some arbitrary point
+			 * observed _after_ we moved to joining state (so that causal
+			 * reads commits start waiting, closing a race).  The standby
+			 * won't become available until it has replayed up to here.
+			 */
+			causal_reads_joining_until = GetFlushRecPtr();
+		}
+
+		if (causal_reads_state_changed)
+		{
+			WalSndKeepalive(true);
+			elog(LOG, "standby \"%s\" is %s", application_name,
+				 causal_reads_state == WALSNDCRSTATE_UNAVAILABLE ? "unavailable for causal reads" :
+				 causal_reads_state == WALSNDCRSTATE_JOINING ? "joining as a causal reads standby..." :
+				 causal_reads_state == WALSNDCRSTATE_AVAILABLE ? "available for causal reads" :
+				 "UNKNOWN");
+		}
 	}
 
 	if (!am_cascading_walsender)
-		SyncRepReleaseWaiters();
+		SyncRepReleaseWaiters(MyWalSnd->causal_reads_state >= WALSNDCRSTATE_JOINING);
 
 	/*
 	 * Advance our local xmin horizon when the client confirmed a flush.
@@ -1736,33 +1905,53 @@ ProcessStandbyHSFeedbackMessage(void)
  * If wal_sender_timeout is enabled we want to wake up in time to send
  * keepalives and to abort the connection if wal_sender_timeout has been
  * reached.
+ *
+ * But if causal_reads_timeout is enabled, we override that and send
+ * keepalives at a constant rate to replace expiring leases.
  */
 static long
 WalSndComputeSleeptime(TimestampTz now)
 {
 	long		sleeptime = 10000;		/* 10 s */
 
-	if (wal_sender_timeout > 0 && last_reply_timestamp > 0)
+	if ((wal_sender_timeout > 0 && last_reply_timestamp > 0) ||
+		am_potential_causal_reads_standby)
 	{
 		TimestampTz wakeup_time;
 		long		sec_to_timeout;
 		int			microsec_to_timeout;
 
-		/*
-		 * At the latest stop sleeping once wal_sender_timeout has been
-		 * reached.
-		 */
-		wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-												  wal_sender_timeout);
-
-		/*
-		 * If no ping has been sent yet, wakeup when it's time to do so.
-		 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
-		 * the timeout passed without a response.
-		 */
-		if (!waiting_for_ping_response)
+		if (am_potential_causal_reads_standby)
+		{
+			/*
+			 * Leases last for a period of between 50% and 100% of
+			 * causal_reads_timeout, depending on clock skew, assuming clock
+			 * skew is under the 25% of causal_reads_timeout.  We send new
+			 * leases every half a lease, so that there are no gaps between
+			 * leases.
+			 */
+			wakeup_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+													  causal_reads_timeout /
+													  CAUSAL_READS_KEEPALIVE_RATIO);
+		}
+		else
+		{
+			/*
+			 * At the latest stop sleeping once wal_sender_timeout has been
+			 * reached.
+			 */
 			wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-													  wal_sender_timeout / 2);
+													  wal_sender_timeout);
+
+			/*
+			 * If no ping has been sent yet, wakeup when it's time to do so.
+			 * WalSndKeepaliveIfNecessary() wants to send a keepalive once
+			 * half of the timeout passed without a response.
+			 */
+			if (!waiting_for_ping_response)
+				wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+														  wal_sender_timeout / 2);
+		}
 
 		/* Compute relative time until wakeup. */
 		TimestampDifference(now, wakeup_time,
@@ -1778,20 +1967,33 @@ WalSndComputeSleeptime(TimestampTz now)
 /*
  * Check whether there have been responses by the client within
  * wal_sender_timeout and shutdown if not.
+ *
+ * If causal_reads_timeout is configured we override that, so that
+ * unresponsive standbys are detected sooner.
  */
 static void
 WalSndCheckTimeOut(TimestampTz now)
 {
 	TimestampTz timeout;
+	int allowed_time;
 
 	/* don't bail out if we're doing something that doesn't require timeouts */
 	if (last_reply_timestamp <= 0)
 		return;
 
-	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
-										  wal_sender_timeout);
+	/*
+	 * If a causal_reads_timeout is configured, it is used instead of
+	 * wal_sender_timeout, to limit the time before an unresponsive causal
+	 * reads standby is dropped.
+	 */
+	if (am_potential_causal_reads_standby)
+		allowed_time = causal_reads_timeout;
+	else
+		allowed_time = wal_sender_timeout;
 
-	if (wal_sender_timeout > 0 && now >= timeout)
+	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
+										  allowed_time);
+	if (allowed_time > 0 && now >= timeout)
 	{
 		/*
 		 * Since typically expiration of replication timeout means
@@ -1824,6 +2026,9 @@ WalSndLoop(WalSndSendDataCallback send_data)
 	last_reply_timestamp = GetCurrentTimestamp();
 	waiting_for_ping_response = false;
 
+	/* Check if we are managing potential causal_reads standby. */
+	am_potential_causal_reads_standby = CausalReadsPotentialStandby();
+
 	/*
 	 * Loop until we reach the end of this timeline or the client requests to
 	 * stop streaming.
@@ -1984,6 +2189,7 @@ InitWalSenderSlot(void)
 			walsnd->flush = InvalidXLogRecPtr;
 			walsnd->apply = InvalidXLogRecPtr;
 			walsnd->state = WALSNDSTATE_STARTUP;
+			walsnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
 			walsnd->latch = &MyProc->procLatch;
 			SpinLockRelease(&walsnd->mutex);
 			/* don't need the lock anymore */
@@ -2753,6 +2959,24 @@ WalSndGetStateString(WalSndState state)
 	return "UNKNOWN";
 }
 
+/*
+ * Return a string constant representing the causal reads state. This is used
+ * in system views, and should *not* be translated.
+ */
+static const char *
+WalSndGetCausalReadsStateString(WalSndCausalReadsState causal_reads_state)
+{
+	switch (causal_reads_state)
+	{
+		case WALSNDCRSTATE_UNAVAILABLE:
+			return "unavailable";
+		case WALSNDCRSTATE_JOINING:
+			return "joining";
+		case WALSNDCRSTATE_AVAILABLE:
+			return "available";
+	}
+	return "UNKNOWN";
+}
 
 /*
  * Returns activity of walsenders, including pids and xlog locations sent to
@@ -2761,7 +2985,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	9
+#define PG_STAT_GET_WAL_SENDERS_COLS	10
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2812,6 +3036,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		int64		applyLagUs;
 		int			priority;
 		WalSndState state;
+		WalSndCausalReadsState causalReadsState;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
 		bool		nulls[PG_STAT_GET_WAL_SENDERS_COLS];
 
@@ -2821,6 +3046,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		SpinLockAcquire(&walsnd->mutex);
 		sentPtr = walsnd->sentPtr;
 		state = walsnd->state;
+		causalReadsState = walsnd->causal_reads_state;
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
@@ -2895,6 +3121,9 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				values[8] = CStringGetTextDatum("sync");
 			else
 				values[8] = CStringGetTextDatum("potential");
+
+			values[9] =
+				CStringGetTextDatum(WalSndGetCausalReadsStateString(causalReadsState));
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
@@ -2914,14 +3143,52 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 static void
 WalSndKeepalive(bool requestReply)
 {
+	TimestampTz now;
+	TimestampTz causal_reads_lease;
+
 	elog(DEBUG2, "sending replication keepalive");
 
+	/*
+	 * If the walsender currently deems the standby to be available for causal
+	 * reads, then it grants a causal reads lease.  The lease authorizes the
+	 * standby to consider itself available for causal reads until a short
+	 * time in the future.  The primary promises to uphold the causal reads
+	 * guarantee until that time, by stalling commits until the the lease has
+	 * expired if necessary.
+	 */
+	now = GetCurrentTimestamp();
+	if (MyWalSnd->causal_reads_state < WALSNDCRSTATE_AVAILABLE)
+		causal_reads_lease = 0; /* Not available, no lease granted. */
+	else
+	{
+		/*
+		 * Since this timestamp is being sent to the standby where it will be
+		 * compared against a time generated by the standby's system clock, we
+		 * must consider clock skew.  First, we decide on a maximum tolerable
+		 * difference between system clocks.  If the primary's clock is ahead
+		 * of the standby's by more than this, then all bets are off (the
+		 * standby could falsely believe it has a valid lease).  If the
+		 * primary's clock is behind the standby's by more than this, then the
+		 * standby will err the other way and generate spurious errors in
+		 * causal_reads mode.  Rather than having a separate GUC for this, we
+		 * derive it from causal_reads_timeout.
+		 */
+		int max_clock_skew = causal_reads_timeout / CAUSAL_READS_CLOCK_SKEW_RATIO;
+
+		/* Compute and remember the expiry time of the lease we're granting. */
+		causal_reads_last_lease = TimestampTzPlusMilliseconds(now, causal_reads_timeout);
+		/* The version we'll send to the standby is adjusted to tolerate clock skew. */
+		causal_reads_lease =
+			TimestampTzPlusMilliseconds(causal_reads_last_lease, -max_clock_skew);
+	}
+
 	/* construct the message... */
 	resetStringInfo(&output_message);
 	pq_sendbyte(&output_message, 'k');
 	pq_sendint64(&output_message, sentPtr);
-	pq_sendint64(&output_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(now));
 	pq_sendbyte(&output_message, requestReply ? 1 : 0);
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(causal_reads_lease));
 
 	/* ... and send it wrapped in CopyData */
 	pq_putmessage_noblock('d', output_message.data, output_message.len);
@@ -2939,23 +3206,35 @@ WalSndKeepaliveIfNecessary(TimestampTz now)
 	 * Don't send keepalive messages if timeouts are globally disabled or
 	 * we're doing something not partaking in timeouts.
 	 */
-	if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
-		return;
-
-	if (waiting_for_ping_response)
-		return;
+	if (!am_potential_causal_reads_standby)
+	{
+		if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
+			return;
+		if (waiting_for_ping_response)
+			return;
+	}
 
 	/*
 	 * If half of wal_sender_timeout has lapsed without receiving any reply
 	 * from the standby, send a keep-alive message to the standby requesting
 	 * an immediate reply.
+	 *
+	 * If causal_reads_timeout has been configured, use it to control
+	 * keepalive intervals rather than wal_sender_timeout, so that we can keep
+	 * replacing leases at the right frequency.
 	 */
-	ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-											wal_sender_timeout / 2);
+	if (am_potential_causal_reads_standby)
+		ping_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+												causal_reads_timeout /
+												CAUSAL_READS_KEEPALIVE_RATIO);
+	else
+		ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+												wal_sender_timeout / 2);
 	if (now >= ping_time)
 	{
 		WalSndKeepalive(true);
 		waiting_for_ping_response = true;
+		last_keepalive_timestamp = now;
 
 		/* Try to flush pending output to the client */
 		if (pq_flush_if_writable() != 0)
diff --git a/src/backend/utils/errcodes.txt b/src/backend/utils/errcodes.txt
index 04c9c00..d4bf0c0 100644
--- a/src/backend/utils/errcodes.txt
+++ b/src/backend/utils/errcodes.txt
@@ -302,6 +302,7 @@ Section: Class 40 - Transaction Rollback
 40001    E    ERRCODE_T_R_SERIALIZATION_FAILURE                              serialization_failure
 40003    E    ERRCODE_T_R_STATEMENT_COMPLETION_UNKNOWN                       statement_completion_unknown
 40P01    E    ERRCODE_T_R_DEADLOCK_DETECTED                                  deadlock_detected
+40P02    E    ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE                         causal_reads_not_available
 
 Section: Class 42 - Syntax Error or Access Rule Violation
 
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 88c4624..d47792a 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -1633,6 +1633,16 @@ static struct config_bool ConfigureNamesBool[] =
 		NULL, NULL, NULL
 	},
 
+	{
+		{"causal_reads", PGC_USERSET, REPLICATION_STANDBY,
+		 gettext_noop("Enables causal reads."),
+		 NULL
+		},
+		&causal_reads,
+		false,
+		NULL, NULL, NULL
+	},
+
 	/* End-of-list marker */
 	{
 		{NULL, 0, 0, NULL, NULL}, NULL, false, NULL, NULL, NULL
@@ -1791,6 +1801,17 @@ static struct config_int ConfigureNamesInt[] =
 	},
 
 	{
+		{"causal_reads_timeout", PGC_SIGHUP, REPLICATION_STANDBY,
+			gettext_noop("Sets the maximum apply lag before causal reads standbys are no longer available."),
+			NULL,
+			GUC_UNIT_MS
+		},
+		&causal_reads_timeout,
+		0, 0, INT_MAX,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"max_connections", PGC_POSTMASTER, CONN_AUTH_SETTINGS,
 			gettext_noop("Sets the maximum number of concurrent connections."),
 			NULL
@@ -3423,10 +3444,21 @@ static struct config_string ConfigureNamesString[] =
 		},
 		&SyncRepStandbyNames,
 		"",
-		check_synchronous_standby_names, NULL, NULL
+		check_standby_names, NULL, NULL
 	},
 
 	{
+		{"causal_reads_standby_names", PGC_SIGHUP, REPLICATION_MASTER,
+			gettext_noop("List of names of potential causal reads standbys."),
+			NULL,
+			GUC_LIST_INPUT
+		},
+		&causal_reads_standby_names,
+		"*",
+		check_standby_names, NULL, NULL
+ 	},
+
+	{
 		{"default_text_search_config", PGC_USERSET, CLIENT_CONN_LOCALE,
 			gettext_noop("Sets default text search configuration."),
 			NULL
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index 085099c..733b9e5 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -244,6 +244,15 @@
 				# from standby(s); '*' = all
 #vacuum_defer_cleanup_age = 0	# number of xacts by which cleanup is delayed
 
+#causal_reads_timeout = 0s      # maximum replication delay to tolerate from
+                                # standbys before dropping them from the set of
+				# available causal reads peers; 0 to disable
+				# causal reads
+
+#causal_reads_standy_names = '*'
+                                # standby servers that can potentially become
+				# available for causal reads; '*' = all
+
 # - Standby Servers -
 
 # These settings are ignored on a master server.
@@ -266,6 +275,14 @@
 #wal_retrieve_retry_interval = 5s	# time to wait before retrying to
 					# retrieve WAL after a failed attempt
 
+# - All Servers -
+
+#causal_reads = off                     # "on" in any pair of consecutive
+                                        # transactions guarantees that the second
+					# can see the first (even if the second
+					# is run on a standby), or will raise an
+					# error to report that the standby is
+					# unavailable for causal reads
 
 #------------------------------------------------------------------------------
 # QUERY TUNING
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index b88e012..6336240 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -46,8 +46,11 @@
 
 #include "access/transam.h"
 #include "access/xact.h"
+#include "access/xlog.h"
 #include "lib/pairingheap.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
+#include "replication/walreceiver.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
 #include "storage/procarray.h"
@@ -209,6 +212,16 @@ GetTransactionSnapshot(void)
 				 "cannot take query snapshot during a parallel operation");
 
 		/*
+		 * In causal_reads mode on a standby, check if we have definitely
+		 * applied WAL for any COMMIT that returned successfully on the
+		 * primary.
+		 */
+		if (causal_reads && RecoveryInProgress() && !WalRcvCausalReadsAvailable())
+			ereport(ERROR,
+					(errcode(ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE),
+					 errmsg("standby is not available for causal reads")));
+
+		/*
 		 * In transaction-snapshot mode, the first snapshot must live until
 		 * end of xact regardless of what the caller does with it, so we must
 		 * make a copy of it rather than returning CurrentSnapshotData
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index b42fc68..5435cd9 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2712,7 +2712,7 @@ DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f
 DESCR("statistics: information about currently active backends");
 DATA(insert OID = 3318 (  pg_stat_get_progress_info           PGNSP PGUID 12 1 100 0 0 f f f f f t s r 1 0 2249 "25" "{25,23,26,26,20,20,20,20,20,20,20,20,20,20}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{cmdtype,pid,datid,relid,param1,param2,param3,param4,param5,param6,param7,param8,param9,param10}" _null_ _null_ pg_stat_get_progress_info _null_ _null_ _null_ ));
 DESCR("statistics: information about progress of backends running maintenance command");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25}" "{o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25,25}" "{o,o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state,causal_reads_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index 28b68f6..253f831 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -24,14 +24,33 @@
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
 #define SYNC_REP_WAIT_APPLY		2
+#define SYNC_REP_WAIT_CAUSAL_READS 3
 
-#define NUM_SYNC_REP_WAIT_MODE	3
+#define NUM_SYNC_REP_WAIT_MODE	4
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
 #define SYNC_REP_WAITING			1
 #define SYNC_REP_WAIT_COMPLETE		2
 
+/*
+ * ratio of causal_read_timeout to max_clock_skew (4 means than the maximum
+ * tolerated clock difference between primary and standbys using causal_reads
+ * is 1/4 of causal_reads_timeout)
+ */
+#define CAUSAL_READS_CLOCK_SKEW_RATIO 4
+
+/*
+ * ratio of causal_reads_timeout to keepalive time (2 means that the effective
+ * keepalive time is 1/2 of the causal_reads_timeout GUC when it is non-zero)
+ */
+#define CAUSAL_READS_KEEPALIVE_RATIO 2
+
+/* GUC variables */
+extern int causal_reads_timeout;
+extern bool causal_reads;
+extern char *causal_reads_standby_names;
+
 /* user-settable parameters for synchronous replication */
 extern char *SyncRepStandbyNames;
 
@@ -43,16 +62,23 @@ extern void SyncRepCleanupAtProcExit(void);
 
 /* called by wal sender */
 extern void SyncRepInitConfig(void);
-extern void SyncRepReleaseWaiters(void);
+extern void SyncRepReleaseWaiters(bool walsender_cr_available_or_joining);
 
 /* called by checkpointer */
 extern void SyncRepUpdateSyncStandbysDefined(void);
 
+/* called by user backend (xact.c) */
+extern void CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN);
+
+/* called by wal sender */
+extern void CausalReadsBeginStall(TimestampTz lease_expiry_time);
+extern bool CausalReadsPotentialStandby(void);
+
 /* forward declaration to avoid pulling in walsender_private.h */
 struct WalSnd;
 extern struct WalSnd *SyncRepGetSynchronousStandby(void);
 
-extern bool check_synchronous_standby_names(char **newval, void **extra, GucSource source);
+extern bool check_standby_names(char **newval, void **extra, GucSource source);
 extern void assign_synchronous_commit(int newval, void *extra);
 
 #endif   /* _SYNCREP_H */
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 3294df9..7f83934 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -80,6 +80,13 @@ typedef struct
 	TimeLineID	receivedTLI;
 
 	/*
+	 * causalReadsLease is the time until which the primary has authorized
+	 * this standby to consider itself available for causal_reads mode, or 0
+	 * for not authorized.
+	 */
+	TimestampTz causalReadsLease;
+
+	/*
 	 * latestChunkStart is the starting byte position of the current "batch"
 	 * of received WAL.  It's actually the same as the previous value of
 	 * receivedUpto before the last flush to disk.  Startup process can use
@@ -164,4 +171,6 @@ extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
 extern void WalRcvWakeup(void);
 
+extern bool WalRcvCausalReadsAvailable(void);
+
 #endif   /* _WALRECEIVER_H */
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 4de43e8..f6e0e9e 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -27,6 +27,13 @@ typedef enum WalSndState
 	WALSNDSTATE_STREAMING
 } WalSndState;
 
+typedef enum WalSndCausalReadsState
+{
+	WALSNDCRSTATE_UNAVAILABLE = 0,
+	WALSNDCRSTATE_JOINING,
+	WALSNDCRSTATE_AVAILABLE
+} WalSndCausalReadsState;
+
 /*
  * Each walsender has a WalSnd struct in shared memory.
  */
@@ -34,6 +41,7 @@ typedef struct WalSnd
 {
 	pid_t		pid;			/* this walsender's process id, or 0 */
 	WalSndState state;			/* this walsender's state */
+	WalSndCausalReadsState causal_reads_state; /* the walsender's causal reads state */
 	XLogRecPtr	sentPtr;		/* WAL has been sent up to this point */
 	bool		needreload;		/* does currently-open file need to be
 								 * reloaded? */
@@ -89,6 +97,12 @@ typedef struct
 	 */
 	bool		sync_standbys_defined;
 
+	/*
+	 * Until when must commits in causal_reads stall?  This is used to wait
+	 * for causal reads leases to expire.
+	 */
+	TimestampTz	stall_causal_reads_until;
+
 	WalSnd		walsnds[FLEXIBLE_ARRAY_MEMBER];
 } WalSndCtlData;
 
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index f02a60c..a03654c 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1763,10 +1763,11 @@ pg_stat_replication| SELECT s.pid,
     w.replay_location,
     w.replay_lag,
     w.sync_priority,
-    w.sync_state
+    w.sync_state,
+    w.causal_reads_state
    FROM pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, wait_event_type, wait_event, xact_start, query_start, backend_start, state_change, client_addr, client_hostname, client_port, backend_xid, backend_xmin, ssl, sslversion, sslcipher, sslbits, sslcompression, sslclientdn),
     pg_authid u,
-    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state)
+    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state, causal_reads_state)
   WHERE ((s.usesysid = u.oid) AND (s.pid = w.pid));
 pg_stat_ssl| SELECT s.pid,
     s.ssl,
#49Robert Haas
robertmhaas@gmail.com
In reply to: Thomas Munro (#48)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Sun, Mar 13, 2016 at 11:50 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

The last patches I posted don't apply today due to changes in master,
so here's a freshly merged patch series.

+ from the current synchronous stanbyindicates it has received the

Uh, no.

-    SyncRepWaitForLSN(gxact->prepare_end_lsn);
+    {
+        /*
+         * Don't wait for the prepare to be applied remotely in remote_apply
+         * mode, just wait for it to be flushed to the WAL.  We will wait for
+         * apply when the transaction eventuallly commits or aborts.
+         */
+        if (synchronous_commit == SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+            assign_synchronous_commit(SYNCHRONOUS_COMMIT_REMOTE_FLUSH, NULL);
+
+        SyncRepWaitForLSN(gxact->prepare_end_lsn);
+
+        if (synchronous_commit == SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+            assign_synchronous_commit(SYNCHRONOUS_COMMIT_REMOTE_APPLY, NULL);
+    }

What's with the extra block?

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#50Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Robert Haas (#49)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Tue, Mar 15, 2016 at 6:58 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Sun, Mar 13, 2016 at 11:50 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

The last patches I posted don't apply today due to changes in master,
so here's a freshly merged patch series.

+ from the current synchronous stanbyindicates it has received the

Uh, no.

Oops, thanks, fixed. I'll wait for some more feedback or a conflict
with master before sending a new version.

-    SyncRepWaitForLSN(gxact->prepare_end_lsn);
+    {
+        /*
+         * Don't wait for the prepare to be applied remotely in remote_apply
+         * mode, just wait for it to be flushed to the WAL.  We will wait for
+         * apply when the transaction eventuallly commits or aborts.
+         */
+        if (synchronous_commit == SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+            assign_synchronous_commit(SYNCHRONOUS_COMMIT_REMOTE_FLUSH, NULL);
+
+        SyncRepWaitForLSN(gxact->prepare_end_lsn);
+
+        if (synchronous_commit == SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+            assign_synchronous_commit(SYNCHRONOUS_COMMIT_REMOTE_APPLY, NULL);
+    }

What's with the extra block?

Yeah, that's silly, thanks. Tidied up for the next version.

--
Thomas Munro
http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#51Robert Haas
robertmhaas@gmail.com
In reply to: Thomas Munro (#50)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Mon, Mar 14, 2016 at 2:38 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

What's with the extra block?

Yeah, that's silly, thanks. Tidied up for the next version.

Some more comments on 0001:

+ <literal>remote_write</>, <literal> remote_apply</>,
<literal>local</>, and <literal>off</>.

Extra space.

+ * apply when the transaction eventuallly commits or aborts.

Spelling.

+        if (synchronous_commit == SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+            assign_synchronous_commit(SYNCHRONOUS_COMMIT_REMOTE_FLUSH, NULL);
+
+        SyncRepWaitForLSN(gxact->prepare_end_lsn);
+
+        if (synchronous_commit == SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+            assign_synchronous_commit(SYNCHRONOUS_COMMIT_REMOTE_APPLY, NULL);

You can't do this. Directly changing the value of a variable that is
backing a GUC is verboten, and doing it through the thin veneer of
calling the assign-hook will not avoid the terrible wrath of the
powers that dwell in the outer dark, and/or Pittsburgh. You probably
need a dance here similar to the way forcePageWrites/fullPageWrites
work.

     /*
+     * Check if the caller would like to ask standbys for immediate feedback
+     * once this commit is applied.
+    */

Whitespace.

+    /*
+     * Check if the caller would like to ask standbys for immediate feedback
+     * once this abort is applied.
+    */

Whitespace again.

 /*
+ * doRequestWalReceiverReply is used by recovery code to ask the main recovery
+ * loop to trigger a walreceiver reply.
+ */
+static bool doRequestWalReceiverReply;

This is the sort of comment that leads me to ask "why even bother
writing a comment?". Try to say something that's not obvious from the
variable name. The comment for XLogRequestWalReceiverReply has a
similar issue.

+static void WalRcvBlockSigUsr2(void)

Style - newline after void.

+static void WalRcvUnblockSigUsr2(void)

And again here.

+                WalRcvUnblockSigUsr2();
                 len = walrcv_receive(NAPTIME_PER_CYCLE, &buf);
+                WalRcvBlockSigUsr2();

This does not seem like it will be cheap on all operating systems. I
think you should try to rejigger this somehow so that it can just set
the process latch and the wal receiver figures it out from looking at
shared memory. Like maybe a flag in WalRcvData? An advantage of this
is that it should cut down on the number of signals significantly,
because it won't need to send SIGUSR1 when the latch is already set.

+ * Although only "on", "off", "remote_apply", "remote_write", and "local" are
+ * documented, we accept all the likely variants of "on" and "off".

Maybe switch to listing the undocumented values.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#52Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Robert Haas (#51)
4 attachment(s)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Wed, Mar 23, 2016 at 12:37 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Mon, Mar 14, 2016 at 2:38 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

What's with the extra block?

Yeah, that's silly, thanks. Tidied up for the next version.

Some more comments on 0001:

+ <literal>remote_write</>, <literal> remote_apply</>,
<literal>local</>, and <literal>off</>.

Extra space.

Fixed.

+ * apply when the transaction eventuallly commits or aborts.

Spelling.

Fixed.

+        if (synchronous_commit == SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+            assign_synchronous_commit(SYNCHRONOUS_COMMIT_REMOTE_FLUSH, NULL);
+
+        SyncRepWaitForLSN(gxact->prepare_end_lsn);
+
+        if (synchronous_commit == SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+            assign_synchronous_commit(SYNCHRONOUS_COMMIT_REMOTE_APPLY, NULL);

You can't do this. Directly changing the value of a variable that is
backing a GUC is verboten, and doing it through the thin veneer of
calling the assign-hook will not avoid the terrible wrath of the
powers that dwell in the outer dark, and/or Pittsburgh. You probably
need a dance here similar to the way forcePageWrites/fullPageWrites
work.

Yeah, that was terrible. Instead of that I have now made this interface change:

-SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
+SyncRepWaitForLSN(XLogRecPtr lsn, bool commit)

If you want to wait for non-commit records (prepare being the only
case of that), you pass false in the second argument, and then the
wait level is capped at remote flush inside that function. There is
no way to wait for non-commit records to be applied and there is no
point in making it so that you can (they don't have any user-visible
effect).

/*
+     * Check if the caller would like to ask standbys for immediate feedback
+     * once this commit is applied.
+    */

Whitespace.

Fixed.

+    /*
+     * Check if the caller would like to ask standbys for immediate feedback
+     * once this abort is applied.
+    */

Whitespace again.

Fixed.

/*
+ * doRequestWalReceiverReply is used by recovery code to ask the main recovery
+ * loop to trigger a walreceiver reply.
+ */
+static bool doRequestWalReceiverReply;

This is the sort of comment that leads me to ask "why even bother
writing a comment?". Try to say something that's not obvious from the
variable name. The comment for XLogRequestWalReceiverReply has a
similar issue.

Changed.

+static void WalRcvBlockSigUsr2(void)

Style - newline after void.

Fixed.

+static void WalRcvUnblockSigUsr2(void)

And again here.

Fixed.

+                WalRcvUnblockSigUsr2();
len = walrcv_receive(NAPTIME_PER_CYCLE, &buf);
+                WalRcvBlockSigUsr2();

This does not seem like it will be cheap on all operating systems. I
think you should try to rejigger this somehow so that it can just set
the process latch and the wal receiver figures it out from looking at
shared memory. Like maybe a flag in WalRcvData? An advantage of this
is that it should cut down on the number of signals significantly,
because it won't need to send SIGUSR1 when the latch is already set.

Still experimenting with a latch here. I will come back on this point soon.

+ * Although only "on", "off", "remote_apply", "remote_write", and "local" are
+ * documented, we accept all the likely variants of "on" and "off".

Maybe switch to listing the undocumented values.

It follows a pattern used by several nearby bits of code, so it
doesn't look like it should be different, and besides you can see what
the undocumented values are, they're right below.

Here are some test results run on a bunch of Amazon EC2 "m3.large"
under Ubuntu Trusty in the Oregon zone, all in the same subnet.
Defaults except 1GB shared_buffers.

1. Simple sequential updates (using 'test-causal-reads.c', already
posted up-thread):

synchronous_commit TPS
==================== ====
off 9234
local 1223
remote_write 907
on 587
remote_apply 555

causal_reads TPS
==================== ====
0 cr standbys 1112
1 cr standbys 541
2 cr standbys 487
3 cr standbys 467

2. Some pgbench -c4 -j2 -N bench2 runs:

synchronous_commit TPS
==================== ====
off 3937
local 1984
remote_write 1701
on 1373
remote_apply 1349

causal_reads TPS
==================== ====
0 cr standbys 1973
1 cr standbys 1413
2 cr standbys 1282
3 cr standbys 1163

--
Thomas Munro
http://www.enterprisedb.com

Attachments:

0003-refactor-syncrep-exit-v5.patchapplication/octet-stream; name=0003-refactor-syncrep-exit-v5.patchDownload
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 1ee1bc5..376ddf4 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -83,6 +83,64 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * ===========================================================
  */
 
+static bool
+SyncRepCheckEarlyExit(void)
+{
+	/*
+	 * If a wait for synchronous replication is pending, we can neither
+	 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
+	 * lead the client to believe that the transaction aborted, which
+	 * is not true: it's already committed locally. The former is no good
+	 * either: the client has requested synchronous replication, and is
+	 * entitled to assume that an acknowledged commit is also replicated,
+	 * which might not be true. So in this case we issue a WARNING (which
+	 * some clients may be able to interpret) and shut off further output.
+	 * We do NOT reset ProcDiePending, so that the process will die after
+	 * the commit is cleaned up.
+	 */
+	if (ProcDiePending)
+	{
+		ereport(WARNING,
+				(errcode(ERRCODE_ADMIN_SHUTDOWN),
+				 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * It's unclear what to do if a query cancel interrupt arrives.  We
+	 * can't actually abort at this point, but ignoring the interrupt
+	 * altogether is not helpful, so we just terminate the wait with a
+	 * suitable warning.
+	 */
+	if (QueryCancelPending)
+	{
+		QueryCancelPending = false;
+		ereport(WARNING,
+				(errmsg("canceling wait for synchronous replication due to user request"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * If the postmaster dies, we'll probably never get an
+	 * acknowledgement, because all the wal sender processes will exit. So
+	 * just bail out.
+	 */
+	if (!PostmasterIsAlive())
+	{
+		ProcDiePending = true;
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	return false;
+}
+
 /*
  * Wait for synchronous replication, if requested by user.
  *
@@ -192,57 +250,9 @@ SyncRepWaitForLSN(XLogRecPtr lsn, bool commit)
 		if (syncRepState == SYNC_REP_WAIT_COMPLETE)
 			break;
 
-		/*
-		 * If a wait for synchronous replication is pending, we can neither
-		 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
-		 * lead the client to believe that the transaction aborted, which
-		 * is not true: it's already committed locally. The former is no good
-		 * either: the client has requested synchronous replication, and is
-		 * entitled to assume that an acknowledged commit is also replicated,
-		 * which might not be true. So in this case we issue a WARNING (which
-		 * some clients may be able to interpret) and shut off further output.
-		 * We do NOT reset ProcDiePending, so that the process will die after
-		 * the commit is cleaned up.
-		 */
-		if (ProcDiePending)
-		{
-			ereport(WARNING,
-					(errcode(ERRCODE_ADMIN_SHUTDOWN),
-					 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit())
 			break;
-		}
-
-		/*
-		 * It's unclear what to do if a query cancel interrupt arrives.  We
-		 * can't actually abort at this point, but ignoring the interrupt
-		 * altogether is not helpful, so we just terminate the wait with a
-		 * suitable warning.
-		 */
-		if (QueryCancelPending)
-		{
-			QueryCancelPending = false;
-			ereport(WARNING,
-					(errmsg("canceling wait for synchronous replication due to user request"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * If the postmaster dies, we'll probably never get an
-		 * acknowledgement, because all the wal sender processes will exit. So
-		 * just bail out.
-		 */
-		if (!PostmasterIsAlive())
-		{
-			ProcDiePending = true;
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
-			break;
-		}
 
 		/*
 		 * Wait on latch.  Any condition that should wake us up will set the
0004-causal-reads-v5.patchapplication/octet-stream; name=0004-causal-reads-v5.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 898979a..8b267b7 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -2749,6 +2749,35 @@ include_dir 'conf.d'
      across the cluster without problems if that is required.
     </para>
 
+    <sect2 id="runtime-config-replication-all">
+     <title>All Servers</title>
+     <para>
+      These parameters can be set on the primary or any standby.
+     </para>
+     <variablelist>
+      <varlistentry id="guc-causal-reads" xreflabel="causal_reads">
+       <term><varname>causal_reads</varname> (<type>boolean</type>)
+       <indexterm>
+        <primary><varname>causal_reads</> configuration parameter</primary>
+       </indexterm>
+       </term>
+       <listitem>
+        <para>
+         Enables causal consistency between transactions run on different
+         servers.  A transaction that is run on a standby
+         with <varname>causal_reads</> set to <literal>on</> is guaranteed
+         either to see the effects of all completed transactions run on the
+         primary with the setting on, or to receive an error "standby is not
+         available for causal reads".  Note that both transactions involved in
+         a causal dependency (a write on the primary followed by a read on any
+         server which must see the write) must be run with the setting on.
+         See <xref linkend="causal-reads"> for more details.
+        </para>
+       </listitem>
+      </varlistentry>
+     </variablelist>     
+    </sect2>
+
     <sect2 id="runtime-config-replication-sender">
      <title>Sending Server(s)</title>
 
@@ -2980,6 +3009,48 @@ include_dir 'conf.d'
       </listitem>
      </varlistentry>
 
+     <varlistentry>
+      <term><varname>causal_reads_timeout</varname> (<type>integer</type>)
+       <indexterm>
+        <primary><varname>causal_reads_timeout</> configuration parameter</primary>
+       </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies the maximum replay lag the primary will tolerate from a
+        standby before dropping it from the set of standbys available for
+        causal reads.
+       </para>
+       <para>
+        This setting is also used to control the <firstterm>leases</> used to
+        maintain the causal reads guarantee.  It must be set to a value which
+        is at least 4 times the maximum possible difference in system clocks
+        between the primary and standby servers, as described
+        in <xref linkend="causal-reads">.
+       </para>
+      </listitem>
+     </varlistentry>
+
+     <varlistentry id="guc-causal-reads-standby-names" xreflabel="causal-reads-standby-names">
+      <term><varname>causal_reads_standby_names</varname> (<type>string</type>)
+      <indexterm>
+       <primary><varname>causal_reads_standby_names</> configuration parameter</primary>
+      </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies a comma-separated list of standby names that can support
+        <firstterm>causal reads</>, as described in
+        <xref linkend="causal-reads">.  Follows the same convention
+        as <link linkend="guc-synchronous-standby-names"><literal>synchronous_standby_name</></>.
+        The default is <literal>*</>, matching all standbys.
+       </para>
+       <para>
+        This setting has no effect if <varname>causal_reads_timeout</> is not set.
+       </para>
+      </listitem>
+     </varlistentry>
+
      </variablelist>
     </sect2>
 
diff --git a/doc/src/sgml/high-availability.sgml b/doc/src/sgml/high-availability.sgml
index 03c6c30..7a0910d 100644
--- a/doc/src/sgml/high-availability.sgml
+++ b/doc/src/sgml/high-availability.sgml
@@ -1115,6 +1115,9 @@ primary_slot_name = 'node_a_slot'
     that it has replayed the transaction, making it visible to user queries.
     In simple cases, this allows for load balancing with causal consistency
     on a single hot standby.
+    (See also
+    <xref linkend="causal-reads"> which deals with multiple standbys and
+    standby failure.)
    </para>
 
    <para>
@@ -1233,6 +1236,119 @@ primary_slot_name = 'node_a_slot'
    </sect3>
   </sect2>
 
+  <sect2 id="causal-reads">
+   <title>Causal reads</title>
+   <indexterm>
+    <primary>causal reads</primary>
+    <secondary>in standby</secondary>
+   </indexterm>
+
+   <para>
+    The causal reads feature allows read-only queries to run on hot standby
+    servers without exposing stale data to the client, providing a form of
+    causal consistency.  Transactions can run on any standby with the
+    following guarantee about the visibility of preceding transactions: If you
+    set <varname>causal_reads</> to <literal>on</> in any pair of consecutive
+    transactions tx1, tx2 where tx2 begins after tx1 successfully returns,
+    then tx2 will either see tx1 or fail with a new error "standby is not
+    available for causal reads", no matter which server it runs on.  Although
+    the guarantee is expressed in terms of two individual transactions, the
+    GUC can also be set at session, role or system level to make the guarantee
+    generally, allowing for load balancing of applications that were not
+    designed with load balancing in mind.
+   </para>
+
+   <para>
+    In order to enable the feature, <varname>causal_reads_timeout</> must be
+    set to a non-zero value on the primary server.  The
+    GUC <varname>causal_reads_standby_names</> can be used to limit the set of
+    standbys that can join the dynamic set of causal reads standbys by
+    providing a comma-separated list of application names.  By default, all
+    standbys are candidates, if the feature is enabled.
+   </para>
+
+   <para>
+    The current set of servers that the primary considers to be available for
+    causal reads can be seen in
+    the <link linkend="monitoring-stats-views-table"> <literal>pg_stat_replication</></>
+    view.  Administrators, applications and load balancing middleware can use
+    this view to discover standbys that can currently handle causal reads
+    transactions without raising the error.  Since that information is only an
+    instantantaneous snapshot, clients should still be prepared for the error
+    to be raised at any time, and consider redirecting transactions to another
+    standby.
+   </para>
+
+   <para>
+    The advantages of the causal reads feature over simply
+    setting <varname>synchronous_commit</> to <literal>remote_apply</> are:
+    <orderedlist>
+      <listitem>
+       <para>
+        It allows the primary to wait for multiple standbys to replay
+        transactions.
+       </para>
+      </listitem>
+      <listitem>
+       <para>
+        It places a configurable limit on how much replay lag (and therefore
+        delay at commit time) the primary tolerates from standbys before it
+        drops them from the dynamic set of standbys it waits for.
+       </para>   
+      </listitem>
+      <listitem>
+       <para>
+        It upholds the causal reads guarantee during the transitions that
+        occur when new standbys are added or removed from the set of standbys,
+        including scenarios where contact has been lost between the primary
+        and standbys but the standby is still alive and running client
+        queries.
+       </para>
+      </listitem>
+    </orderedlist>
+   </para>
+
+   <para>
+    The protocol used to uphold the guarantee even in the case of network
+    failure depends on the system clocks of the primary and standby servers
+    being synchronized, with an allowance for a difference up to one quarter
+    of <varname>causal_reads_timeout</>.  For example,
+    if <varname>causal_reads_timeout</> is set to <literal>4s</>, then the
+    clocks must not be further than 1 second apart for the guarantee to be
+    upheld reliably during transitions.  The ubiquity of the Network Time
+    Protocol (NTP) on modern operating systems and availability of high
+    quality time servers makes it possible to choose a tolerance significantly
+    higher than the maximum expected clock difference.  An effort is
+    nevertheless made to detect and report misconfigured and faulty systems
+    with clock differences greater than the configured tolerance.
+   </para>
+
+   <note>
+    <para>
+     Current hardware clocks, NTP implementations and public time servers are
+     unlikely to allow the system clocks to differ more than tens or hundreds
+     of milliseconds, and systems synchronized with dedicated local time
+     servers may be considerably more accurate, but you should only consider
+     setting <varname>causal_reads_timeout</> below 4 seconds (allowing up to
+     1 second of clock difference) after researching your time synchronization
+     infrastructure thoroughly.
+    </para>  
+   </note>
+
+   <note>
+    <para>
+      While similar to synchronous replication in the sense that both involve
+      the primary server waiting for responses from standby servers, the
+      causal reads feature is not concerned with avoiding data loss.  A
+      primary configured for causal reads will drop all standbys that stop
+      responding or replay too slowly from the dynamic set that it waits for,
+      so you should consider configuring both synchronous replication and
+      causal reads if you need data loss avoidance guarantees and causal
+      consistency guarantees for load balancing.
+    </para>
+   </note>
+  </sect2>
+
   <sect2 id="continuous-archiving-in-standby">
    <title>Continuous archiving in standby</title>
 
@@ -1581,7 +1697,16 @@ if (!triggered)
     so there will be a measurable delay between primary and standby. Running the
     same query nearly simultaneously on both primary and standby might therefore
     return differing results. We say that data on the standby is
-    <firstterm>eventually consistent</firstterm> with the primary.  Once the
+    <firstterm>eventually consistent</firstterm> with the primary by default.
+    The data visible to a transaction running on a standby can be
+    made <firstterm>causally consistent</> with respect to a transaction that
+    has completed on the primary by setting <varname>causal_reads</>
+    to <literal>on</> in both transactions.  For more details,
+    see <xref linkend="causal-reads">.
+   </para>
+
+   <para>
+    Once the    
     commit record for a transaction is replayed on the standby, the changes
     made by that transaction will be visible to any new snapshots taken on
     the standby.  Snapshots may be taken at the start of each query or at the
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 7d63782..23d68d5 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1224,6 +1224,17 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
      <entry><type>text</></entry>
      <entry>Synchronous state of this standby server</entry>
     </row>
+    <row>
+     <entry><structfield>causal_reads_state</></entry>
+     <entry><type>text</></entry>
+     <entry>Causal reads state of this standby server.  This field will be
+     non-null only if <varname>cause_reads_timeout</> is set.  If a standby is
+     in <literal>available</> state, then it can currently serve causal reads
+     queries.  If it is not replaying fast enough or not responding to
+     keepalive messages, it will be in <literal>unavailable</> state, and if
+     it is currently transitioning to availability it will be
+     in <literal>joining</> state for a short time.</entry>
+    </row>
    </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 893c2fa..111198a 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -2098,11 +2098,12 @@ RecordTransactionCommitPrepared(TransactionId xid,
 	END_CRIT_SECTION();
 
 	/*
-	 * Wait for synchronous replication, if required.
+	 * Wait for causal reads and synchronous replication, if required.
 	 *
 	 * Note that at this stage we have marked clog, but still show as running
 	 * in the procarray and continue to hold locks.
 	 */
+	CausalReadsWaitForLSN(recptr);
 	SyncRepWaitForLSN(recptr, true);
 }
 
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 48a5950..4530a6e 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,10 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
+	{
+		CausalReadsWaitForLSN(XactLastRecEnd);
 		SyncRepWaitForLSN(XactLastRecEnd, true);
+	}
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5126,7 +5129,7 @@ XactLogCommitRecord(TimestampTz commit_time,
 	 * Check if the caller would like to ask standbys for immediate feedback
 	 * once this commit is applied.
 	 */
-	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY || causal_reads)
 		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
 
 	/*
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index a53f07b..276ac12 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -664,7 +664,8 @@ CREATE VIEW pg_stat_replication AS
             W.replay_location,
             W.replay_lag,
             W.sync_priority,
-            W.sync_state
+            W.sync_state,
+            W.causal_reads_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
             pg_stat_get_wal_senders() AS W
     WHERE S.usesysid = U.oid AND
diff --git a/src/backend/replication/README.causal_reads b/src/backend/replication/README.causal_reads
new file mode 100644
index 0000000..1fddd62
--- /dev/null
+++ b/src/backend/replication/README.causal_reads
@@ -0,0 +1,193 @@
+The causal reads guarantee says: If you run any two consecutive
+transactions tx1, tx2 where tx1 completes before tx2 begins, with
+causal_reads set to "on" in both transactions, tx2 will see tx1 or
+raise an error to complain that it can't guarantee causal consistency,
+no matter which servers (primary or any standby) you run each
+transaction on.
+
+When both transactions run on the primary, the guarantee is trivially
+upheld.
+
+To deal with read-only physical streaming standbys, the primary keeps
+track of a set of standbys that it considers to be currently
+"available" for causal reads, and sends a stream of "leases" to those
+standbys granting them the right to handle causal reads transactions
+for a short time without any further communication with the primary.
+
+In general, the primary provides the guarantee by waiting for all of
+the "available" standbys to report that they have applied a
+transaction.  However, the set of available standbys is dynamic, and
+things get more complicated during state transitions.  There are two
+types of transitions to consider:
+
+1.  unavailable->joining->available
+
+Standbys start out as "unavailable".  If a standby is unavailable and
+is applying fast enough and matches causal_reads_standby_names, the
+primary transitions it to "available", but first it sets it to
+"joining" until it is sure that any transaction committed while it was
+unavailable has definitely been applied on the standby.  This closes a
+race that would otherwise exist if we moved directly to available
+state: tx1 might not wait for a given standby because it's
+unavailable, then a lease might be granted, and then tx2 might run a
+causal reads transaction without error but see stale data.  The
+joining state acts as an airlock: while in joining state, the primary
+waits for that standby to replay causal reads transactions in
+anticipation of the move to available, but it doesn't progress to
+available state and grant a lease to the standby until everything
+preceding joining state has also been applied.
+
+2.  available->unavailable
+
+If a standby is not applying fast enough or not responding to
+keepalive messages, then the primary kicks that standby out of the
+dynamic set of available standbys, that is, marks it as "unavailable".
+In order to make sure that the standby has started rejecting causal
+reads transactions, it needs to revoke the lease it most recently
+granted.  It does that by waiting for the lease to expire before
+allowing any causal reads commits to return.  (In future there could
+be a fast-path revocation message which waits for a serial-numbered
+acknowledgement to reduce waiting in the case where the standby is
+lagging but still reachable and responding).
+
+The rest of this document illustrates how clock skew affects the
+available->unavailable transition.
+
+The following 4 variables are derived from a single GUC, and these
+values will be used in the following illustrations:
+
+causal_reads_timeout = 4s
+lease_time           = 4s (= causal_reads_timeout)
+keepalive_time       = 2s (= lease_time / 2)
+max_clock_skew       = 1s (= lease_time / 4)
+
+Every keepalive_time, the primary transmits a lease that expires at
+local_clock_time + lease_time - max_clock_skew, shown in the following
+diagram as 't' for transmission time and '|' for expiry time.  If
+contact is lost with a standby, the primary will wait until sent_time
++ lease_time for the most recently granted lease to expire, shown on
+the following diagram 'x', to be sure that the standby's clock has
+reached the expiry time even if its clock differs by up to
+max_clock_skew.  In other words, the primary tells the standby that
+the expiry time is at one time, but it trusts that the standby will
+surely agree if it gives it some extra time.  The extra time is
+max_clock_skew.  If the clocks differ by more than max_clock_skew, all
+bets are off (but see below for attempt to detect obvious cases).
+
+0     1     2     3     4     5     6     7     8     9
+t-----------------|-----x
+            t-----------------|-----x
+                        t-----------------|-----x
+                                    t-----------------|...
+                                                t------...
+
+A standby whose clock is 2 seconds ahead of the primary's clock
+perceives gaps in the stream of leases, and will reject causal_reads
+transactions in those intervals.  The causal reads guarantee is
+upheld, but spurious errors are raised between leases, as a
+consequence of the clock skew being greater than max_clock_skew.  In
+the following diagram 'r' shows reception time, and the timeline along
+the top shows the standby's local clock time.
+
+2     3     4     5     6     7     8     9    10    11
+r-----|
+            r-----|
+                        r-----|
+                                    r-----|
+                                                r-----|
+
+If there were no network latency, a standby whose clock is exactly 1
+second ahead of the primary's clock would perceive the stream of
+leases as being replaced just in time, so there is no gap.  Since in
+reality the time of receipt is some time after the time of
+transmission due to network latency, if the standby's clock is exactly
+1 second behind, then there will be small network-latency-sized gaps
+before the next lease arrives, but still no correctness problem with
+respect to the causal reads guarantee.
+
+1     2     3     4     5     6     7     8     9    10
+r-----------|
+            r-----------|
+                        r-----------|
+                                    r-----------|
+                                                r------...
+
+A standby whose clock is perfectly in sync with the primary's
+perceives the stream of leases overlapping (this matches the primary's
+perception of the leases it sent):
+
+0     1     2     3     4     5     6     7     8     9
+r-----------------|
+            r-----------------|
+                        r-----------------|
+                                    r-----------------|
+                                                r------...
+
+A standby whose clock is exactly 1 second behind the primary's
+perceives the stream of leases as overlapping even more, but the time
+of expiry as judged by the standby is no later than the time the
+primary will wait for if required ('x').  That is, if contact is lost
+with the standby, the primary can still reliably hold up causal reads
+commits until the standby has started raising the error in
+causal_reads transactions.
+
+-1    0     1     2     3     4     5     6     7     8
+r-----------------------|
+            r-----------------------|
+                        r-----------------------|
+                                    r------------------...
+                                                r------...
+
+
+A standby whose clock is 2 seconds behind the primary's would perceive
+the stream of leases overlapping even more, and the primary would no
+longer be able to wait for a lease to expire if it wanted to revoke
+it.  But because the expiry time is after local_clock_time +
+lease_time, the standby can immediately see that its own clock must be
+more than 1 second behind the primary's, so it ignores the lease and
+logs a clock skew warning.  In the following diagram a lease expiry
+time that is obviously generated by a primary with a clock set too far
+in the future compared to the local clock is shown with a '!'.
+
+-2    -1    0     1     2     3     4     5     6     7
+r-----------------------------!
+            r-----------------------------!
+                        r-----------------------------!
+                                    r------------------...
+                                                r------...
+
+A danger window exists when the standby's clock is more than
+max_clock_skew behind the primary's clock, but not more than
+max_clock_skew + network latency time behind.  If the clock difference
+is in that range, then the algorithm presented above which is based on
+time of receipt cannot detect that the local clock is too far behind.
+The consequence of this problem could be as follows:
+
+1.  The standby loses contact with the primary due to a network fault.
+
+2.  The primary decides to drop the standby from the set of available
+    causal reads standbys due to lack of keepalive responses or
+    excessive lag, which necessitates holding up commits of causal
+    reads transactions until the most recently sent lease expires, in
+    the belief that the standby will definitely have started raising
+    the 'causal reads unavailable' error in causal reads transactions
+    by that time, if it is still alive and servicing requests.
+
+3.  The standby still has clients connected and running queries.
+
+4.  Due to clock skew in the problematic range, in the standby's
+    opinion the lease lasts slightly longer than the primary waits.
+
+5.  For a short window at most the duration of the network latency
+    time, clients running causal reads transactions are allowed to see
+    potentially stale data.
+
+For this reason we say that the causal reads guarantee only holds as
+long as the absolute difference between the system clocks of the
+machines is no more than max_clock_skew.  The theory is that NTP makes
+it possible to reason about the maximum possible clock difference
+between machines and choose a value that allows for a much larger
+difference.  However, we do make a best effort attempt to detect
+wildly divergent systems as described above, to catch the case of
+servers not running a correctly configured ntp daemon, or with a clock
+so far out of whack that ntp refuses to fix it.
\ No newline at end of file
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 376ddf4..8240d0d 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -57,6 +57,11 @@
 #include "utils/builtins.h"
 #include "utils/ps_status.h"
 
+/* GUC variables */
+int causal_reads_timeout;
+bool causal_reads;
+char *causal_reads_standby_names;
+
 /* User-settable parameters for sync rep */
 char	   *SyncRepStandbyNames;
 
@@ -69,7 +74,7 @@ static int	SyncRepWaitMode = SYNC_REP_NO_WAIT;
 
 static void SyncRepQueueInsert(int mode);
 static void SyncRepCancelWait(void);
-static int	SyncRepWakeQueue(bool all, int mode);
+static int	SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn);
 
 static int	SyncRepGetStandbyPriority(void);
 
@@ -142,6 +147,198 @@ SyncRepCheckEarlyExit(void)
 }
 
 /*
+ * Check if we can stop waiting for causal consistency.  We can stop waiting
+ * when the following conditions are met:
+ *
+ * 1.  All walsenders currently in 'joining' or 'available' state have
+ * applied the target LSN.
+ *
+ * 2.  Any stall periods caused by standbys dropping out of 'available' state
+ * have passed, so that we can be sure that their leases have expired and they
+ * have started rejecting causal reads transactions.
+ *
+ * The output parameter 'waitingFor' is set to the number of nodes we are
+ * currently waiting for.  The output parameters 'stallTimeMillis' is set to
+ * the number of milliseconds we need to wait for to observe any current
+ * commit stall.
+ *
+ * Returns true if commit can return control, because every standby has either
+ * applied the LSN or started rejecting causal_reads transactions.
+ */
+static bool
+CausalReadsCommitCanReturn(XLogRecPtr XactCommitLSN,
+						   int *waitingFor,
+						   long *stallTimeMillis)
+{
+	int i;
+	TimestampTz now;
+
+	/* Count how many joining/available nodes we are waiting for. */
+	*waitingFor = 0;
+	for (i = 0; i < max_wal_senders; ++i)
+	{
+		WalSnd *walsnd = &WalSndCtl->walsnds[i];
+
+		/*
+		 * Assuming atomic read of pid_t, we can check walsnd->pid without
+		 * acquiring the spinlock to avoid memory synchronization costs for
+		 * unused walsender slots.  We see a value that existed sometime at
+		 * least as recently as the last memory barrier.
+		 */
+		if (walsnd->pid != 0)
+		{
+			/*
+			 * We need to hold the spinlock to read LSNs, because we can't be
+			 * sure they can be read atomically.
+			 */
+			SpinLockAcquire(&walsnd->mutex);
+			if (walsnd->pid != 0 && walsnd->causal_reads_state >= WALSNDCRSTATE_JOINING)
+			{
+				if (walsnd->apply < XactCommitLSN)
+					++*waitingFor;
+			}
+			SpinLockRelease(&walsnd->mutex);
+		}
+	}
+
+	/* Check if there is a stall in progress that we need to observe. */
+	now = GetCurrentTimestamp();
+	LWLockAcquire(SyncRepLock, LW_SHARED);
+	if (WalSndCtl->stall_causal_reads_until > now)
+	{
+		long seconds;
+		int usecs;
+
+		/* Compute how long we have to wait, rounded up to nearest ms. */
+		TimestampDifference(now, WalSndCtl->stall_causal_reads_until,
+							&seconds, &usecs);
+		*stallTimeMillis = seconds * 1000 + (usecs + 999) / 1000;
+	}
+	else
+		*stallTimeMillis = 0;
+	LWLockRelease(SyncRepLock);
+
+	/* We are done if we are not waiting for any nodes or stalls. */
+	return *waitingFor == 0 && *stallTimeMillis == 0;
+}
+
+/*
+ * Wait for causal consistency in causal_reads mode, if requested by user.
+ */
+void
+CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN)
+{
+	long stallTimeMillis;
+	int waitingFor;
+	char *ps_display_buffer = NULL;
+
+	/* Leave if we aren't in causal_reads mode. */
+	if (!causal_reads)
+		return;
+
+	for (;;)
+	{
+		/* Reset latch before checking state. */
+		ResetLatch(MyLatch);
+
+		/*
+		 * Join the queue to be woken up if any causal reads joining/available
+		 * standby applies XactCommitLSN or the set of causal reads standbys
+		 * changes (if we aren't already in the queue).  We don't actually know
+		 * if we need to wait for any peers to reach the target LSN yet, but
+		 * we have to register just in case before checking the walsenders'
+		 * state to avoid a race condition that could occur if we did it after
+		 * calling CausalReadsCommitCanReturn.  (SyncRepWaitForLSN doesn't
+		 * have to do this because it can check the highest-seen LSN in
+		 * walsndctl->lsn[mode] which is protected by SyncRepLock, the same
+		 * lock as the queues.  We can't do that here, because there is no
+		 * single highest-seen LSN that is useful.  We must check
+		 * walsnd->apply for all relevant walsenders.  Therefore we must
+		 * register for notifications first, so that we can be notified via
+		 * our latch of any standby applying the LSN we're interested in after
+		 * we check but before we start waiting, or we could wait forever for
+		 * something that has already happened.)
+		 */
+		LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+		if (MyProc->syncRepState != SYNC_REP_WAITING)
+		{
+			MyProc->waitLSN = XactCommitLSN;
+			MyProc->syncRepState = SYNC_REP_WAITING;
+			SyncRepQueueInsert(SYNC_REP_WAIT_CAUSAL_READS);
+			Assert(SyncRepQueueIsOrderedByLSN(SYNC_REP_WAIT_CAUSAL_READS));
+		}
+		LWLockRelease(SyncRepLock);
+
+		/* Check if we're done. */
+		if (CausalReadsCommitCanReturn(XactCommitLSN, &waitingFor, &stallTimeMillis))
+		{
+			SyncRepCancelWait();
+			break;
+		}
+
+		Assert(waitingFor > 0 || stallTimeMillis > 0);
+
+		/* If we aren't actually waiting for any standbys, leave the queue. */
+		if (waitingFor == 0)
+			SyncRepCancelWait();
+
+		/* Update the ps title. */
+		if (update_process_title)
+		{
+			char buffer[80];
+
+			/* Remember the old value if this is our first update. */
+			if (ps_display_buffer == NULL)
+			{
+				int len;
+				const char *ps_display = get_ps_display(&len);
+
+				ps_display_buffer = palloc(len + 1);
+				memcpy(ps_display_buffer, ps_display, len);
+				ps_display_buffer[len] = '\0';
+			}
+
+			snprintf(buffer, sizeof(buffer),
+					 "waiting for %d peer(s) to apply %X/%X%s",
+					 waitingFor,
+					 (uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN,
+					 stallTimeMillis > 0 ? " (stalling)" : "");
+			set_ps_display(buffer, false);
+		}
+
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit()) /* Calls SyncRepCancelWait() if true. */
+			break;
+
+		/*
+		 * If are still waiting for peers, then we wait for any joining or
+		 * available peer to reach the LSN (or possibly stop being in one of
+		 * those states or go away).
+		 *
+		 * If not, there must be a non-zero stall time, so we wait for that to
+		 * elapse.
+		 */
+		if (waitingFor > 0)
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH, -1);
+		else
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH | WL_TIMEOUT,
+					  stallTimeMillis);
+	}
+
+	/* There is no way out of the loop that could leave us in the queue. */
+	Assert(SHMQueueIsDetached(&(MyProc->syncRepLinks)));
+	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
+	MyProc->waitLSN = 0;
+
+	/* Restore the ps display. */
+	if (ps_display_buffer != NULL)
+	{
+		set_ps_display(ps_display_buffer, false);
+		pfree(ps_display_buffer);
+	}
+}
+
+/*
  * Wait for synchronous replication, if requested by user.
  *
  * Initially backends start in state SYNC_REP_NOT_WAITING and then
@@ -425,6 +622,53 @@ SyncRepGetSynchronousStandby(void)
 }
 
 /*
+ * Check if the current WALSender process's application_name matches a name in
+ * causal_reads_standby_names (including '*' for wildcard).
+ */
+bool
+CausalReadsPotentialStandby(void)
+{
+	char *rawstring;
+	List	   *elemlist;
+	ListCell   *l;
+	bool		found = false;
+
+	/* If the feature is disable, then no. */
+	if (causal_reads_timeout == 0)
+		return false;
+
+	/* Need a modifiable copy of string */
+	rawstring = pstrdup(causal_reads_standby_names);
+
+	/* Parse string into list of identifiers */
+	if (!SplitIdentifierString(rawstring, ',', &elemlist))
+	{
+		/* syntax error in list */
+		pfree(rawstring);
+		list_free(elemlist);
+		/* GUC machinery will have already complained - no need to do again */
+		return 0;
+	}
+
+	foreach(l, elemlist)
+	{
+		char	   *standby_name = (char *) lfirst(l);
+
+		if (pg_strcasecmp(standby_name, application_name) == 0 ||
+			pg_strcasecmp(standby_name, "*") == 0)
+		{
+			found = true;
+			break;
+		}
+	}
+
+	pfree(rawstring);
+	list_free(elemlist);
+
+	return found;
+}
+
+/*
  * Update the LSNs on each queue based upon our latest state. This
  * implements a simple policy of first-valid-standby-releases-waiter.
  *
@@ -432,23 +676,27 @@ SyncRepGetSynchronousStandby(void)
  * perhaps also which information we store as well.
  */
 void
-SyncRepReleaseWaiters(void)
+SyncRepReleaseWaiters(bool walsender_cr_available_or_joining)
 {
 	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
 	int			numapply = 0;
+	int			numcausalreadsapply = 0;
+	bool		is_highest_priority_sync_standby;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
-	 * potential sync standbys then we have nothing to do. If we are still
-	 * starting up, still running base backup or the current flush position
-	 * is still invalid, then leave quickly also.
+	 * potential sync standbys and not in a state that causal_reads waits for,
+	 * then we have nothing to do. If we are still starting up, still running
+	 * base backup or the current flush position is still invalid, then leave
+	 * quickly also.
 	 */
-	if (MyWalSnd->sync_standby_priority == 0 ||
-		MyWalSnd->state < WALSNDSTATE_STREAMING ||
-		XLogRecPtrIsInvalid(MyWalSnd->flush))
+	if (!walsender_cr_available_or_joining &&
+		(MyWalSnd->sync_standby_priority == 0 ||
+		 MyWalSnd->state < WALSNDSTATE_STREAMING ||
+		 XLogRecPtrIsInvalid(MyWalSnd->flush)))
 		return;
 
 	/*
@@ -458,13 +706,19 @@ SyncRepReleaseWaiters(void)
 	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
 	syncWalSnd = SyncRepGetSynchronousStandby();
 
-	/* We should have found ourselves at least */
-	Assert(syncWalSnd != NULL);
+	/*
+	 * If we aren't managing the highest priority standby then make a note of
+	 * that so we can announce a takeover in the log if we ever get that job.
+	 */
+	is_highest_priority_sync_standby = syncWalSnd == MyWalSnd;
+	if (!is_highest_priority_sync_standby)
+		announce_next_takeover = true;
 
 	/*
-	 * If we aren't managing the highest priority standby then just leave.
+	 * If we aren't managing the highest priority standby or a standby in
+	 * causal reads 'joining' or 'available' state, then just leave.
 	 */
-	if (syncWalSnd != MyWalSnd)
+	if (!is_highest_priority_sync_standby && !walsender_cr_available_or_joining)
 	{
 		LWLockRelease(SyncRepLock);
 		announce_next_takeover = true;
@@ -473,24 +727,45 @@ SyncRepReleaseWaiters(void)
 
 	/*
 	 * Set the lsn first so that when we wake backends they will release up to
-	 * this location.
+	 * this location.  For the single-standby synchronous commit levels, we
+	 * only do this if we are the current synchronous standby and we are
+	 * advancing the LSN further than it has been advanced before, so that
+	 * SyncRepWaitForLSN can skip waiting in some cases.
 	 */
-	if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
-		numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
-		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+	if (is_highest_priority_sync_standby)
 	{
-		walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
-		numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY);
+		if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
+			numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE,
+										MyWalSnd->write);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
+			numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH,
+										MyWalSnd->flush);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+			numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY,
+										MyWalSnd->apply);
+		}
 	}
 
+	/*
+	 * For causal_reads, all walsenders currently in available or joining
+	 * state must reach the LSN on their own, and standbys will reach LSNs in
+	 * any order.  It doesn't make sense to keep the highest seen LSN in a
+	 * single walsndctl->lsn element.  (CausalReadsWaitForLSN has handling for
+	 * LSNs that have already been reached).
+	 */
+	if (walsender_cr_available_or_joining)
+		numcausalreadsapply =
+			SyncRepWakeQueue(false, SYNC_REP_WAIT_CAUSAL_READS,
+							 MyWalSnd->apply);
+
 	LWLockRelease(SyncRepLock);
 
 	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%x",
@@ -502,7 +777,7 @@ SyncRepReleaseWaiters(void)
 	 * If we are managing the highest priority standby, though we weren't
 	 * prior to this, then announce we are now the sync standby.
 	 */
-	if (announce_next_takeover)
+	if (is_highest_priority_sync_standby && announce_next_takeover)
 	{
 		announce_next_takeover = false;
 		ereport(LOG,
@@ -577,9 +852,8 @@ SyncRepGetStandbyPriority(void)
  * Must hold SyncRepLock.
  */
 static int
-SyncRepWakeQueue(bool all, int mode)
+SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn)
 {
-	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	PGPROC	   *proc = NULL;
 	PGPROC	   *thisproc = NULL;
 	int			numprocs = 0;
@@ -596,7 +870,7 @@ SyncRepWakeQueue(bool all, int mode)
 		/*
 		 * Assume the queue is ordered by LSN
 		 */
-		if (!all && walsndctl->lsn[mode] < proc->waitLSN)
+		if (!all && lsn < proc->waitLSN)
 			return numprocs;
 
 		/*
@@ -656,7 +930,7 @@ SyncRepUpdateSyncStandbysDefined(void)
 			int			i;
 
 			for (i = 0; i < NUM_SYNC_REP_WAIT_MODE; i++)
-				SyncRepWakeQueue(true, i);
+				SyncRepWakeQueue(true, i, InvalidXLogRecPtr);
 		}
 
 		/*
@@ -708,13 +982,31 @@ SyncRepQueueIsOrderedByLSN(int mode)
 #endif
 
 /*
+ * Make sure that CausalReadsWaitForLSN can't return until after the given
+ * lease expiry time has been reached.  In other words, revoke the lease.
+ *
+ * Wake up all backends waiting in CausalReadsWaitForLSN, because the set of
+ * available/joining peers has changed, and there is a new stall time they
+ * need to observe.
+ */
+void
+CausalReadsBeginStall(TimestampTz lease_expiry_time)
+{
+	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+	WalSndCtl->stall_causal_reads_until =
+		Max(WalSndCtl->stall_causal_reads_until, lease_expiry_time);
+	SyncRepWakeQueue(true, SYNC_REP_WAIT_CAUSAL_READS, InvalidXLogRecPtr);
+	LWLockRelease(SyncRepLock);
+}
+
+/*
  * ===========================================================
  * Synchronous Replication functions executed by any process
  * ===========================================================
  */
 
 bool
-check_synchronous_standby_names(char **newval, void **extra, GucSource source)
+check_standby_names(char **newval, void **extra, GucSource source)
 {
 	char	   *rawstring;
 	List	   *elemlist;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 22bff89..de50ef0 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -55,6 +55,7 @@
 #include "libpq/pqformat.h"
 #include "libpq/pqsignal.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "replication/walsender.h"
 #include "storage/ipc.h"
@@ -150,7 +151,8 @@ static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
 static void XLogWalRcvSendReply(bool force, bool requestReply, bool includeApplyTimestamp);
 static void XLogWalRcvSendHSFeedback(bool immed);
-static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
+static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+								  TimestampTz *causalReadsUntil);
 
 /* Signal handlers */
 static void WalRcvSigHupHandler(SIGNAL_ARGS);
@@ -859,6 +861,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 	XLogRecPtr	walEnd;
 	TimestampTz sendTime;
 	bool		replyRequested;
+	TimestampTz causalReadsLease;
 
 	resetStringInfo(&incoming_message);
 
@@ -879,7 +882,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				walEnd = pq_getmsgint64(&incoming_message);
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, NULL);
 
 				buf += hdrlen;
 				len -= hdrlen;
@@ -891,7 +894,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				bool reportApplyTimestamp = false;
 
 				/* copy message to StringInfo */
-				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
+				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char) + sizeof(int64);
 				if (len != hdrlen)
 					ereport(ERROR,
 							(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -903,8 +906,10 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
 				replyRequested = pq_getmsgbyte(&incoming_message);
+				causalReadsLease = IntegerTimestampToTimestampTz(
+					pq_getmsgint64(&incoming_message));
 
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, &causalReadsLease);
 
 				/*
 				 * If no apply timestamps have been sent at the request of the
@@ -1255,15 +1260,52 @@ XLogWalRcvSendHSFeedback(bool immed)
  * Update shared memory status upon receiving a message from primary.
  *
  * 'walEnd' and 'sendTime' are the end-of-WAL and timestamp of the latest
- * message, reported by primary.
+ * message, reported by primary.  'causalReadsLease' is a pointer to
+ * the time the primary promises that this standby can safely claim to be
+ * causally consistent, to 0 if it cannot, or a NULL pointer for no change.
  */
 static void
-ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
+ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+					  TimestampTz *causalReadsLease)
 {
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
 	static TimestampTz lastRecordedTimestamp = 0;
 
+	/* Sanity check for the causalReadsLease time. */
+	if (causalReadsLease != NULL && *causalReadsLease != 0)
+	{
+		/* Deduce max_clock_skew from the causalReadsLease and sendTime. */
+#ifdef HAVE_INT64_TIMESTAMP
+		int64 diffMillis = (*causalReadsLease - sendTime) / 1000;
+#else
+		int64 diffMillis = (*causalReadsLease - sendTime) * 1000;
+#endif
+		int64 max_clock_skew = diffMillis / (CAUSAL_READS_CLOCK_SKEW_RATIO - 1);
+
+		if (sendTime > TimestampTzPlusMilliseconds(lastMsgReceiptTime, max_clock_skew))
+		{
+			/*
+			 * The primary's clock is more than max_clock_skew + network
+			 * latency ahead of the standby's clock.  (If the primary's clock
+			 * is more than max_clock_skew ahead of the standby's clock, but
+			 * by less than the network latency, then there isn't much we can
+			 * do to detect that; but it still seems useful to have this basic
+			 * sanity check for wildly misconfigured servers.)
+			 */
+			elog(LOG, "the primary server's clock time is too far ahead");
+			causalReadsLease = NULL;
+		}
+		/*
+		 * We could also try to detect cases where sendTime is more than
+		 * max_clock_skew in the past according to the standby's clock, but
+		 * that is indistinguishable from network latency/buffering, so we
+		 * could produce misleading error messages; if we do nothing, the
+		 * consequence is 'standby is not available for causal reads' errors
+		 * which should cause the user to investigate.
+		 */
+	}
+
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
 	if (walrcv->latestWalEnd < walEnd)
@@ -1271,6 +1313,8 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->latestWalEnd = walEnd;
 	walrcv->lastMsgSendTime = sendTime;
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
+	if (causalReadsLease != NULL)
+		walrcv->causalReadsLease = *causalReadsLease;
 	SpinLockRelease(&walrcv->mutex);
 
 	/*
diff --git a/src/backend/replication/walreceiverfuncs.c b/src/backend/replication/walreceiverfuncs.c
index 5f6e423..e502f74 100644
--- a/src/backend/replication/walreceiverfuncs.c
+++ b/src/backend/replication/walreceiverfuncs.c
@@ -28,6 +28,7 @@
 #include "replication/walreceiver.h"
 #include "storage/pmsignal.h"
 #include "storage/shmem.h"
+#include "utils/guc.h"
 #include "utils/timestamp.h"
 
 WalRcvData *WalRcv = NULL;
@@ -374,3 +375,21 @@ GetReplicationTransferLatency(void)
 
 	return ms;
 }
+
+/*
+ * Used by snapmgr to check if this standby has a valid lease, granting it the
+ * right to consider itself available for causal reads.
+ */
+bool
+WalRcvCausalReadsAvailable(void)
+{
+	WalRcvData *walrcv = WalRcv;
+	TimestampTz now = GetCurrentTimestamp();
+	bool result;
+
+	SpinLockAcquire(&walrcv->mutex);
+	result = walrcv->causalReadsLease != 0 && now <= walrcv->causalReadsLease;
+	SpinLockRelease(&walrcv->mutex);
+
+	return result;
+}
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 16d7abc..b4dad72 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -154,9 +154,20 @@ static StringInfoData tmpbuf;
  */
 static TimestampTz last_reply_timestamp = 0;
 
+static TimestampTz last_keepalive_timestamp = 0;
+
 /* Have we sent a heartbeat message asking for reply, since last reply? */
 static bool waiting_for_ping_response = false;
 
+/* How long do need to stay in JOINING state? */
+static XLogRecPtr causal_reads_joining_until = 0;
+
+/* The last causal reads lease sent to the standby. */
+static TimestampTz causal_reads_last_lease = 0;
+
+/* Is this WALSender listed in causal_reads_standby_names? */
+static bool am_potential_causal_reads_standby = false;
+
 /*
  * While streaming WAL in Copy mode, streamingDoneSending is set to true
  * after we have sent CopyDone. We should not send any more CopyData messages
@@ -242,6 +253,57 @@ InitWalSender(void)
 	SendPostmasterSignal(PMSIGNAL_ADVANCE_STATE_MACHINE);
 }
 
+ /*
+ * If we are exiting unexpectedly, we may need to communicate with concurrent
+ * causal_reads commits to maintain the causal consistency guarantee.
+ */
+static void
+PrepareUncleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * We've lost contact with the standby, but it may still be alive.  We
+		 * can't let any causal_reads transactions return until we've stalled
+		 * for long enough for a zombie standby to start raising errors
+		 * because its lease has expired.
+		 */
+		elog(LOG, "standby \"%s\" is lost (no longer available for causal reads)", application_name);
+		CausalReadsBeginStall(causal_reads_last_lease);
+
+		/*
+		 * We set the state to a lower level _after_ beginning the stall,
+		 * otherwise there would be a tiny window where commits could return
+		 * without observing the stall.
+		 */
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
+ * We are shutting down because we received a goodbye message from the
+ * walreceiver.
+ */
+static void
+PrepareCleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * The standby is shutting down, so it won't be running any more
+		 * transactions.  It is therefore safe to stop waiting for it, and no
+		 * stall is necessary.
+		 */
+		elog(LOG, "standby \"%s\" is leaving (no longer available for causal reads)", application_name);
+
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
 /*
  * Clean up after an error.
  *
@@ -266,7 +328,10 @@ WalSndErrorCleanup(void)
 
 	replication_active = false;
 	if (walsender_ready_to_stop)
+	{
+		PrepareUncleanExit();
 		proc_exit(0);
+	}
 
 	/* Revert back to startup state */
 	WalSndSetState(WALSNDSTATE_STARTUP);
@@ -278,6 +343,8 @@ WalSndErrorCleanup(void)
 static void
 WalSndShutdown(void)
 {
+	PrepareUncleanExit();
+
 	/*
 	 * Reset whereToSendOutput to prevent ereport from attempting to send any
 	 * more messages to the standby.
@@ -1388,6 +1455,7 @@ ProcessRepliesIfAny(void)
 		if (r < 0)
 		{
 			/* unexpected error or EOF */
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1404,6 +1472,7 @@ ProcessRepliesIfAny(void)
 		resetStringInfo(&reply_message);
 		if (pq_getmessage(&reply_message, 0))
 		{
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1453,6 +1522,7 @@ ProcessRepliesIfAny(void)
 				 * 'X' means that the standby is closing down the socket.
 				 */
 			case 'X':
+				PrepareCleanExit();
 				proc_exit(0);
 
 			default:
@@ -1584,6 +1654,83 @@ ProcessStandbyReplyMessage(void)
 	 */
 	{
 		WalSnd *walsnd = MyWalSnd;
+		WalSndCausalReadsState causal_reads_state = walsnd->causal_reads_state;
+		bool causal_reads_state_changed = false;
+		bool causal_reads_set_joining_until = false;
+
+		/*
+		 * Handle causal reads state transitions, if a causal_reads_timeout is
+		 * configured, this standby is listed in causal_reads_standby_names,
+		 * and we are a primary database (not a cascading standby).
+		 */
+		if (am_potential_causal_reads_standby &&
+			!am_cascading_walsender &&
+			applyLagUs >= 0)
+		{
+			if (applyLagUs / 1000 < causal_reads_timeout)
+			{
+				if (causal_reads_state == WALSNDCRSTATE_UNAVAILABLE)
+				{
+					/*
+					 * The standby is applying fast enough.  We can't grant a
+					 * lease yet though, we need to wait for everything that
+					 * was committed while this standby was unavailable to be
+					 * applied first.  We move to joining state while we wait
+					 * for the standby to catch up.
+					 */
+					causal_reads_state = WALSNDCRSTATE_JOINING;
+					causal_reads_set_joining_until = true;
+					causal_reads_state_changed = true;
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING &&
+						 applyPtr >= causal_reads_joining_until)
+				{
+					/*
+					 * The standby has applied everything committed before we
+					 * reached joining state, and has been waiting for remote
+					 * apply on this standby while it's been in joining state,
+					 * so it is safe to move to available state and send a
+					 * lease.
+					 */
+					causal_reads_state = WALSNDCRSTATE_AVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+			else
+			{
+				if (causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+					/*
+					 * We are dropping a causal reads available standby, so we
+					 * mustn't let any commit command that is waiting in
+					 * CausalReadsWaitForLSN return until we are sure that the
+					 * standby definitely knows that it's not available and
+					 * starts raising errors for causal_reads transactions.
+					 * TODO: We could just wait until the standby acks that
+					 * its lease has been cancelled, and start numbering
+					 * keepalives and sending the number back in replies, so
+					 * we know it's acking the right message; then lagging
+					 * standbys would be less disruptive, but for now we just
+					 * wait for the lease to expire, as we do when we lose
+					 * contact with a standby, for the sake of simplicity.
+					 */
+					CausalReadsBeginStall(causal_reads_last_lease);
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING)
+				{
+					/*
+					 * Dropping a joining standby doesn't require a stall,
+					 * because the standby doesn't think it's available, so
+					 * it's already raising the error for causal_reads
+					 * transactions.
+					 */
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+		}
 
 		SpinLockAcquire(&walsnd->mutex);
 		walsnd->write = writePtr;
@@ -1591,11 +1738,33 @@ ProcessStandbyReplyMessage(void)
 		walsnd->apply = applyPtr;
 		if (applyLagUs >= 0)
 			walsnd->applyLagUs = applyLagUs;
+		walsnd->causal_reads_state = causal_reads_state;
 		SpinLockRelease(&walsnd->mutex);
+
+		if (causal_reads_set_joining_until)
+		{
+			/*
+			 * Record the end of the primary's WAL at some arbitrary point
+			 * observed _after_ we moved to joining state (so that causal
+			 * reads commits start waiting, closing a race).  The standby
+			 * won't become available until it has replayed up to here.
+			 */
+			causal_reads_joining_until = GetFlushRecPtr();
+		}
+
+		if (causal_reads_state_changed)
+		{
+			WalSndKeepalive(true);
+			elog(LOG, "standby \"%s\" is %s", application_name,
+				 causal_reads_state == WALSNDCRSTATE_UNAVAILABLE ? "unavailable for causal reads" :
+				 causal_reads_state == WALSNDCRSTATE_JOINING ? "joining as a causal reads standby..." :
+				 causal_reads_state == WALSNDCRSTATE_AVAILABLE ? "available for causal reads" :
+				 "UNKNOWN");
+		}
 	}
 
 	if (!am_cascading_walsender)
-		SyncRepReleaseWaiters();
+		SyncRepReleaseWaiters(MyWalSnd->causal_reads_state >= WALSNDCRSTATE_JOINING);
 
 	/*
 	 * Advance our local xmin horizon when the client confirmed a flush.
@@ -1736,33 +1905,53 @@ ProcessStandbyHSFeedbackMessage(void)
  * If wal_sender_timeout is enabled we want to wake up in time to send
  * keepalives and to abort the connection if wal_sender_timeout has been
  * reached.
+ *
+ * But if causal_reads_timeout is enabled, we override that and send
+ * keepalives at a constant rate to replace expiring leases.
  */
 static long
 WalSndComputeSleeptime(TimestampTz now)
 {
 	long		sleeptime = 10000;		/* 10 s */
 
-	if (wal_sender_timeout > 0 && last_reply_timestamp > 0)
+	if ((wal_sender_timeout > 0 && last_reply_timestamp > 0) ||
+		am_potential_causal_reads_standby)
 	{
 		TimestampTz wakeup_time;
 		long		sec_to_timeout;
 		int			microsec_to_timeout;
 
-		/*
-		 * At the latest stop sleeping once wal_sender_timeout has been
-		 * reached.
-		 */
-		wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-												  wal_sender_timeout);
-
-		/*
-		 * If no ping has been sent yet, wakeup when it's time to do so.
-		 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
-		 * the timeout passed without a response.
-		 */
-		if (!waiting_for_ping_response)
+		if (am_potential_causal_reads_standby)
+		{
+			/*
+			 * Leases last for a period of between 50% and 100% of
+			 * causal_reads_timeout, depending on clock skew, assuming clock
+			 * skew is under the 25% of causal_reads_timeout.  We send new
+			 * leases every half a lease, so that there are no gaps between
+			 * leases.
+			 */
+			wakeup_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+													  causal_reads_timeout /
+													  CAUSAL_READS_KEEPALIVE_RATIO);
+		}
+		else
+		{
+			/*
+			 * At the latest stop sleeping once wal_sender_timeout has been
+			 * reached.
+			 */
 			wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-													  wal_sender_timeout / 2);
+													  wal_sender_timeout);
+
+			/*
+			 * If no ping has been sent yet, wakeup when it's time to do so.
+			 * WalSndKeepaliveIfNecessary() wants to send a keepalive once
+			 * half of the timeout passed without a response.
+			 */
+			if (!waiting_for_ping_response)
+				wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+														  wal_sender_timeout / 2);
+		}
 
 		/* Compute relative time until wakeup. */
 		TimestampDifference(now, wakeup_time,
@@ -1778,20 +1967,33 @@ WalSndComputeSleeptime(TimestampTz now)
 /*
  * Check whether there have been responses by the client within
  * wal_sender_timeout and shutdown if not.
+ *
+ * If causal_reads_timeout is configured we override that, so that
+ * unresponsive standbys are detected sooner.
  */
 static void
 WalSndCheckTimeOut(TimestampTz now)
 {
 	TimestampTz timeout;
+	int allowed_time;
 
 	/* don't bail out if we're doing something that doesn't require timeouts */
 	if (last_reply_timestamp <= 0)
 		return;
 
-	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
-										  wal_sender_timeout);
+	/*
+	 * If a causal_reads_timeout is configured, it is used instead of
+	 * wal_sender_timeout, to limit the time before an unresponsive causal
+	 * reads standby is dropped.
+	 */
+	if (am_potential_causal_reads_standby)
+		allowed_time = causal_reads_timeout;
+	else
+		allowed_time = wal_sender_timeout;
 
-	if (wal_sender_timeout > 0 && now >= timeout)
+	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
+										  allowed_time);
+	if (allowed_time > 0 && now >= timeout)
 	{
 		/*
 		 * Since typically expiration of replication timeout means
@@ -1824,6 +2026,9 @@ WalSndLoop(WalSndSendDataCallback send_data)
 	last_reply_timestamp = GetCurrentTimestamp();
 	waiting_for_ping_response = false;
 
+	/* Check if we are managing potential causal_reads standby. */
+	am_potential_causal_reads_standby = CausalReadsPotentialStandby();
+
 	/*
 	 * Loop until we reach the end of this timeline or the client requests to
 	 * stop streaming.
@@ -1984,6 +2189,7 @@ InitWalSenderSlot(void)
 			walsnd->flush = InvalidXLogRecPtr;
 			walsnd->apply = InvalidXLogRecPtr;
 			walsnd->state = WALSNDSTATE_STARTUP;
+			walsnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
 			walsnd->latch = &MyProc->procLatch;
 			SpinLockRelease(&walsnd->mutex);
 			/* don't need the lock anymore */
@@ -2753,6 +2959,24 @@ WalSndGetStateString(WalSndState state)
 	return "UNKNOWN";
 }
 
+/*
+ * Return a string constant representing the causal reads state. This is used
+ * in system views, and should *not* be translated.
+ */
+static const char *
+WalSndGetCausalReadsStateString(WalSndCausalReadsState causal_reads_state)
+{
+	switch (causal_reads_state)
+	{
+		case WALSNDCRSTATE_UNAVAILABLE:
+			return "unavailable";
+		case WALSNDCRSTATE_JOINING:
+			return "joining";
+		case WALSNDCRSTATE_AVAILABLE:
+			return "available";
+	}
+	return "UNKNOWN";
+}
 
 /*
  * Returns activity of walsenders, including pids and xlog locations sent to
@@ -2761,7 +2985,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	9
+#define PG_STAT_GET_WAL_SENDERS_COLS	10
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2812,6 +3036,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		int64		applyLagUs;
 		int			priority;
 		WalSndState state;
+		WalSndCausalReadsState causalReadsState;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
 		bool		nulls[PG_STAT_GET_WAL_SENDERS_COLS];
 
@@ -2821,6 +3046,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		SpinLockAcquire(&walsnd->mutex);
 		sentPtr = walsnd->sentPtr;
 		state = walsnd->state;
+		causalReadsState = walsnd->causal_reads_state;
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
@@ -2895,6 +3121,9 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				values[8] = CStringGetTextDatum("sync");
 			else
 				values[8] = CStringGetTextDatum("potential");
+
+			values[9] =
+				CStringGetTextDatum(WalSndGetCausalReadsStateString(causalReadsState));
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
@@ -2914,14 +3143,52 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 static void
 WalSndKeepalive(bool requestReply)
 {
+	TimestampTz now;
+	TimestampTz causal_reads_lease;
+
 	elog(DEBUG2, "sending replication keepalive");
 
+	/*
+	 * If the walsender currently deems the standby to be available for causal
+	 * reads, then it grants a causal reads lease.  The lease authorizes the
+	 * standby to consider itself available for causal reads until a short
+	 * time in the future.  The primary promises to uphold the causal reads
+	 * guarantee until that time, by stalling commits until the the lease has
+	 * expired if necessary.
+	 */
+	now = GetCurrentTimestamp();
+	if (MyWalSnd->causal_reads_state < WALSNDCRSTATE_AVAILABLE)
+		causal_reads_lease = 0; /* Not available, no lease granted. */
+	else
+	{
+		/*
+		 * Since this timestamp is being sent to the standby where it will be
+		 * compared against a time generated by the standby's system clock, we
+		 * must consider clock skew.  First, we decide on a maximum tolerable
+		 * difference between system clocks.  If the primary's clock is ahead
+		 * of the standby's by more than this, then all bets are off (the
+		 * standby could falsely believe it has a valid lease).  If the
+		 * primary's clock is behind the standby's by more than this, then the
+		 * standby will err the other way and generate spurious errors in
+		 * causal_reads mode.  Rather than having a separate GUC for this, we
+		 * derive it from causal_reads_timeout.
+		 */
+		int max_clock_skew = causal_reads_timeout / CAUSAL_READS_CLOCK_SKEW_RATIO;
+
+		/* Compute and remember the expiry time of the lease we're granting. */
+		causal_reads_last_lease = TimestampTzPlusMilliseconds(now, causal_reads_timeout);
+		/* The version we'll send to the standby is adjusted to tolerate clock skew. */
+		causal_reads_lease =
+			TimestampTzPlusMilliseconds(causal_reads_last_lease, -max_clock_skew);
+	}
+
 	/* construct the message... */
 	resetStringInfo(&output_message);
 	pq_sendbyte(&output_message, 'k');
 	pq_sendint64(&output_message, sentPtr);
-	pq_sendint64(&output_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(now));
 	pq_sendbyte(&output_message, requestReply ? 1 : 0);
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(causal_reads_lease));
 
 	/* ... and send it wrapped in CopyData */
 	pq_putmessage_noblock('d', output_message.data, output_message.len);
@@ -2939,23 +3206,35 @@ WalSndKeepaliveIfNecessary(TimestampTz now)
 	 * Don't send keepalive messages if timeouts are globally disabled or
 	 * we're doing something not partaking in timeouts.
 	 */
-	if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
-		return;
-
-	if (waiting_for_ping_response)
-		return;
+	if (!am_potential_causal_reads_standby)
+	{
+		if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
+			return;
+		if (waiting_for_ping_response)
+			return;
+	}
 
 	/*
 	 * If half of wal_sender_timeout has lapsed without receiving any reply
 	 * from the standby, send a keep-alive message to the standby requesting
 	 * an immediate reply.
+	 *
+	 * If causal_reads_timeout has been configured, use it to control
+	 * keepalive intervals rather than wal_sender_timeout, so that we can keep
+	 * replacing leases at the right frequency.
 	 */
-	ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-											wal_sender_timeout / 2);
+	if (am_potential_causal_reads_standby)
+		ping_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+												causal_reads_timeout /
+												CAUSAL_READS_KEEPALIVE_RATIO);
+	else
+		ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+												wal_sender_timeout / 2);
 	if (now >= ping_time)
 	{
 		WalSndKeepalive(true);
 		waiting_for_ping_response = true;
+		last_keepalive_timestamp = now;
 
 		/* Try to flush pending output to the client */
 		if (pq_flush_if_writable() != 0)
diff --git a/src/backend/utils/errcodes.txt b/src/backend/utils/errcodes.txt
index 1a920e8..9fdc992 100644
--- a/src/backend/utils/errcodes.txt
+++ b/src/backend/utils/errcodes.txt
@@ -303,6 +303,7 @@ Section: Class 40 - Transaction Rollback
 40001    E    ERRCODE_T_R_SERIALIZATION_FAILURE                              serialization_failure
 40003    E    ERRCODE_T_R_STATEMENT_COMPLETION_UNKNOWN                       statement_completion_unknown
 40P01    E    ERRCODE_T_R_DEADLOCK_DETECTED                                  deadlock_detected
+40P02    E    ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE                         causal_reads_not_available
 
 Section: Class 42 - Syntax Error or Access Rule Violation
 
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 06cb166..ac422e7 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -1634,6 +1634,16 @@ static struct config_bool ConfigureNamesBool[] =
 	},
 
 	{
+		{"causal_reads", PGC_USERSET, REPLICATION_STANDBY,
+		 gettext_noop("Enables causal reads."),
+		 NULL
+		},
+		&causal_reads,
+		false,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"syslog_sequence_numbers", PGC_SIGHUP, LOGGING_WHERE,
 			gettext_noop("Add sequence number to syslog messages to avoid duplicate suppression."),
 			NULL
@@ -1811,6 +1821,17 @@ static struct config_int ConfigureNamesInt[] =
 	},
 
 	{
+		{"causal_reads_timeout", PGC_SIGHUP, REPLICATION_STANDBY,
+			gettext_noop("Sets the maximum apply lag before causal reads standbys are no longer available."),
+			NULL,
+			GUC_UNIT_MS
+		},
+		&causal_reads_timeout,
+		0, 0, INT_MAX,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"max_connections", PGC_POSTMASTER, CONN_AUTH_SETTINGS,
 			gettext_noop("Sets the maximum number of concurrent connections."),
 			NULL
@@ -3454,7 +3475,18 @@ static struct config_string ConfigureNamesString[] =
 		},
 		&SyncRepStandbyNames,
 		"",
-		check_synchronous_standby_names, NULL, NULL
+		check_standby_names, NULL, NULL
+	},
+
+	{
+		{"causal_reads_standby_names", PGC_SIGHUP, REPLICATION_MASTER,
+			gettext_noop("List of names of potential causal reads standbys."),
+			NULL,
+			GUC_LIST_INPUT
+		},
+		&causal_reads_standby_names,
+		"*",
+		check_standby_names, NULL, NULL
 	},
 
 	{
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index ec4427f..fcc2c35 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -244,6 +244,15 @@
 				# from standby(s); '*' = all
 #vacuum_defer_cleanup_age = 0	# number of xacts by which cleanup is delayed
 
+#causal_reads_timeout = 0s      # maximum replication delay to tolerate from
+                                # standbys before dropping them from the set of
+				# available causal reads peers; 0 to disable
+				# causal reads
+
+#causal_reads_standy_names = '*'
+                                # standby servers that can potentially become
+				# available for causal reads; '*' = all
+
 # - Standby Servers -
 
 # These settings are ignored on a master server.
@@ -266,6 +275,14 @@
 #wal_retrieve_retry_interval = 5s	# time to wait before retrying to
 					# retrieve WAL after a failed attempt
 
+# - All Servers -
+
+#causal_reads = off                     # "on" in any pair of consecutive
+                                        # transactions guarantees that the second
+					# can see the first (even if the second
+					# is run on a standby), or will raise an
+					# error to report that the standby is
+					# unavailable for causal reads
 
 #------------------------------------------------------------------------------
 # QUERY TUNING
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index b88e012..6336240 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -46,8 +46,11 @@
 
 #include "access/transam.h"
 #include "access/xact.h"
+#include "access/xlog.h"
 #include "lib/pairingheap.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
+#include "replication/walreceiver.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
 #include "storage/procarray.h"
@@ -209,6 +212,16 @@ GetTransactionSnapshot(void)
 				 "cannot take query snapshot during a parallel operation");
 
 		/*
+		 * In causal_reads mode on a standby, check if we have definitely
+		 * applied WAL for any COMMIT that returned successfully on the
+		 * primary.
+		 */
+		if (causal_reads && RecoveryInProgress() && !WalRcvCausalReadsAvailable())
+			ereport(ERROR,
+					(errcode(ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE),
+					 errmsg("standby is not available for causal reads")));
+
+		/*
 		 * In transaction-snapshot mode, the first snapshot must live until
 		 * end of xact regardless of what the caller does with it, so we must
 		 * make a copy of it rather than returning CurrentSnapshotData
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 4054726..c0d7173 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2712,7 +2712,7 @@ DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f
 DESCR("statistics: information about currently active backends");
 DATA(insert OID = 3318 (  pg_stat_get_progress_info           PGNSP PGUID 12 1 100 0 0 f f f f t t s r 1 0 2249 "25" "{25,23,26,26,20,20,20,20,20,20,20,20,20,20}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{cmdtype,pid,datid,relid,param1,param2,param3,param4,param5,param6,param7,param8,param9,param10}" _null_ _null_ pg_stat_get_progress_info _null_ _null_ _null_ ));
 DESCR("statistics: information about progress of backends running maintenance command");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25}" "{o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25,25}" "{o,o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state,causal_reads_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index c005a42..dbfd601 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -24,14 +24,33 @@
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
 #define SYNC_REP_WAIT_APPLY		2
+#define SYNC_REP_WAIT_CAUSAL_READS 3
 
-#define NUM_SYNC_REP_WAIT_MODE	3
+#define NUM_SYNC_REP_WAIT_MODE	4
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
 #define SYNC_REP_WAITING			1
 #define SYNC_REP_WAIT_COMPLETE		2
 
+/*
+ * ratio of causal_read_timeout to max_clock_skew (4 means than the maximum
+ * tolerated clock difference between primary and standbys using causal_reads
+ * is 1/4 of causal_reads_timeout)
+ */
+#define CAUSAL_READS_CLOCK_SKEW_RATIO 4
+
+/*
+ * ratio of causal_reads_timeout to keepalive time (2 means that the effective
+ * keepalive time is 1/2 of the causal_reads_timeout GUC when it is non-zero)
+ */
+#define CAUSAL_READS_KEEPALIVE_RATIO 2
+
+/* GUC variables */
+extern int causal_reads_timeout;
+extern bool causal_reads;
+extern char *causal_reads_standby_names;
+
 /* user-settable parameters for synchronous replication */
 extern char *SyncRepStandbyNames;
 
@@ -43,16 +62,23 @@ extern void SyncRepCleanupAtProcExit(void);
 
 /* called by wal sender */
 extern void SyncRepInitConfig(void);
-extern void SyncRepReleaseWaiters(void);
+extern void SyncRepReleaseWaiters(bool walsender_cr_available_or_joining);
 
 /* called by checkpointer */
 extern void SyncRepUpdateSyncStandbysDefined(void);
 
+/* called by user backend (xact.c) */
+extern void CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN);
+
+/* called by wal sender */
+extern void CausalReadsBeginStall(TimestampTz lease_expiry_time);
+extern bool CausalReadsPotentialStandby(void);
+
 /* forward declaration to avoid pulling in walsender_private.h */
 struct WalSnd;
 extern struct WalSnd *SyncRepGetSynchronousStandby(void);
 
-extern bool check_synchronous_standby_names(char **newval, void **extra, GucSource source);
+extern bool check_standby_names(char **newval, void **extra, GucSource source);
 extern void assign_synchronous_commit(int newval, void *extra);
 
 #endif   /* _SYNCREP_H */
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 3294df9..7f83934 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -80,6 +80,13 @@ typedef struct
 	TimeLineID	receivedTLI;
 
 	/*
+	 * causalReadsLease is the time until which the primary has authorized
+	 * this standby to consider itself available for causal_reads mode, or 0
+	 * for not authorized.
+	 */
+	TimestampTz causalReadsLease;
+
+	/*
 	 * latestChunkStart is the starting byte position of the current "batch"
 	 * of received WAL.  It's actually the same as the previous value of
 	 * receivedUpto before the last flush to disk.  Startup process can use
@@ -164,4 +171,6 @@ extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
 extern void WalRcvWakeup(void);
 
+extern bool WalRcvCausalReadsAvailable(void);
+
 #endif   /* _WALRECEIVER_H */
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 4de43e8..f6e0e9e 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -27,6 +27,13 @@ typedef enum WalSndState
 	WALSNDSTATE_STREAMING
 } WalSndState;
 
+typedef enum WalSndCausalReadsState
+{
+	WALSNDCRSTATE_UNAVAILABLE = 0,
+	WALSNDCRSTATE_JOINING,
+	WALSNDCRSTATE_AVAILABLE
+} WalSndCausalReadsState;
+
 /*
  * Each walsender has a WalSnd struct in shared memory.
  */
@@ -34,6 +41,7 @@ typedef struct WalSnd
 {
 	pid_t		pid;			/* this walsender's process id, or 0 */
 	WalSndState state;			/* this walsender's state */
+	WalSndCausalReadsState causal_reads_state; /* the walsender's causal reads state */
 	XLogRecPtr	sentPtr;		/* WAL has been sent up to this point */
 	bool		needreload;		/* does currently-open file need to be
 								 * reloaded? */
@@ -89,6 +97,12 @@ typedef struct
 	 */
 	bool		sync_standbys_defined;
 
+	/*
+	 * Until when must commits in causal_reads stall?  This is used to wait
+	 * for causal reads leases to expire.
+	 */
+	TimestampTz	stall_causal_reads_until;
+
 	WalSnd		walsnds[FLEXIBLE_ARRAY_MEMBER];
 } WalSndCtlData;
 
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index fc4b765..44f826f 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1785,10 +1785,11 @@ pg_stat_replication| SELECT s.pid,
     w.replay_location,
     w.replay_lag,
     w.sync_priority,
-    w.sync_state
+    w.sync_state,
+    w.causal_reads_state
    FROM pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, wait_event_type, wait_event, xact_start, query_start, backend_start, state_change, client_addr, client_hostname, client_port, backend_xid, backend_xmin, ssl, sslversion, sslcipher, sslbits, sslcompression, sslclientdn),
     pg_authid u,
-    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state)
+    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state, causal_reads_state)
   WHERE ((s.usesysid = u.oid) AND (s.pid = w.pid));
 pg_stat_ssl| SELECT s.pid,
     s.ssl,
0001-remote-apply-v5.patchapplication/octet-stream; name=0001-remote-apply-v5.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index d48a13f..898979a 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -2143,7 +2143,7 @@ include_dir 'conf.d'
         Specifies whether transaction commit will wait for WAL records
         to be written to disk before the command returns a <quote>success</>
         indication to the client.  Valid values are <literal>on</>,
-        <literal>remote_write</>, <literal>local</>, and <literal>off</>.
+        <literal>remote_write</>, <literal>remote_apply</>, <literal>local</>, and <literal>off</>.
         The default, and safe, setting
         is <literal>on</>.  When <literal>off</>, there can be a delay between
         when success is reported to the client and when the transaction is
@@ -2177,6 +2177,10 @@ include_dir 'conf.d'
         ensure data preservation even if the standby instance of
         <productname>PostgreSQL</> were to crash, but not if the standby
         suffers an operating-system-level crash.
+        When set to <literal>remote_apply</>, commits will wait until a reply
+        from the current synchronous standby indicates it has received the
+        commit record of the transaction and applied it, so that it has become
+        visible to queries.
        </para>
        <para>
         When synchronous
diff --git a/doc/src/sgml/high-availability.sgml b/doc/src/sgml/high-availability.sgml
index 19d613e..03c6c30 100644
--- a/doc/src/sgml/high-availability.sgml
+++ b/doc/src/sgml/high-availability.sgml
@@ -1081,6 +1081,9 @@ primary_slot_name = 'node_a_slot'
     WAL record is then sent to the standby. The standby sends reply
     messages each time a new batch of WAL data is written to disk, unless
     <varname>wal_receiver_status_interval</> is set to zero on the standby.
+    In the case that <varname>synchronous_commit</> is set to
+    <literal>remote_apply</>, the standby sends reply messages when the commit
+    record is replayed, making the transaction visible.
     If the standby is the first matching standby, as specified in
     <varname>synchronous_standby_names</> on the primary, the reply
     messages from that standby will be used to wake users waiting for
@@ -1107,6 +1110,14 @@ primary_slot_name = 'node_a_slot'
    </para>
 
    <para>
+    Setting <varname>synchronous_commit</> to <literal>remote_apply</> will
+    cause each commit to wait until the current synchronous standby reports
+    that it has replayed the transaction, making it visible to user queries.
+    In simple cases, this allows for load balancing with causal consistency
+    on a single hot standby.
+   </para>
+
+   <para>
     Users will stop waiting if a fast shutdown is requested.  However, as
     when using asynchronous replication, the server will not fully
     shutdown until all outstanding WAL records are transferred to the currently
@@ -1160,9 +1171,10 @@ primary_slot_name = 'node_a_slot'
     <title>Planning for High Availability</title>
 
    <para>
-    Commits made when <varname>synchronous_commit</> is set to <literal>on</>
-    or <literal>remote_write</> will wait until the synchronous standby responds. The response
-    may never occur if the last, or only, standby should crash.
+    Commits made when <varname>synchronous_commit</> is set to <literal>on</>,
+    <literal>remote_write</> or <literal>remote_apply</> will wait until the
+    synchronous standby responds. The response may never occur if the last, or
+    only, standby should crash.
    </para>
 
    <para>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index e7234c8..893c2fa 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -1107,7 +1107,7 @@ EndPrepare(GlobalTransaction gxact)
 	 * Note that at this stage we have marked the prepare, but still show as
 	 * running in the procarray (twice!) and continue to hold locks.
 	 */
-	SyncRepWaitForLSN(gxact->prepare_end_lsn);
+	SyncRepWaitForLSN(gxact->prepare_end_lsn, false);
 
 	records.tail = records.head = NULL;
 	records.num_chunks = 0;
@@ -2103,7 +2103,7 @@ RecordTransactionCommitPrepared(TransactionId xid,
 	 * Note that at this stage we have marked clog, but still show as running
 	 * in the procarray and continue to hold locks.
 	 */
-	SyncRepWaitForLSN(recptr);
+	SyncRepWaitForLSN(recptr, true);
 }
 
 /*
@@ -2156,5 +2156,5 @@ RecordTransactionAbortPrepared(TransactionId xid,
 	 * Note that at this stage we have marked clog, but still show as running
 	 * in the procarray and continue to hold locks.
 	 */
-	SyncRepWaitForLSN(recptr);
+	SyncRepWaitForLSN(recptr, true);
 }
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 89a14b4..130b56b 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,7 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
-		SyncRepWaitForLSN(XactLastRecEnd);
+		SyncRepWaitForLSN(XactLastRecEnd, true);
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5123,6 +5123,13 @@ XactLogCommitRecord(TimestampTz commit_time,
 		xl_xinfo.xinfo |= XACT_COMPLETION_FORCE_SYNC_COMMIT;
 
 	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this commit is applied.
+	 */
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
+	/*
 	 * Relcache invalidations requires information about the current database
 	 * and so does logical decoding.
 	 */
@@ -5300,6 +5307,13 @@ XactLogAbortRecord(TimestampTz abort_time,
 	if (xl_xinfo.xinfo & XACT_XINFO_HAS_TWOPHASE)
 		XLogRegisterData((char *) (&xl_twophase), sizeof(xl_xact_twophase));
 
+	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this abort is applied.
+	 */
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
 	return XLogInsert(RM_XACT_ID, info);
 }
 
@@ -5458,6 +5472,13 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 	if (XactCompletionForceSyncCommit(parsed->xinfo))
 		XLogFlush(lsn);
 
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit = remote_apply), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 /*
@@ -5544,6 +5565,14 @@ xact_redo_abort(xl_xact_parsed_abort *parsed, TransactionId xid)
 		smgrdounlink(srel, true);
 		smgrclose(srel);
 	}
+
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit = remote_apply), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 void
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index b119a47..3e454f5 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -345,6 +345,9 @@ static XLogRecPtr RedoRecPtr;
  */
 static bool doPageWrites;
 
+/* Has the recovery code requested a walreceiver wakeup? */
+static bool doRequestWalReceiverReply;
+
 /*
  * RedoStartLSN points to the checkpoint's REDO location which is specified
  * in a backup label file, backup history file or control file. In standby
@@ -6879,6 +6882,19 @@ StartupXLOG(void)
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
 				SpinLockRelease(&XLogCtl->info_lck);
 
+				/*
+				 * If rm_redo reported that it applied a commit record that
+				 * the master is waiting for by calling
+				 * XLogRequestWalReceiverReply, then we wake up the receiver
+				 * so that it notices the updated lastReplayedEndRecPtr and
+				 * sends a reply to the master.
+				 */
+				if (doRequestWalReceiverReply)
+				{
+					doRequestWalReceiverReply = false;
+					WalRcvWakeup();
+				}
+
 				/* Remember this record as the last-applied one */
 				LastRec = ReadRecPtr;
 
@@ -11594,3 +11610,12 @@ SetWalWriterSleeping(bool sleeping)
 	XLogCtl->WalWriterSleeping = sleeping;
 	SpinLockRelease(&XLogCtl->info_lck);
 }
+
+/*
+ * Schedule a walreceiver wakeup in the main recovery loop.
+ */
+void
+XLogRequestWalReceiverReply(void)
+{
+	doRequestWalReceiverReply = true;
+}
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 92faf4e..1ee1bc5 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -91,13 +91,25 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * to the wait queue. During SyncRepWakeQueue() a WALSender changes
  * the state to SYNC_REP_WAIT_COMPLETE once replication is confirmed.
  * This backend then resets its state to SYNC_REP_NOT_WAITING.
+ *
+ * 'lsn' represents the LSN to wait for.  'commit' indicates whether this LSN
+ * represents a commit/abort record.  If it's not, then we wait only for the
+ * WAL to be flushed if synchronous_commit is set to the higher level of
+ * remote_apply, because standbys only send apply feedback for commit/abort
+ * records.
  */
 void
-SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
+SyncRepWaitForLSN(XLogRecPtr lsn, bool commit)
 {
 	char	   *new_status = NULL;
 	const char *old_status;
-	int			mode = SyncRepWaitMode;
+	int			mode;
+
+	/* Cap the level for non-commit records to remote flush only. */
+	if (commit)
+		mode = SyncRepWaitMode;
+	else
+		mode = Max(SyncRepWaitMode, SYNC_REP_WAIT_FLUSH);
 
 	/*
 	 * Fast exit if user has not requested sync replication, or there are no
@@ -122,7 +134,7 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 	 * to be a low cost check.
 	 */
 	if (!WalSndCtl->sync_standbys_defined ||
-		XactCommitLSN <= WalSndCtl->lsn[mode])
+		lsn <= WalSndCtl->lsn[mode])
 	{
 		LWLockRelease(SyncRepLock);
 		return;
@@ -132,7 +144,7 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 	 * Set our waitLSN so WALSender will know when to wake us, and add
 	 * ourselves to the queue.
 	 */
-	MyProc->waitLSN = XactCommitLSN;
+	MyProc->waitLSN = lsn;
 	MyProc->syncRepState = SYNC_REP_WAITING;
 	SyncRepQueueInsert(mode);
 	Assert(SyncRepQueueIsOrderedByLSN(mode));
@@ -147,7 +159,7 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 		new_status = (char *) palloc(len + 32 + 1);
 		memcpy(new_status, old_status, len);
 		sprintf(new_status + len, " waiting for %X/%X",
-				(uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN);
+				(uint32) (lsn >> 32), (uint32) lsn);
 		set_ps_display(new_status, false);
 		new_status[len] = '\0'; /* truncate off " waiting ..." */
 	}
@@ -416,6 +428,7 @@ SyncRepReleaseWaiters(void)
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
+	int			numapply = 0;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
@@ -462,12 +475,18 @@ SyncRepReleaseWaiters(void)
 		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
 		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
 	}
+	if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+	{
+		walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+		numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY);
+	}
 
 	LWLockRelease(SyncRepLock);
 
-	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X",
+	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%x",
 		 numwrite, (uint32) (MyWalSnd->write >> 32), (uint32) MyWalSnd->write,
-	   numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush);
+		 numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush,
+		 numapply, (uint32) (MyWalSnd->apply >> 32), (uint32) MyWalSnd->apply);
 
 	/*
 	 * If we are managing the highest priority standby, though we weren't
@@ -728,6 +747,9 @@ assign_synchronous_commit(int newval, void *extra)
 		case SYNCHRONOUS_COMMIT_REMOTE_FLUSH:
 			SyncRepWaitMode = SYNC_REP_WAIT_FLUSH;
 			break;
+		case SYNCHRONOUS_COMMIT_REMOTE_APPLY:
+			SyncRepWaitMode = SYNC_REP_WAIT_APPLY;
+			break;
 		default:
 			SyncRepWaitMode = SYNC_REP_NO_WAIT;
 			break;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 7b36e02..59e65c2 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -101,6 +101,7 @@ static uint32 recvOff = 0;
  */
 static volatile sig_atomic_t got_SIGHUP = false;
 static volatile sig_atomic_t got_SIGTERM = false;
+static volatile sig_atomic_t got_SIGUSR2 = false;
 
 /*
  * LogstreamResult indicates the byte positions that we have already
@@ -150,9 +151,29 @@ static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
 /* Signal handlers */
 static void WalRcvSigHupHandler(SIGNAL_ARGS);
 static void WalRcvSigUsr1Handler(SIGNAL_ARGS);
+static void WalRcvSigUsr2Handler(SIGNAL_ARGS);
 static void WalRcvShutdownHandler(SIGNAL_ARGS);
 static void WalRcvQuickDieHandler(SIGNAL_ARGS);
 
+static void
+WalRcvBlockSigUsr2(void)
+{
+	sigset_t mask;
+
+	sigemptyset(&mask);
+	sigaddset(&mask, SIGUSR2);
+	sigprocmask(SIG_BLOCK, &mask, NULL);
+}
+
+static void
+WalRcvUnblockSigUsr2(void)
+{
+	sigset_t mask;
+
+	sigemptyset(&mask);
+	sigaddset(&mask, SIGUSR2);
+	sigprocmask(SIG_UNBLOCK, &mask, NULL);
+}
 
 static void
 ProcessWalRcvInterrupts(void)
@@ -200,6 +221,7 @@ WalReceiverMain(void)
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz last_recv_timestamp;
 	bool		ping_sent;
+	bool		forceReply;
 
 	/*
 	 * WalRcv should be set up already (if we are a backend, we inherit this
@@ -268,7 +290,7 @@ WalReceiverMain(void)
 	pqsignal(SIGALRM, SIG_IGN);
 	pqsignal(SIGPIPE, SIG_IGN);
 	pqsignal(SIGUSR1, WalRcvSigUsr1Handler);
-	pqsignal(SIGUSR2, SIG_IGN);
+	pqsignal(SIGUSR2, WalRcvSigUsr2Handler);
 
 	/* Reset some signals that are accepted by postmaster but not here */
 	pqsignal(SIGCHLD, SIG_DFL);
@@ -299,6 +321,10 @@ WalReceiverMain(void)
 	/* Unblock signals (they were blocked when the postmaster forked us) */
 	PG_SETMASK(&UnBlockSig);
 
+	/* Block SIGUSR2 (we unblock it only during network waits). */
+	WalRcvBlockSigUsr2();
+	got_SIGUSR2 = false;
+
 	/* Establish the connection to the primary for XLOG streaming */
 	EnableWalRcvImmediateExit();
 	walrcv_connect(conninfo);
@@ -408,7 +434,9 @@ WalReceiverMain(void)
 				}
 
 				/* Wait a while for data to arrive */
+				WalRcvUnblockSigUsr2();
 				len = walrcv_receive(NAPTIME_PER_CYCLE, &buf);
+				WalRcvBlockSigUsr2();
 				if (len != 0)
 				{
 					/*
@@ -439,11 +467,21 @@ WalReceiverMain(void)
 							endofwal = true;
 							break;
 						}
+						WalRcvUnblockSigUsr2();
 						len = walrcv_receive(0, &buf);
+						WalRcvBlockSigUsr2();
+					}
+
+					if (got_SIGUSR2)
+					{
+						/* The recovery process asked us to force a reply. */
+						got_SIGUSR2 = false;
+						forceReply = true;
 					}
 
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(false, false);
+					XLogWalRcvSendReply(forceReply, false);
+					forceReply = false;
 
 					/*
 					 * If we've written some records, flush them to disk and
@@ -498,7 +536,14 @@ WalReceiverMain(void)
 						}
 					}
 
-					XLogWalRcvSendReply(requestReply, requestReply);
+					if (got_SIGUSR2)
+					{
+						/* The recovery process asked us to force a reply. */
+						got_SIGUSR2 = false;
+						forceReply = true;
+					}
+					XLogWalRcvSendReply(requestReply || forceReply, requestReply);
+					forceReply = false;
 					XLogWalRcvSendHSFeedback(false);
 				}
 			}
@@ -740,6 +785,13 @@ WalRcvSigUsr1Handler(SIGNAL_ARGS)
 	errno = save_errno;
 }
 
+/* SIGUSR2: used to receive wakeups from recovery */
+static void
+WalRcvSigUsr2Handler(SIGNAL_ARGS)
+{
+	got_SIGUSR2 = true;
+}
+
 /* SIGTERM: set flag for main loop, or shutdown immediately if safe */
 static void
 WalRcvShutdownHandler(SIGNAL_ARGS)
@@ -1222,6 +1274,22 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 }
 
 /*
+ * Wake up the walreceiver if it happens to be blocked in walrcv_receive,
+ * and tell it that a commit record has been applied.
+ *
+ * This is called by the startup process whenever interesting xlog records
+ * are applied, so that walreceiver can check if it needs to send an apply
+ * notification back to the master which may be waiting in a COMMIT with
+ * synchronous_commit = remote_apply.
+ */
+void
+WalRcvWakeup(void)
+{
+	if (WalRcv->pid != 0)
+		kill(WalRcv->pid, SIGUSR2);
+}
+
+/*
  * Return a string constant representing the state. This is used
  * in system functions and views, and should *not* be translated.
  */
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 65a6cd4..06cb166 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -345,12 +345,13 @@ static const struct config_enum_entry constraint_exclusion_options[] = {
 };
 
 /*
- * Although only "on", "off", "remote_write", and "local" are documented, we
- * accept all the likely variants of "on" and "off".
+ * Although only "on", "off", "remote_apply", "remote_write", and "local" are
+ * documented, we accept all the likely variants of "on" and "off".
  */
 static const struct config_enum_entry synchronous_commit_options[] = {
 	{"local", SYNCHRONOUS_COMMIT_LOCAL_FLUSH, false},
 	{"remote_write", SYNCHRONOUS_COMMIT_REMOTE_WRITE, false},
+	{"remote_apply", SYNCHRONOUS_COMMIT_REMOTE_APPLY, false},
 	{"on", SYNCHRONOUS_COMMIT_ON, false},
 	{"off", SYNCHRONOUS_COMMIT_OFF, false},
 	{"true", SYNCHRONOUS_COMMIT_ON, true},
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index 5536012..ec4427f 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -177,7 +177,7 @@
 					# (change requires restart)
 #fsync = on				# turns forced synchronization on or off
 #synchronous_commit = on		# synchronization level;
-					# off, local, remote_write, or on
+					# off, local, remote_write, remote_apply, or on
 #wal_sync_method = fsync		# the default is the first option
 					# supported by the operating system:
 					#   open_datasync
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index ebeb582..ed8d22c 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -60,7 +60,9 @@ typedef enum
 	SYNCHRONOUS_COMMIT_LOCAL_FLUSH,		/* wait for local flush only */
 	SYNCHRONOUS_COMMIT_REMOTE_WRITE,	/* wait for local flush and remote
 										 * write */
-	SYNCHRONOUS_COMMIT_REMOTE_FLUSH		/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_FLUSH,	/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_APPLY		/* wait for local flush and remote
+										 * apply */
 }	SyncCommitLevel;
 
 /* Define the default setting for synchonous_commit */
@@ -144,10 +146,13 @@ typedef void (*SubXactCallback) (SubXactEvent event, SubTransactionId mySubid,
  * EOXact... routines which run at the end of the original transaction
  * completion.
  */
+#define XACT_COMPLETION_SYNC_APPLY_FEEDBACK		(1U << 29)
 #define XACT_COMPLETION_UPDATE_RELCACHE_FILE	(1U << 30)
 #define XACT_COMPLETION_FORCE_SYNC_COMMIT		(1U << 31)
 
 /* Access macros for above flags */
+#define XactCompletionSyncApplyFeedback(xinfo) \
+	(!!(xinfo & XACT_COMPLETION_SYNC_APPLY_FEEDBACK))
 #define XactCompletionRelcacheInitFileInval(xinfo) \
 	(!!(xinfo & XACT_COMPLETION_UPDATE_RELCACHE_FILE))
 #define XactCompletionForceSyncCommit(xinfo) \
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 74a1394..a7dcdae 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -267,6 +267,8 @@ extern bool CheckPromoteSignal(void);
 extern void WakeupRecovery(void);
 extern void SetWalWriterSleeping(bool sleeping);
 
+extern void XLogRequestWalReceiverReply(void);
+
 extern void assign_max_wal_size(int newval, void *extra);
 extern void assign_checkpoint_completion_target(double newval, void *extra);
 
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index 96e059b..c005a42 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -23,8 +23,9 @@
 #define SYNC_REP_NO_WAIT		-1
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
+#define SYNC_REP_WAIT_APPLY		2
 
-#define NUM_SYNC_REP_WAIT_MODE	2
+#define NUM_SYNC_REP_WAIT_MODE	3
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
@@ -35,7 +36,7 @@
 extern char *SyncRepStandbyNames;
 
 /* called by user backend */
-extern void SyncRepWaitForLSN(XLogRecPtr XactCommitLSN);
+extern void SyncRepWaitForLSN(XLogRecPtr lsn, bool commit);
 
 /* called at backend exit */
 extern void SyncRepCleanupAtProcExit(void);
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 6eacb09..3294df9 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -162,5 +162,6 @@ extern void RequestXLogStreaming(TimeLineID tli, XLogRecPtr recptr,
 extern XLogRecPtr GetWalRcvWriteRecPtr(XLogRecPtr *latestChunkStart, TimeLineID *receiveTLI);
 extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
+extern void WalRcvWakeup(void);
 
 #endif   /* _WALRECEIVER_H */
0002-replay-lag-v5.patchapplication/octet-stream; name=0002-replay-lag-v5.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 105d541..7d63782 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1208,6 +1208,12 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
       standby server</entry>
     </row>
     <row>
+     <entry><structfield>replay_lag</></entry>
+     <entry><type>interval</></entry>
+     <entry>Estimated time taken for recent WAL records to be replayed on this
+      standby server</entry>
+    </row>
+    <row>
      <entry><structfield>sync_priority</></entry>
      <entry><type>integer</></entry>
      <entry>Priority of this standby server for being chosen as the
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 130b56b..48a5950 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -5473,6 +5473,12 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 		XLogFlush(lsn);
 
 	/*
+	 * Record the primary's timestamp for the commit record, so it can be used
+	 * for tracking replay lag.
+	 */
+	SetXLogReplayTimestamp(parsed->xact_time);
+
+	/*
 	 * If asked by the primary (because someone is waiting for a synchronous
 	 * commit = remote_apply), we will need to ask walreceiver to send a
 	 * reply immediately.
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 3e454f5..504b4df 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -81,6 +81,8 @@ extern uint32 bootstrap_data_checksum_version;
 #define PROMOTE_SIGNAL_FILE		"promote"
 #define FALLBACK_PROMOTE_SIGNAL_FILE "fallback_promote"
 
+/* Size of the circular buffer of timestamped LSNs. */
+#define MAX_TIMESTAMPED_LSNS 8192
 
 /* User-settable parameters */
 int			max_wal_size = 64;	/* 1 GB */
@@ -360,6 +362,13 @@ static bool doRequestWalReceiverReply;
  */
 static XLogRecPtr RedoStartLSN = InvalidXLogRecPtr;
 
+/*
+ * LastReplayedTimestamp can be set by redo handlers when they apply a record
+ * that carries a timestamp, by calling SetXLogReplayedTimestamp.  The xlog
+ * apply loop can then update the value in shared memory.
+ */
+static TimestampTz LastReplayedTimestamp = 0;
+
 /*----------
  * Shared-memory data structures for XLOG control
  *
@@ -634,6 +643,21 @@ typedef struct XLogCtlData
 	/* current effective recovery target timeline */
 	TimeLineID	RecoveryTargetTLI;
 
+	/* timestamp from the most recently applied record associated with a timestamp. */
+	TimestampTz lastReplayedTimestamp;
+
+	/*
+	 * We maintain a circular buffer of LSNs and associated timestamps.
+	 * Walreceiver writes into it using information from timestamps, and the
+	 * startup recovery process reads from it and notifies walreceiver when
+	 * LSNs are replayed so that the timestamps can eventually be fed back to
+	 * the upstream server, to track lag.
+	 */
+	Index			timestampedLsnRead;
+	Index			timestampedLsnWrite;
+	XLogRecPtr		timestampedLsn[MAX_TIMESTAMPED_LSNS];
+	TimestampTz		timestampedLsnTime[MAX_TIMESTAMPED_LSNS];
+
 	/*
 	 * timestamp of when we started replaying the current chunk of WAL data,
 	 * only relevant for replication or archive recovery
@@ -6874,20 +6898,51 @@ StartupXLOG(void)
 				error_context_stack = errcallback.previous;
 
 				/*
-				 * Update lastReplayedEndRecPtr after this record has been
-				 * successfully replayed.
+				 * Update lastReplayedEndRecPtr and lastReplayedTimestamp
+				 * after this record has been successfully replayed.
 				 */
 				SpinLockAcquire(&XLogCtl->info_lck);
 				XLogCtl->lastReplayedEndRecPtr = EndRecPtr;
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
+				if (LastReplayedTimestamp != 0)
+				{
+					/* If replaying a record produced a timestamp, use that. */
+					XLogCtl->lastReplayedTimestamp = LastReplayedTimestamp;
+					LastReplayedTimestamp = 0;
+				}
+				else
+				{
+					/*
+					 * If we have applied LSNs associated with timestamps
+					 * received by walreceiver, then use the recorded
+					 * timestamp.  We consume from the read end of the
+					 * circular buffer.
+					 */
+					while (XLogCtl->timestampedLsnRead !=
+						   XLogCtl->timestampedLsnWrite &&
+						   XLogCtl->timestampedLsn[XLogCtl->timestampedLsnRead]
+						   <= EndRecPtr)
+					{
+						if (XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead] >
+							XLogCtl->lastReplayedTimestamp)
+						{
+							XLogCtl->lastReplayedTimestamp =
+								XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead];
+							doRequestWalReceiverReply = true;
+						}
+						XLogCtl->timestampedLsnRead =
+							(XLogCtl->timestampedLsnRead + 1) % MAX_TIMESTAMPED_LSNS;
+					}
+				}
 				SpinLockRelease(&XLogCtl->info_lck);
 
 				/*
 				 * If rm_redo reported that it applied a commit record that
 				 * the master is waiting for by calling
-				 * XLogRequestWalReceiverReply, then we wake up the receiver
-				 * so that it notices the updated lastReplayedEndRecPtr and
-				 * sends a reply to the master.
+				 * XLogRequestWalReceiverReply, or we encountered a WAL
+				 * location that was associated with a timestamp above, then
+				 * we wake up the receiver so that it notices the updated
+				 * lastReplayedEndRecPtr and sends a reply to the master.
 				 */
 				if (doRequestWalReceiverReply)
 				{
@@ -11619,3 +11674,91 @@ XLogRequestWalReceiverReply(void)
 {
 	doRequestWalReceiverReply = true;
 }
+
+/*
+ * Record the timestamp that is associated with a WAL position.
+ *
+ * This is called by walreceiver on standby servers when keepalive messages
+ * arrive, using timestamps generated on the primary server.  The timestamp
+ * will be sent back to the primary server when the standby had applied this
+ * WAL position.  The primary can use the elapsed time to estimate the replay
+ * lag.
+ */
+void
+SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn)
+{
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn == XLogCtl->lastReplayedEndRecPtr)
+	{
+		/*
+		 * That is the last replayed LSN: we are fully replayed, so we can
+		 * update the replay timestamp immediately.
+		 */
+		XLogCtl->lastReplayedTimestamp = timestamp;
+	}
+	else
+	{
+		/*
+		 * There is WAL still to be applied.  We will associate the timestamp
+		 * with this WAL position and wait for it to be replayed.  We add it
+		 * at the 'write' end of the circular buffer of LSN/timestamp
+		 * mappings, which the replay loop will eventually read.
+		 */
+		Index w = XLogCtl->timestampedLsnWrite;
+		Index r = XLogCtl->timestampedLsnRead;
+
+		XLogCtl->timestampedLsn[w] = lsn;
+		XLogCtl->timestampedLsnTime[w] = timestamp;
+
+		/* Advance the write point. */
+		w = (w + 1) % MAX_TIMESTAMPED_LSNS;
+		XLogCtl->timestampedLsnWrite = w;
+		if (w == r)
+		{
+			/*
+			 * The buffer is full.  Advance the read point (throwing away
+			 * oldest values; we will begin to overestimate replay lag, until
+			 * lag decreases to a size our buffer can manage, or the next
+			 * commit record is replayed).
+			 */
+			r = (r + 1) % MAX_TIMESTAMPED_LSNS;
+			XLogCtl->timestampedLsnRead = r;
+		}
+	}
+	SpinLockRelease(&XLogCtl->info_lck);
+}
+
+/*
+ * Set the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary.  This can be called by redo handlers that have
+ * an appropriate timestamp (currently only commit records).  Updating the
+ * shared memory value is deferred until after the redo handler returns.
+ */
+void
+SetXLogReplayTimestamp(TimestampTz timestamp)
+{
+	LastReplayedTimestamp = timestamp;
+}
+
+/*
+ * Get the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary, and also the most recently applied LSN.  (Note
+ * that the timestamp and the LSN don't necessarily relate to the same
+ * record.)
+ *
+ * This is similar to GetLatestXTime, except that it is not only advanced by
+ * commit records (see SetXLogReplayTimestampAtLsn).
+ */
+TimestampTz
+GetXLogReplayTimestamp(XLogRecPtr *lsn)
+{
+	TimestampTz result;
+
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn)
+		*lsn = XLogCtl->lastReplayedEndRecPtr;
+	result = XLogCtl->lastReplayedTimestamp;
+	SpinLockRelease(&XLogCtl->info_lck);
+
+	return result;
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 9ae1ef4..a53f07b 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -662,6 +662,7 @@ CREATE VIEW pg_stat_replication AS
             W.write_location,
             W.flush_location,
             W.replay_location,
+            W.replay_lag,
             W.sync_priority,
             W.sync_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 59e65c2..22bff89 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -85,6 +85,8 @@ walrcv_disconnect_type walrcv_disconnect = NULL;
 
 #define NAPTIME_PER_CYCLE 100	/* max sleep time between cycles (100ms) */
 
+#define MIN_TIME_BETWEEN_TIMESTAMPED_LSNS 1000 /* 1s */
+
 /*
  * These variables are used similarly to openLogFile/SegNo/Off,
  * but for walreceiver to write the XLOG. recvFileTLI is the TimeLineID
@@ -103,6 +105,8 @@ static volatile sig_atomic_t got_SIGHUP = false;
 static volatile sig_atomic_t got_SIGTERM = false;
 static volatile sig_atomic_t got_SIGUSR2 = false;
 
+static bool reply_sent_on_recovery_activity = false;
+
 /*
  * LogstreamResult indicates the byte positions that we have already
  * written/fsynced.
@@ -144,7 +148,7 @@ static void WalRcvDie(int code, Datum arg);
 static void XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len);
 static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
-static void XLogWalRcvSendReply(bool force, bool requestReply);
+static void XLogWalRcvSendReply(bool force, bool requestReply, bool includeApplyTimestamp);
 static void XLogWalRcvSendHSFeedback(bool immed);
 static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
 
@@ -221,7 +225,7 @@ WalReceiverMain(void)
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz last_recv_timestamp;
 	bool		ping_sent;
-	bool		forceReply;
+	bool		timestampedWalApplied;
 
 	/*
 	 * WalRcv should be set up already (if we are a backend, we inherit this
@@ -474,14 +478,15 @@ WalReceiverMain(void)
 
 					if (got_SIGUSR2)
 					{
-						/* The recovery process asked us to force a reply. */
+						/* The recovery process asked us to report an applied timestamp. */
 						got_SIGUSR2 = false;
-						forceReply = true;
+						timestampedWalApplied = true;
+						reply_sent_on_recovery_activity = true;
 					}
 
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(forceReply, false);
-					forceReply = false;
+					XLogWalRcvSendReply(timestampedWalApplied, false, timestampedWalApplied);
+					timestampedWalApplied = false;
 
 					/*
 					 * If we've written some records, flush them to disk and
@@ -538,12 +543,14 @@ WalReceiverMain(void)
 
 					if (got_SIGUSR2)
 					{
-						/* The recovery process asked us to force a reply. */
+						/* The recovery process asked us to report an apply timestamp. */
 						got_SIGUSR2 = false;
-						forceReply = true;
+						timestampedWalApplied = true;
+						reply_sent_on_recovery_activity = true;
 					}
-					XLogWalRcvSendReply(requestReply || forceReply, requestReply);
-					forceReply = false;
+					XLogWalRcvSendReply(requestReply || timestampedWalApplied, requestReply,
+										timestampedWalApplied);
+					timestampedWalApplied = false;
 					XLogWalRcvSendHSFeedback(false);
 				}
 			}
@@ -881,6 +888,8 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 			}
 		case 'k':				/* Keepalive */
 			{
+				bool reportApplyTimestamp = false;
+
 				/* copy message to StringInfo */
 				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
 				if (len != hdrlen)
@@ -897,9 +906,22 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 
 				ProcessWalSndrMessage(walEnd, sendTime);
 
-				/* If the primary requested a reply, send one immediately */
-				if (replyRequested)
-					XLogWalRcvSendReply(true, false);
+				/*
+				 * If no apply timestamps have been sent at the request of the
+				 * recovery process since we last received a keepalive, then
+				 * we will send one now.  This allows us to feed back
+				 * timestamps in response to pings if we are idle or if the
+				 * recovery process is somehow blocked, but we don't want to
+				 * do that if it's actively applying and periodically waking
+				 * us up.
+				 */
+				if (!reply_sent_on_recovery_activity)
+					reportApplyTimestamp = true;
+				reply_sent_on_recovery_activity = false;
+
+				/* If the primary requested a reply, send one immediately. */
+				if (replyRequested || reportApplyTimestamp)
+					XLogWalRcvSendReply(true, false, reportApplyTimestamp);
 				break;
 			}
 		default:
@@ -1062,7 +1084,7 @@ XLogWalRcvFlush(bool dying)
 		/* Also let the master know that we made some progress */
 		if (!dying)
 		{
-			XLogWalRcvSendReply(false, false);
+			XLogWalRcvSendReply(false, false, false);
 			XLogWalRcvSendHSFeedback(false);
 		}
 	}
@@ -1080,15 +1102,20 @@ XLogWalRcvFlush(bool dying)
  * If 'requestReply' is true, requests the server to reply immediately upon
  * receiving this message. This is used for heartbearts, when approaching
  * wal_receiver_timeout.
+ *
+ * If 'reportApplyTimestamp' is true, the latest apply timestamp is included.
+ * This is set to true only when this function is called after the recovery
+ * process has replayed a record with an associated timestamp.
  */
 static void
-XLogWalRcvSendReply(bool force, bool requestReply)
+XLogWalRcvSendReply(bool force, bool requestReply, bool reportApplyTimestamp)
 {
 	static XLogRecPtr writePtr = 0;
 	static XLogRecPtr flushPtr = 0;
 	XLogRecPtr	applyPtr;
 	static TimestampTz sendTime = 0;
 	TimestampTz now;
+	TimestampTz applyTimestamp = 0;
 
 	/*
 	 * If the user doesn't want status to be reported to the master, be sure
@@ -1104,10 +1131,8 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	 * We can compare the write and flush positions to the last message we
 	 * sent without taking any lock, but the apply position requires a spin
 	 * lock, so we don't check that unless something else has changed or 10
-	 * seconds have passed.  This means that the apply log position will
-	 * appear, from the master's point of view, to lag slightly, but since
-	 * this is only for reporting purposes and only on idle systems, that's
-	 * probably OK.
+	 * seconds have passed, or the force flag has been set (which happens when
+	 * apply feedback has been requested by the primary).
 	 */
 	if (!force
 		&& writePtr == LogstreamResult.Write
@@ -1120,7 +1145,10 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	/* Construct a new message */
 	writePtr = LogstreamResult.Write;
 	flushPtr = LogstreamResult.Flush;
-	applyPtr = GetXLogReplayRecPtr(NULL);
+	if (reportApplyTimestamp)
+		applyTimestamp = GetXLogReplayTimestamp(&applyPtr);
+	else
+		applyPtr = GetXLogReplayRecPtr(NULL);
 
 	resetStringInfo(&reply_message);
 	pq_sendbyte(&reply_message, 'r');
@@ -1128,6 +1156,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	pq_sendint64(&reply_message, flushPtr);
 	pq_sendint64(&reply_message, applyPtr);
 	pq_sendint64(&reply_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&reply_message, TimestampTzToIntegerTimestamp(applyTimestamp));
 	pq_sendbyte(&reply_message, requestReply ? 1 : 0);
 
 	/* Send it */
@@ -1232,8 +1261,8 @@ static void
 ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 {
 	WalRcvData *walrcv = WalRcv;
-
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
+	static TimestampTz lastRecordedTimestamp = 0;
 
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
@@ -1244,6 +1273,18 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
 	SpinLockRelease(&walrcv->mutex);
 
+	/*
+	 * Remember primary's timestamp at this WAL location.  We throw away
+	 * samples if they are coming too fast because we don't want to fill up
+	 * the finite circular buffer and have to throw away older samples.
+	 */
+	if (lastRecordedTimestamp < TimestampTzPlusMilliseconds(sendTime,
+															-MIN_TIME_BETWEEN_TIMESTAMPED_LSNS))
+	{
+		SetXLogReplayTimestampAtLsn(sendTime, walEnd);
+		lastRecordedTimestamp = sendTime;
+	}
+
 	if (log_min_messages <= DEBUG2)
 	{
 		char	   *sendtime;
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index f98475c..16d7abc 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1545,15 +1545,29 @@ ProcessStandbyReplyMessage(void)
 	XLogRecPtr	writePtr,
 				flushPtr,
 				applyPtr;
+	int64		applyLagUs;
 	bool		replyRequested;
+	TimestampTz now = GetCurrentTimestamp();
+	TimestampTz applyTimestamp;
 
 	/* the caller already consumed the msgtype byte */
 	writePtr = pq_getmsgint64(&reply_message);
 	flushPtr = pq_getmsgint64(&reply_message);
 	applyPtr = pq_getmsgint64(&reply_message);
 	(void) pq_getmsgint64(&reply_message);		/* sendTime; not used ATM */
+	applyTimestamp = IntegerTimestampToTimestampTz(pq_getmsgint64(&reply_message));
 	replyRequested = pq_getmsgbyte(&reply_message);
 
+	/* Compute the apply lag in milliseconds. */
+	if (applyTimestamp == 0)
+		applyLagUs = -1;
+	else
+#ifdef HAVE_INT64_TIMESTAMP
+		applyLagUs = now - applyTimestamp;
+#else
+		applyLagUs = (now - applyTimestamp) * 1000000;
+#endif
+
 	elog(DEBUG2, "write %X/%X flush %X/%X apply %X/%X%s",
 		 (uint32) (writePtr >> 32), (uint32) writePtr,
 		 (uint32) (flushPtr >> 32), (uint32) flushPtr,
@@ -1575,6 +1589,8 @@ ProcessStandbyReplyMessage(void)
 		walsnd->write = writePtr;
 		walsnd->flush = flushPtr;
 		walsnd->apply = applyPtr;
+		if (applyLagUs >= 0)
+			walsnd->applyLagUs = applyLagUs;
 		SpinLockRelease(&walsnd->mutex);
 	}
 
@@ -2745,7 +2761,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	8
+#define PG_STAT_GET_WAL_SENDERS_COLS	9
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2793,6 +2809,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		XLogRecPtr	write;
 		XLogRecPtr	flush;
 		XLogRecPtr	apply;
+		int64		applyLagUs;
 		int			priority;
 		WalSndState state;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
@@ -2807,6 +2824,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
+		applyLagUs = walsnd->applyLagUs;
 		priority = walsnd->sync_standby_priority;
 		SpinLockRelease(&walsnd->mutex);
 
@@ -2841,6 +2859,23 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				nulls[5] = true;
 			values[5] = LSNGetDatum(apply);
 
+			if (applyLagUs < 0)
+				nulls[6] = true;
+			else
+			{
+				Interval *applyLagInterval = palloc(sizeof(Interval));
+
+				applyLagInterval->month = 0;
+				applyLagInterval->day = 0;
+#ifdef HAVE_INT64_TIMESTAMP
+				applyLagInterval->time = applyLagUs;
+#else
+				applyLagInterval->time = applyLagUs / 1000000.0;
+#endif
+				nulls[6] = false;
+				values[6] = IntervalPGetDatum(applyLagInterval);
+			}
+
 			/*
 			 * Treat a standby such as a pg_basebackup background process
 			 * which always returns an invalid flush location, as an
@@ -2848,18 +2883,18 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 			 */
 			priority = XLogRecPtrIsInvalid(walsnd->flush) ? 0 : priority;
 
-			values[6] = Int32GetDatum(priority);
+			values[7] = Int32GetDatum(priority);
 
 			/*
 			 * More easily understood version of standby state. This is purely
 			 * informational, not different from priority.
 			 */
 			if (priority == 0)
-				values[7] = CStringGetTextDatum("async");
+				values[8] = CStringGetTextDatum("async");
 			else if (walsnd == sync_standby)
-				values[7] = CStringGetTextDatum("sync");
+				values[8] = CStringGetTextDatum("sync");
 			else
-				values[7] = CStringGetTextDatum("potential");
+				values[8] = CStringGetTextDatum("potential");
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
diff --git a/src/backend/utils/adt/timestamp.c b/src/backend/utils/adt/timestamp.c
index c9e5270..f382b20 100644
--- a/src/backend/utils/adt/timestamp.c
+++ b/src/backend/utils/adt/timestamp.c
@@ -1629,6 +1629,20 @@ IntegerTimestampToTimestampTz(int64 timestamp)
 #endif
 
 /*
+ * TimestampTzToIntegerTimestamp -- convert a native timestamp to int64 format
+ *
+ * When compiled with --enable-integer-datetimes, this is implemented as a
+ * no-op macro.
+ */
+#ifndef HAVE_INT64_TIMESTAMP
+int64
+TimestampTzToIntegerTimestamp(TimestampTz timestamp)
+{
+	return timestamp * 1000000;
+}
+#endif
+
+/*
  * TimestampDifference -- convert the difference between two timestamps
  *		into integer seconds and microseconds
  *
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index a7dcdae..c8be3ce 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -235,6 +235,9 @@ extern void GetXLogReceiptTime(TimestampTz *rtime, bool *fromStream);
 extern XLogRecPtr GetXLogReplayRecPtr(TimeLineID *replayTLI);
 extern XLogRecPtr GetXLogInsertRecPtr(void);
 extern XLogRecPtr GetXLogWriteRecPtr(void);
+extern void SetXLogReplayTimestamp(TimestampTz timestamp);
+extern void SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn);
+extern TimestampTz GetXLogReplayTimestamp(XLogRecPtr *lsn);
 extern bool RecoveryIsPaused(void);
 extern void SetRecoveryPause(bool recoveryPause);
 extern TimestampTz GetLatestXTime(void);
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index a595327..4054726 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2712,7 +2712,7 @@ DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f
 DESCR("statistics: information about currently active backends");
 DATA(insert OID = 3318 (  pg_stat_get_progress_info           PGNSP PGUID 12 1 100 0 0 f f f f t t s r 1 0 2249 "25" "{25,23,26,26,20,20,20,20,20,20,20,20,20,20}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{cmdtype,pid,datid,relid,param1,param2,param3,param4,param5,param6,param7,param8,param9,param10}" _null_ _null_ pg_stat_get_progress_info _null_ _null_ _null_ ));
 DESCR("statistics: information about progress of backends running maintenance command");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,23,25}" "{o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25}" "{o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 7794aa5..4de43e8 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -46,6 +46,7 @@ typedef struct WalSnd
 	XLogRecPtr	write;
 	XLogRecPtr	flush;
 	XLogRecPtr	apply;
+	int64		applyLagUs;
 
 	/* Protects shared variables shown above. */
 	slock_t		mutex;
diff --git a/src/include/utils/timestamp.h b/src/include/utils/timestamp.h
index fbead3a..297e151 100644
--- a/src/include/utils/timestamp.h
+++ b/src/include/utils/timestamp.h
@@ -227,9 +227,11 @@ extern bool TimestampDifferenceExceeds(TimestampTz start_time,
 #ifndef HAVE_INT64_TIMESTAMP
 extern int64 GetCurrentIntegerTimestamp(void);
 extern TimestampTz IntegerTimestampToTimestampTz(int64 timestamp);
+extern int64 TimestampTzToIntegerTimestamp(TimestampTz timestamp);
 #else
 #define GetCurrentIntegerTimestamp()	GetCurrentTimestamp()
 #define IntegerTimestampToTimestampTz(timestamp) (timestamp)
+#define TimestampTzToIntegerTimestamp(timestamp) (timestamp)
 #endif
 
 extern TimestampTz time_t_to_timestamptz(pg_time_t tm);
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 79f9b23..fc4b765 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1783,11 +1783,12 @@ pg_stat_replication| SELECT s.pid,
     w.write_location,
     w.flush_location,
     w.replay_location,
+    w.replay_lag,
     w.sync_priority,
     w.sync_state
    FROM pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, wait_event_type, wait_event, xact_start, query_start, backend_start, state_change, client_addr, client_hostname, client_port, backend_xid, backend_xmin, ssl, sslversion, sslcipher, sslbits, sslcompression, sslclientdn),
     pg_authid u,
-    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, sync_priority, sync_state)
+    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state)
   WHERE ((s.usesysid = u.oid) AND (s.pid = w.pid));
 pg_stat_ssl| SELECT s.pid,
     s.ssl,
#53Robert Haas
robertmhaas@gmail.com
In reply to: Thomas Munro (#52)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Wed, Mar 23, 2016 at 7:34 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

synchronous_commit TPS
==================== ====
off 9234
local 1223
remote_write 907
on 587
remote_apply 555

synchronous_commit TPS
==================== ====
off 3937
local 1984
remote_write 1701
on 1373
remote_apply 1349

Hmm, so "remote_apply" is barely more expensive than "on". That's encouraging.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#54Michael Paquier
michael.paquier@gmail.com
In reply to: Robert Haas (#53)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Wed, Mar 23, 2016 at 8:39 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Wed, Mar 23, 2016 at 7:34 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

synchronous_commit TPS
==================== ====
off 9234
local 1223
remote_write 907
on 587
remote_apply 555

synchronous_commit TPS
==================== ====
off 3937
local 1984
remote_write 1701
on 1373
remote_apply 1349

Hmm, so "remote_apply" is barely more expensive than "on". That's encouraging.

Indeed, interesting. This is perhaps proving that just having the
possibility to have remote_apply (with feedback messages managed by
signals, which is the best thing proposed for this release) is what we
need to ensure the consistency of reads across nodes, and what would
satisfy most of the user's requirements. Getting a slightly lower TPS
may be worth the cost for some users if they can ensure that reads
across nodes are accessible after a local commit, and there is no need
to have an error management layer at application level to take care of
errors caused by causal read timeouts.
--
Michael

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#55Robert Haas
robertmhaas@gmail.com
In reply to: Michael Paquier (#54)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Wed, Mar 23, 2016 at 8:43 AM, Michael Paquier
<michael.paquier@gmail.com> wrote:

On Wed, Mar 23, 2016 at 8:39 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Wed, Mar 23, 2016 at 7:34 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

synchronous_commit TPS
==================== ====
off 9234
local 1223
remote_write 907
on 587
remote_apply 555

synchronous_commit TPS
==================== ====
off 3937
local 1984
remote_write 1701
on 1373
remote_apply 1349

Hmm, so "remote_apply" is barely more expensive than "on". That's encouraging.

Indeed, interesting. This is perhaps proving that just having the
possibility to have remote_apply (with feedback messages managed by
signals, which is the best thing proposed for this release) is what we
need to ensure the consistency of reads across nodes, and what would
satisfy most of the user's requirements. Getting a slightly lower TPS
may be worth the cost for some users if they can ensure that reads
across nodes are accessible after a local commit, and there is no need
to have an error management layer at application level to take care of
errors caused by causal read timeouts.

Well, I wouldn't go that far. It seems pretty clear that remote_apply
by itself is useful - I can't imagine anybody seriously arguing the
contrary, whatever they think of this implementation. My view,
though, is that by itself that's pretty limiting: you can only have
one standby, and if that standby falls over then you lose
availability. Causal reads fixes both of those problems - admittedly
that requires some knowledge in the application or the pooler, but
it's no worse than SSI in that regard. Still, half a loaf is better
than none, and I imagine even just getting remote_apply would make a
few people quite happy.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#56Michael Paquier
michael.paquier@gmail.com
In reply to: Robert Haas (#55)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Wed, Mar 23, 2016 at 11:32 PM, Robert Haas <robertmhaas@gmail.com> wrote:

Well, I wouldn't go that far. It seems pretty clear that remote_apply
by itself is useful - I can't imagine anybody seriously arguing the
contrary, whatever they think of this implementation. My view,
though, is that by itself that's pretty limiting: you can only have
one standby, and if that standby falls over then you lose
availability. Causal reads fixes both of those problems - admittedly
that requires some knowledge in the application or the pooler, but
it's no worse than SSI in that regard. Still, half a loaf is better
than none, and I imagine even just getting remote_apply would make a
few people quite happy.

OK, let's do so then, even if causal reads don't get into 9.6 users
could get advantage of remote_apply on multiple nodes if the N-sync
patch gets in.

Just looking at 0001.

-        <literal>remote_write</>, <literal>local</>, and <literal>off</>.
+        <literal>remote_write</>, <literal>remote_apply</>,
<literal>local</>, and <literal>off</>.
         The default, and safe, setting
I imagine that a run of pgindent would be welcome for such large lines.
+#define XactCompletionSyncApplyFeedback(xinfo) \
+   (!!(xinfo & XACT_COMPLETION_SYNC_APPLY_FEEDBACK))
That's not directly something this patch should take care of, but the
notation "!!" has better be avoided (see stdbool thread with VS2015).
-   SyncRepWaitForLSN(gxact->prepare_end_lsn);
+   SyncRepWaitForLSN(gxact->prepare_end_lsn, false);
Isn't it important to ensure that a PREPARE LSN is applied as well on
the standby with remote_apply? Say if an application prepares a
transaction, it would commit locally but its LSN may not be applied on
the standby with this patch. That would be a surprising behavior for
the user.

(not commenting on the latch and SIGUSR2 handling, you are still
working on it per your last update).
--
Michael

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#57Robert Haas
robertmhaas@gmail.com
In reply to: Michael Paquier (#56)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Thu, Mar 24, 2016 at 2:11 AM, Michael Paquier
<michael.paquier@gmail.com> wrote:

On Wed, Mar 23, 2016 at 11:32 PM, Robert Haas <robertmhaas@gmail.com> wrote:

Well, I wouldn't go that far. It seems pretty clear that remote_apply
by itself is useful - I can't imagine anybody seriously arguing the
contrary, whatever they think of this implementation. My view,
though, is that by itself that's pretty limiting: you can only have
one standby, and if that standby falls over then you lose
availability. Causal reads fixes both of those problems - admittedly
that requires some knowledge in the application or the pooler, but
it's no worse than SSI in that regard. Still, half a loaf is better
than none, and I imagine even just getting remote_apply would make a
few people quite happy.

OK, let's do so then, even if causal reads don't get into 9.6 users
could get advantage of remote_apply on multiple nodes if the N-sync
patch gets in.

Just looking at 0001.

-        <literal>remote_write</>, <literal>local</>, and <literal>off</>.
+        <literal>remote_write</>, <literal>remote_apply</>,
<literal>local</>, and <literal>off</>.
The default, and safe, setting
I imagine that a run of pgindent would be welcome for such large lines.

I didn't think pgindent touched the docs. But I agree lines over 80
characters should be wrapped if they're being modified anyway.

+#define XactCompletionSyncApplyFeedback(xinfo) \
+   (!!(xinfo & XACT_COMPLETION_SYNC_APPLY_FEEDBACK))
That's not directly something this patch should take care of, but the
notation "!!" has better be avoided (see stdbool thread with VS2015).

+1.

-   SyncRepWaitForLSN(gxact->prepare_end_lsn);
+   SyncRepWaitForLSN(gxact->prepare_end_lsn, false);
Isn't it important to ensure that a PREPARE LSN is applied as well on
the standby with remote_apply? Say if an application prepares a
transaction, it would commit locally but its LSN may not be applied on
the standby with this patch. That would be a surprising behavior for
the user.

You need to wait for COMMIT PREPARED, but I don't see why you need to
wait for PREPARE itself.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#58Michael Paquier
michael.paquier@gmail.com
In reply to: Robert Haas (#57)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Thu, Mar 24, 2016 at 11:20 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Thu, Mar 24, 2016 at 2:11 AM, Michael Paquier
<michael.paquier@gmail.com> wrote:

-   SyncRepWaitForLSN(gxact->prepare_end_lsn);
+   SyncRepWaitForLSN(gxact->prepare_end_lsn, false);
Isn't it important to ensure that a PREPARE LSN is applied as well on
the standby with remote_apply? Say if an application prepares a
transaction, it would commit locally but its LSN may not be applied on
the standby with this patch. That would be a surprising behavior for
the user.

You need to wait for COMMIT PREPARED, but I don't see why you need to
wait for PREPARE itself.

Multi-master conflict resolution. Knowing in-time that a prepared
transaction has been applied is useful on another node for lock
conflicts resolution. Say a PRIMARY KEY insert is prepared, and we
want to know at application level that its prepared state is visible
everywhere.
--
Michael

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#59Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Thomas Munro (#52)
4 attachment(s)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Thu, Mar 24, 2016 at 12:34 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Wed, Mar 23, 2016 at 12:37 PM, Robert Haas <robertmhaas@gmail.com> wrote:

+static void WalRcvUnblockSigUsr2(void)

And again here.

Fixed.

+                WalRcvUnblockSigUsr2();
len = walrcv_receive(NAPTIME_PER_CYCLE, &buf);
+                WalRcvBlockSigUsr2();

This does not seem like it will be cheap on all operating systems. I
think you should try to rejigger this somehow so that it can just set
the process latch and the wal receiver figures it out from looking at
shared memory. Like maybe a flag in WalRcvData? An advantage of this
is that it should cut down on the number of signals significantly,
because it won't need to send SIGUSR1 when the latch is already set.

Still experimenting with a latch here. I will come back on this point soon.

Here is a latch-based version.

On Thu, Mar 24, 2016 at 7:11 PM, Michael Paquier
<michael.paquier@gmail.com> wrote:

Just looking at 0001.

-        <literal>remote_write</>, <literal>local</>, and <literal>off</>.
+        <literal>remote_write</>, <literal>remote_apply</>,
<literal>local</>, and <literal>off</>.
The default, and safe, setting
I imagine that a run of pgindent would be welcome for such large lines.

Fixed.

+#define XactCompletionSyncApplyFeedback(xinfo) \
+   (!!(xinfo & XACT_COMPLETION_SYNC_APPLY_FEEDBACK))
That's not directly something this patch should take care of, but the
notation "!!" has better be avoided (see stdbool thread with VS2015).

Changed.

-   SyncRepWaitForLSN(gxact->prepare_end_lsn);
+   SyncRepWaitForLSN(gxact->prepare_end_lsn, false);
Isn't it important to ensure that a PREPARE LSN is applied as well on
the standby with remote_apply? Say if an application prepares a
transaction, it would commit locally but its LSN may not be applied on
the standby with this patch. That would be a surprising behavior for
the user.

My reasoning here was that this isn't a commit, so you shouldn't wait
for it to be applied (just like we don't wait for any other
non-committed stuff to be applied), because it has no user-visible
effect other than the ability to COMMIT PREPARED on the standby if it
is promoted after that point. For that reason I do wait for it to be
flushed. After it is flushed, it is guaranteed to be applied some
time before the recovery completes and a user could potentially run
COMMIT PREPARED on the newly promoted master.

--
Thomas Munro
http://www.enterprisedb.com

Attachments:

0001-remote-apply-v6.patchapplication/octet-stream; name=0001-remote-apply-v6.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index d48a13f..6ab65a7 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -2143,8 +2143,8 @@ include_dir 'conf.d'
         Specifies whether transaction commit will wait for WAL records
         to be written to disk before the command returns a <quote>success</>
         indication to the client.  Valid values are <literal>on</>,
-        <literal>remote_write</>, <literal>local</>, and <literal>off</>.
-        The default, and safe, setting
+        <literal>remote_write</>, <literal>remote_apply</>, <literal>local</>,
+        and <literal>off</>.  The default, and safe, setting
         is <literal>on</>.  When <literal>off</>, there can be a delay between
         when success is reported to the client and when the transaction is
         really guaranteed to be safe against a server crash.  (The maximum
@@ -2177,6 +2177,10 @@ include_dir 'conf.d'
         ensure data preservation even if the standby instance of
         <productname>PostgreSQL</> were to crash, but not if the standby
         suffers an operating-system-level crash.
+        When set to <literal>remote_apply</>, commits will wait until a reply
+        from the current synchronous standby indicates it has received the
+        commit record of the transaction and applied it, so that it has become
+        visible to queries.
        </para>
        <para>
         When synchronous
diff --git a/doc/src/sgml/high-availability.sgml b/doc/src/sgml/high-availability.sgml
index 19d613e..03c6c30 100644
--- a/doc/src/sgml/high-availability.sgml
+++ b/doc/src/sgml/high-availability.sgml
@@ -1081,6 +1081,9 @@ primary_slot_name = 'node_a_slot'
     WAL record is then sent to the standby. The standby sends reply
     messages each time a new batch of WAL data is written to disk, unless
     <varname>wal_receiver_status_interval</> is set to zero on the standby.
+    In the case that <varname>synchronous_commit</> is set to
+    <literal>remote_apply</>, the standby sends reply messages when the commit
+    record is replayed, making the transaction visible.
     If the standby is the first matching standby, as specified in
     <varname>synchronous_standby_names</> on the primary, the reply
     messages from that standby will be used to wake users waiting for
@@ -1107,6 +1110,14 @@ primary_slot_name = 'node_a_slot'
    </para>
 
    <para>
+    Setting <varname>synchronous_commit</> to <literal>remote_apply</> will
+    cause each commit to wait until the current synchronous standby reports
+    that it has replayed the transaction, making it visible to user queries.
+    In simple cases, this allows for load balancing with causal consistency
+    on a single hot standby.
+   </para>
+
+   <para>
     Users will stop waiting if a fast shutdown is requested.  However, as
     when using asynchronous replication, the server will not fully
     shutdown until all outstanding WAL records are transferred to the currently
@@ -1160,9 +1171,10 @@ primary_slot_name = 'node_a_slot'
     <title>Planning for High Availability</title>
 
    <para>
-    Commits made when <varname>synchronous_commit</> is set to <literal>on</>
-    or <literal>remote_write</> will wait until the synchronous standby responds. The response
-    may never occur if the last, or only, standby should crash.
+    Commits made when <varname>synchronous_commit</> is set to <literal>on</>,
+    <literal>remote_write</> or <literal>remote_apply</> will wait until the
+    synchronous standby responds. The response may never occur if the last, or
+    only, standby should crash.
    </para>
 
    <para>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index e7234c8..893c2fa 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -1107,7 +1107,7 @@ EndPrepare(GlobalTransaction gxact)
 	 * Note that at this stage we have marked the prepare, but still show as
 	 * running in the procarray (twice!) and continue to hold locks.
 	 */
-	SyncRepWaitForLSN(gxact->prepare_end_lsn);
+	SyncRepWaitForLSN(gxact->prepare_end_lsn, false);
 
 	records.tail = records.head = NULL;
 	records.num_chunks = 0;
@@ -2103,7 +2103,7 @@ RecordTransactionCommitPrepared(TransactionId xid,
 	 * Note that at this stage we have marked clog, but still show as running
 	 * in the procarray and continue to hold locks.
 	 */
-	SyncRepWaitForLSN(recptr);
+	SyncRepWaitForLSN(recptr, true);
 }
 
 /*
@@ -2156,5 +2156,5 @@ RecordTransactionAbortPrepared(TransactionId xid,
 	 * Note that at this stage we have marked clog, but still show as running
 	 * in the procarray and continue to hold locks.
 	 */
-	SyncRepWaitForLSN(recptr);
+	SyncRepWaitForLSN(recptr, true);
 }
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 89a14b4..130b56b 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,7 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
-		SyncRepWaitForLSN(XactLastRecEnd);
+		SyncRepWaitForLSN(XactLastRecEnd, true);
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5123,6 +5123,13 @@ XactLogCommitRecord(TimestampTz commit_time,
 		xl_xinfo.xinfo |= XACT_COMPLETION_FORCE_SYNC_COMMIT;
 
 	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this commit is applied.
+	 */
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
+	/*
 	 * Relcache invalidations requires information about the current database
 	 * and so does logical decoding.
 	 */
@@ -5300,6 +5307,13 @@ XactLogAbortRecord(TimestampTz abort_time,
 	if (xl_xinfo.xinfo & XACT_XINFO_HAS_TWOPHASE)
 		XLogRegisterData((char *) (&xl_twophase), sizeof(xl_xact_twophase));
 
+	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this abort is applied.
+	 */
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
 	return XLogInsert(RM_XACT_ID, info);
 }
 
@@ -5458,6 +5472,13 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 	if (XactCompletionForceSyncCommit(parsed->xinfo))
 		XLogFlush(lsn);
 
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit = remote_apply), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 /*
@@ -5544,6 +5565,14 @@ xact_redo_abort(xl_xact_parsed_abort *parsed, TransactionId xid)
 		smgrdounlink(srel, true);
 		smgrclose(srel);
 	}
+
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit = remote_apply), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 void
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index b119a47..3e454f5 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -345,6 +345,9 @@ static XLogRecPtr RedoRecPtr;
  */
 static bool doPageWrites;
 
+/* Has the recovery code requested a walreceiver wakeup? */
+static bool doRequestWalReceiverReply;
+
 /*
  * RedoStartLSN points to the checkpoint's REDO location which is specified
  * in a backup label file, backup history file or control file. In standby
@@ -6879,6 +6882,19 @@ StartupXLOG(void)
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
 				SpinLockRelease(&XLogCtl->info_lck);
 
+				/*
+				 * If rm_redo reported that it applied a commit record that
+				 * the master is waiting for by calling
+				 * XLogRequestWalReceiverReply, then we wake up the receiver
+				 * so that it notices the updated lastReplayedEndRecPtr and
+				 * sends a reply to the master.
+				 */
+				if (doRequestWalReceiverReply)
+				{
+					doRequestWalReceiverReply = false;
+					WalRcvWakeup();
+				}
+
 				/* Remember this record as the last-applied one */
 				LastRec = ReadRecPtr;
 
@@ -11594,3 +11610,12 @@ SetWalWriterSleeping(bool sleeping)
 	XLogCtl->WalWriterSleeping = sleeping;
 	SpinLockRelease(&XLogCtl->info_lck);
 }
+
+/*
+ * Schedule a walreceiver wakeup in the main recovery loop.
+ */
+void
+XLogRequestWalReceiverReply(void)
+{
+	doRequestWalReceiverReply = true;
+}
diff --git a/src/backend/replication/README b/src/backend/replication/README
index 8e5bf0d..8ec8ef2 100644
--- a/src/backend/replication/README
+++ b/src/backend/replication/README
@@ -16,14 +16,15 @@ bool walrcv_connect(char *conninfo, XLogRecPtr startpoint)
 Establish connection to the primary, and starts streaming from 'startpoint'.
 Returns true on success.
 
-bool walrcv_receive(int timeout, unsigned char *type, char **buffer, int *len)
+int walrcv_receive(int timeout, char **buffer, Latch *latch)
 
 Retrieve any message available through the connection, blocking for
-maximum of 'timeout' ms. If a message was successfully read, returns true,
-otherwise false. On success, a pointer to the message payload is stored in
-*buffer, length in *len, and the type of message received in *type. The
-returned buffer is valid until the next call to walrcv_* functions, the
-caller should not attempt freeing it.
+maximum of 'timeout' ms. If a message was successfully read, returns
+its length. Otherwise returns 0 for timeout, WALRCV_RECEIVE_COPY_ENDED
+for disconnection or WALRCV_RECEIVE_LATCH_SET. On success, a pointer
+to the message payload is stored in *buffer. The returned buffer is
+valid until the next call to walrcv_* functions, and the caller should
+not attempt to free it.
 
 void walrcv_send(const char *buffer, int nbytes)
 
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index 4ee4d71..a7e989b 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -25,16 +25,6 @@
 #include "replication/walreceiver.h"
 #include "utils/builtins.h"
 
-#ifdef HAVE_POLL_H
-#include <poll.h>
-#endif
-#ifdef HAVE_SYS_POLL_H
-#include <sys/poll.h>
-#endif
-#ifdef HAVE_SYS_SELECT_H
-#include <sys/select.h>
-#endif
-
 PG_MODULE_MAGIC;
 
 void		_PG_init(void);
@@ -52,12 +42,12 @@ static void libpqrcv_readtimelinehistoryfile(TimeLineID tli, char **filename, ch
 static bool libpqrcv_startstreaming(TimeLineID tli, XLogRecPtr startpoint,
 						char *slotname);
 static void libpqrcv_endstreaming(TimeLineID *next_tli);
-static int	libpqrcv_receive(int timeout, char **buffer);
+static int	libpqrcv_receive(int timeout, char **buffer, Latch *latch);
 static void libpqrcv_send(const char *buffer, int nbytes);
 static void libpqrcv_disconnect(void);
 
 /* Prototypes for private functions */
-static bool libpq_select(int timeout_ms);
+static int libpqrcv_wait(int timeout_ms, Latch *latch);
 static PGresult *libpqrcv_PQexec(const char *query);
 
 /*
@@ -315,17 +305,16 @@ libpqrcv_readtimelinehistoryfile(TimeLineID tli,
 }
 
 /*
- * Wait until we can read WAL stream, or timeout.
- *
- * Returns true if data has become available for reading, false if timed out
- * or interrupted by signal.
+ * Wait until we can read WAL stream, a timeout is reached or (optionally) a
+ * latch is set.
  *
- * This is based on pqSocketCheck.
+ * Returns WL_SOCKET_READABLE, WL_TIMEOUT or WL_LATCH_SET.
  */
-static bool
-libpq_select(int timeout_ms)
+static int
+libpqrcv_wait(int timeout_ms, Latch *latch)
 {
 	int			ret;
+	int			events;
 
 	Assert(streamConn != NULL);
 	if (PQsocket(streamConn) < 0)
@@ -333,51 +322,39 @@ libpq_select(int timeout_ms)
 				(errcode_for_socket_access(),
 				 errmsg("invalid socket: %s", PQerrorMessage(streamConn))));
 
-	/* We use poll(2) if available, otherwise select(2) */
-	{
-#ifdef HAVE_POLL
-		struct pollfd input_fd;
-
-		input_fd.fd = PQsocket(streamConn);
-		input_fd.events = POLLIN | POLLERR;
-		input_fd.revents = 0;
-
-		ret = poll(&input_fd, 1, timeout_ms);
-#else							/* !HAVE_POLL */
-
-		fd_set		input_mask;
-		struct timeval timeout;
-		struct timeval *ptr_timeout;
-
-		FD_ZERO(&input_mask);
-		FD_SET(PQsocket(streamConn), &input_mask);
-
-		if (timeout_ms < 0)
-			ptr_timeout = NULL;
-		else
-		{
-			timeout.tv_sec = timeout_ms / 1000;
-			timeout.tv_usec = (timeout_ms % 1000) * 1000;
-			ptr_timeout = &timeout;
-		}
+	events = WL_SOCKET_READABLE | WL_POSTMASTER_DEATH;
+	if (timeout_ms > 0)
+		events |= WL_TIMEOUT;
+	if (latch != NULL)
+		events |= WL_LATCH_SET;
 
-		ret = select(PQsocket(streamConn) + 1, &input_mask,
-					 NULL, NULL, ptr_timeout);
-#endif   /* HAVE_POLL */
+	/*
+	 * Ideally we would reuse a WaitEventSet object repeatedly here to avoid
+	 * the overheads of WaitLatchOrSocket on epoll systems, but we can't be
+	 * sure that libpq has the same socket (even if the fd is the same number,
+	 * it may have been closed and reopened since the last call to
+	 * libpqrcv_wait).  In future, if there is a function for removing sockets
+	 * from WaitEventSet, then we could add and remove just the socket each
+	 * time, potentially avoiding some system calls.
+	 */
+	ret = WaitLatchOrSocket(latch, events, PQsocket(streamConn), timeout_ms);
+
+	if (ret & WL_POSTMASTER_DEATH)
+		exit(0);
+	else if (ret & WL_LATCH_SET)
+		return WL_LATCH_SET;
+	else if (ret & WL_SOCKET_READABLE)
+		return WL_SOCKET_READABLE;
+	else
+	{
+		Assert(ret & WL_TIMEOUT);
+		return WL_TIMEOUT;
 	}
-
-	if (ret == 0 || (ret < 0 && errno == EINTR))
-		return false;
-	if (ret < 0)
-		ereport(ERROR,
-				(errcode_for_socket_access(),
-				 errmsg("select() failed: %m")));
-	return true;
 }
 
 /*
  * Send a query and wait for the results by using the asynchronous libpq
- * functions and the backend version of select().
+ * functions and WaitLatchOrSocket.
  *
  * We must not use the regular blocking libpq functions like PQexec()
  * since they are uninterruptible by signals on some platforms, such as
@@ -424,8 +401,7 @@ libpqrcv_PQexec(const char *query)
 			 * elog(FATAL) within SIGTERM signal handler if the signal arrives
 			 * in the middle of establishment of replication connection.
 			 */
-			if (!libpq_select(-1))
-				continue;		/* interrupted */
+			libpqrcv_wait(-1, NULL); /* WL_SOCKET_READABLE */
 			if (PQconsumeInput(streamConn) == 0)
 				return NULL;	/* trouble */
 		}
@@ -463,8 +439,9 @@ libpqrcv_disconnect(void)
 }
 
 /*
- * Receive a message available from XLOG stream, blocking for
- * maximum of 'timeout' ms.
+ * Receive a message available from XLOG stream, blocking for maximum of
+ * 'timeout' ms.  If latch is not NULL, return early if the latch it points to
+ * is set.
  *
  * Returns:
  *
@@ -472,15 +449,16 @@ libpqrcv_disconnect(void)
  *	 point to a buffer holding the received message. The buffer is only valid
  *	 until the next libpqrcv_* call.
  *
- *	 0 if no data was available within timeout, or wait was interrupted
- *	 by signal.
+ *	 0 if no data was available within timeout.
  *
- *	 -1 if the server ended the COPY.
+ *	 WALRCV_RECEIVE_COPY_ENDED if the server ended the COPY.
+ *
+ *	 WALRCV_RECEIVE_LATCH_SET if the latch was set.
  *
  * ereports on error.
  */
 static int
-libpqrcv_receive(int timeout, char **buffer)
+libpqrcv_receive(int timeout, char **buffer, Latch *latch)
 {
 	int			rawlen;
 
@@ -498,8 +476,14 @@ libpqrcv_receive(int timeout, char **buffer)
 		 */
 		if (timeout > 0)
 		{
-			if (!libpq_select(timeout))
+			int ready = libpqrcv_wait(timeout, latch);
+
+			if (ready == WL_TIMEOUT)
 				return 0;
+			else if (ready == WL_LATCH_SET)
+				return WALRCV_RECEIVE_LATCH_SET;
+
+			Assert(ready == WL_SOCKET_READABLE);
 		}
 
 		if (PQconsumeInput(streamConn) == 0)
@@ -521,7 +505,7 @@ libpqrcv_receive(int timeout, char **buffer)
 			PQresultStatus(res) == PGRES_COPY_IN)
 		{
 			PQclear(res);
-			return -1;
+			return WALRCV_RECEIVE_COPY_ENDED;
 		}
 		else
 		{
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 92faf4e..1ee1bc5 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -91,13 +91,25 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * to the wait queue. During SyncRepWakeQueue() a WALSender changes
  * the state to SYNC_REP_WAIT_COMPLETE once replication is confirmed.
  * This backend then resets its state to SYNC_REP_NOT_WAITING.
+ *
+ * 'lsn' represents the LSN to wait for.  'commit' indicates whether this LSN
+ * represents a commit/abort record.  If it's not, then we wait only for the
+ * WAL to be flushed if synchronous_commit is set to the higher level of
+ * remote_apply, because standbys only send apply feedback for commit/abort
+ * records.
  */
 void
-SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
+SyncRepWaitForLSN(XLogRecPtr lsn, bool commit)
 {
 	char	   *new_status = NULL;
 	const char *old_status;
-	int			mode = SyncRepWaitMode;
+	int			mode;
+
+	/* Cap the level for non-commit records to remote flush only. */
+	if (commit)
+		mode = SyncRepWaitMode;
+	else
+		mode = Max(SyncRepWaitMode, SYNC_REP_WAIT_FLUSH);
 
 	/*
 	 * Fast exit if user has not requested sync replication, or there are no
@@ -122,7 +134,7 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 	 * to be a low cost check.
 	 */
 	if (!WalSndCtl->sync_standbys_defined ||
-		XactCommitLSN <= WalSndCtl->lsn[mode])
+		lsn <= WalSndCtl->lsn[mode])
 	{
 		LWLockRelease(SyncRepLock);
 		return;
@@ -132,7 +144,7 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 	 * Set our waitLSN so WALSender will know when to wake us, and add
 	 * ourselves to the queue.
 	 */
-	MyProc->waitLSN = XactCommitLSN;
+	MyProc->waitLSN = lsn;
 	MyProc->syncRepState = SYNC_REP_WAITING;
 	SyncRepQueueInsert(mode);
 	Assert(SyncRepQueueIsOrderedByLSN(mode));
@@ -147,7 +159,7 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 		new_status = (char *) palloc(len + 32 + 1);
 		memcpy(new_status, old_status, len);
 		sprintf(new_status + len, " waiting for %X/%X",
-				(uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN);
+				(uint32) (lsn >> 32), (uint32) lsn);
 		set_ps_display(new_status, false);
 		new_status[len] = '\0'; /* truncate off " waiting ..." */
 	}
@@ -416,6 +428,7 @@ SyncRepReleaseWaiters(void)
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
+	int			numapply = 0;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
@@ -462,12 +475,18 @@ SyncRepReleaseWaiters(void)
 		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
 		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
 	}
+	if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+	{
+		walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+		numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY);
+	}
 
 	LWLockRelease(SyncRepLock);
 
-	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X",
+	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%x",
 		 numwrite, (uint32) (MyWalSnd->write >> 32), (uint32) MyWalSnd->write,
-	   numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush);
+		 numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush,
+		 numapply, (uint32) (MyWalSnd->apply >> 32), (uint32) MyWalSnd->apply);
 
 	/*
 	 * If we are managing the highest priority standby, though we weren't
@@ -728,6 +747,9 @@ assign_synchronous_commit(int newval, void *extra)
 		case SYNCHRONOUS_COMMIT_REMOTE_FLUSH:
 			SyncRepWaitMode = SYNC_REP_WAIT_FLUSH;
 			break;
+		case SYNCHRONOUS_COMMIT_REMOTE_APPLY:
+			SyncRepWaitMode = SYNC_REP_WAIT_APPLY;
+			break;
 		default:
 			SyncRepWaitMode = SYNC_REP_NO_WAIT;
 			break;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 7b36e02..2fa996d 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -153,7 +153,6 @@ static void WalRcvSigUsr1Handler(SIGNAL_ARGS);
 static void WalRcvShutdownHandler(SIGNAL_ARGS);
 static void WalRcvQuickDieHandler(SIGNAL_ARGS);
 
-
 static void
 ProcessWalRcvInterrupts(void)
 {
@@ -200,6 +199,7 @@ WalReceiverMain(void)
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz last_recv_timestamp;
 	bool		ping_sent;
+	bool		apply_feedback_requested = false;
 
 	/*
 	 * WalRcv should be set up already (if we are a backend, we inherit this
@@ -407,8 +407,8 @@ WalReceiverMain(void)
 					XLogWalRcvSendHSFeedback(true);
 				}
 
-				/* Wait a while for data to arrive */
-				len = walrcv_receive(NAPTIME_PER_CYCLE, &buf);
+				/* Wait a while for data to arrive or our latch to be set */
+				len = walrcv_receive(NAPTIME_PER_CYCLE, &buf, &walrcv->latch);
 				if (len != 0)
 				{
 					/*
@@ -429,7 +429,7 @@ WalReceiverMain(void)
 						}
 						else if (len == 0)
 							break;
-						else if (len < 0)
+						else if (len == WALRCV_RECEIVE_COPY_ENDED)
 						{
 							ereport(LOG,
 									(errmsg("replication terminated by primary server"),
@@ -439,11 +439,18 @@ WalReceiverMain(void)
 							endofwal = true;
 							break;
 						}
-						len = walrcv_receive(0, &buf);
+						else if (len == WALRCV_RECEIVE_LATCH_SET)
+						{
+							/* The recovery process has new apply feedback to report. */
+							apply_feedback_requested = true;
+							ResetLatch(&walrcv->latch);
+						}
+						len = walrcv_receive(0, &buf, &walrcv->latch);
 					}
 
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(false, false);
+					XLogWalRcvSendReply(apply_feedback_requested, false);
+					apply_feedback_requested = false;
 
 					/*
 					 * If we've written some records, flush them to disk and
@@ -1222,6 +1229,21 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 }
 
 /*
+ * Wake up the walreceiver if it happens to be blocked in walrcv_receive,
+ * and tell it that a commit record has been applied.
+ *
+ * This is called by the startup process whenever interesting xlog records
+ * are applied, so that walreceiver can check if it needs to send an apply
+ * notification back to the master which may be waiting in a COMMIT with
+ * synchronous_commit = remote_apply.
+ */
+void
+WalRcvWakeup(void)
+{
+	SetLatch(&WalRcv->latch);
+}
+
+/*
  * Return a string constant representing the state. This is used
  * in system functions and views, and should *not* be translated.
  */
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 65a6cd4..06cb166 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -345,12 +345,13 @@ static const struct config_enum_entry constraint_exclusion_options[] = {
 };
 
 /*
- * Although only "on", "off", "remote_write", and "local" are documented, we
- * accept all the likely variants of "on" and "off".
+ * Although only "on", "off", "remote_apply", "remote_write", and "local" are
+ * documented, we accept all the likely variants of "on" and "off".
  */
 static const struct config_enum_entry synchronous_commit_options[] = {
 	{"local", SYNCHRONOUS_COMMIT_LOCAL_FLUSH, false},
 	{"remote_write", SYNCHRONOUS_COMMIT_REMOTE_WRITE, false},
+	{"remote_apply", SYNCHRONOUS_COMMIT_REMOTE_APPLY, false},
 	{"on", SYNCHRONOUS_COMMIT_ON, false},
 	{"off", SYNCHRONOUS_COMMIT_OFF, false},
 	{"true", SYNCHRONOUS_COMMIT_ON, true},
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index 5536012..ec4427f 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -177,7 +177,7 @@
 					# (change requires restart)
 #fsync = on				# turns forced synchronization on or off
 #synchronous_commit = on		# synchronization level;
-					# off, local, remote_write, or on
+					# off, local, remote_write, remote_apply, or on
 #wal_sync_method = fsync		# the default is the first option
 					# supported by the operating system:
 					#   open_datasync
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index ebeb582..6b706db 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -60,7 +60,9 @@ typedef enum
 	SYNCHRONOUS_COMMIT_LOCAL_FLUSH,		/* wait for local flush only */
 	SYNCHRONOUS_COMMIT_REMOTE_WRITE,	/* wait for local flush and remote
 										 * write */
-	SYNCHRONOUS_COMMIT_REMOTE_FLUSH		/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_FLUSH,	/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_APPLY		/* wait for local flush and remote
+										 * apply */
 }	SyncCommitLevel;
 
 /* Define the default setting for synchonous_commit */
@@ -144,10 +146,13 @@ typedef void (*SubXactCallback) (SubXactEvent event, SubTransactionId mySubid,
  * EOXact... routines which run at the end of the original transaction
  * completion.
  */
+#define XACT_COMPLETION_SYNC_APPLY_FEEDBACK		(1U << 29)
 #define XACT_COMPLETION_UPDATE_RELCACHE_FILE	(1U << 30)
 #define XACT_COMPLETION_FORCE_SYNC_COMMIT		(1U << 31)
 
 /* Access macros for above flags */
+#define XactCompletionSyncApplyFeedback(xinfo) \
+	((xinfo & XACT_COMPLETION_SYNC_APPLY_FEEDBACK) != 0)
 #define XactCompletionRelcacheInitFileInval(xinfo) \
 	(!!(xinfo & XACT_COMPLETION_UPDATE_RELCACHE_FILE))
 #define XactCompletionForceSyncCommit(xinfo) \
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 74a1394..a7dcdae 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -267,6 +267,8 @@ extern bool CheckPromoteSignal(void);
 extern void WakeupRecovery(void);
 extern void SetWalWriterSleeping(bool sleeping);
 
+extern void XLogRequestWalReceiverReply(void);
+
 extern void assign_max_wal_size(int newval, void *extra);
 extern void assign_checkpoint_completion_target(double newval, void *extra);
 
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index 96e059b..c005a42 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -23,8 +23,9 @@
 #define SYNC_REP_NO_WAIT		-1
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
+#define SYNC_REP_WAIT_APPLY		2
 
-#define NUM_SYNC_REP_WAIT_MODE	2
+#define NUM_SYNC_REP_WAIT_MODE	3
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
@@ -35,7 +36,7 @@
 extern char *SyncRepStandbyNames;
 
 /* called by user backend */
-extern void SyncRepWaitForLSN(XLogRecPtr XactCommitLSN);
+extern void SyncRepWaitForLSN(XLogRecPtr lsn, bool commit);
 
 /* called at backend exit */
 extern void SyncRepCleanupAtProcExit(void);
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 6eacb09..ee60dd6 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -138,7 +138,7 @@ extern PGDLLIMPORT walrcv_startstreaming_type walrcv_startstreaming;
 typedef void (*walrcv_endstreaming_type) (TimeLineID *next_tli);
 extern PGDLLIMPORT walrcv_endstreaming_type walrcv_endstreaming;
 
-typedef int (*walrcv_receive_type) (int timeout, char **buffer);
+typedef int (*walrcv_receive_type) (int timeout, char **buffer, Latch *latch);
 extern PGDLLIMPORT walrcv_receive_type walrcv_receive;
 
 typedef void (*walrcv_send_type) (const char *buffer, int nbytes);
@@ -147,6 +147,10 @@ extern PGDLLIMPORT walrcv_send_type walrcv_send;
 typedef void (*walrcv_disconnect_type) (void);
 extern PGDLLIMPORT walrcv_disconnect_type walrcv_disconnect;
 
+/* special values returned by walrcv_receive */
+#define WALRCV_RECEIVE_COPY_ENDED -1
+#define WALRCV_RECEIVE_LATCH_SET -2
+
 /* prototypes for functions in walreceiver.c */
 extern void WalReceiverMain(void) pg_attribute_noreturn();
 extern Datum pg_stat_get_wal_receiver(PG_FUNCTION_ARGS);
@@ -162,5 +166,6 @@ extern void RequestXLogStreaming(TimeLineID tli, XLogRecPtr recptr,
 extern XLogRecPtr GetWalRcvWriteRecPtr(XLogRecPtr *latestChunkStart, TimeLineID *receiveTLI);
 extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
+extern void WalRcvWakeup(void);
 
 #endif   /* _WALRECEIVER_H */
0002-replay-lag-v6.patchapplication/octet-stream; name=0002-replay-lag-v6.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 105d541..7d63782 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1208,6 +1208,12 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
       standby server</entry>
     </row>
     <row>
+     <entry><structfield>replay_lag</></entry>
+     <entry><type>interval</></entry>
+     <entry>Estimated time taken for recent WAL records to be replayed on this
+      standby server</entry>
+    </row>
+    <row>
      <entry><structfield>sync_priority</></entry>
      <entry><type>integer</></entry>
      <entry>Priority of this standby server for being chosen as the
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 130b56b..48a5950 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -5473,6 +5473,12 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 		XLogFlush(lsn);
 
 	/*
+	 * Record the primary's timestamp for the commit record, so it can be used
+	 * for tracking replay lag.
+	 */
+	SetXLogReplayTimestamp(parsed->xact_time);
+
+	/*
 	 * If asked by the primary (because someone is waiting for a synchronous
 	 * commit = remote_apply), we will need to ask walreceiver to send a
 	 * reply immediately.
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 3e454f5..504b4df 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -81,6 +81,8 @@ extern uint32 bootstrap_data_checksum_version;
 #define PROMOTE_SIGNAL_FILE		"promote"
 #define FALLBACK_PROMOTE_SIGNAL_FILE "fallback_promote"
 
+/* Size of the circular buffer of timestamped LSNs. */
+#define MAX_TIMESTAMPED_LSNS 8192
 
 /* User-settable parameters */
 int			max_wal_size = 64;	/* 1 GB */
@@ -360,6 +362,13 @@ static bool doRequestWalReceiverReply;
  */
 static XLogRecPtr RedoStartLSN = InvalidXLogRecPtr;
 
+/*
+ * LastReplayedTimestamp can be set by redo handlers when they apply a record
+ * that carries a timestamp, by calling SetXLogReplayedTimestamp.  The xlog
+ * apply loop can then update the value in shared memory.
+ */
+static TimestampTz LastReplayedTimestamp = 0;
+
 /*----------
  * Shared-memory data structures for XLOG control
  *
@@ -634,6 +643,21 @@ typedef struct XLogCtlData
 	/* current effective recovery target timeline */
 	TimeLineID	RecoveryTargetTLI;
 
+	/* timestamp from the most recently applied record associated with a timestamp. */
+	TimestampTz lastReplayedTimestamp;
+
+	/*
+	 * We maintain a circular buffer of LSNs and associated timestamps.
+	 * Walreceiver writes into it using information from timestamps, and the
+	 * startup recovery process reads from it and notifies walreceiver when
+	 * LSNs are replayed so that the timestamps can eventually be fed back to
+	 * the upstream server, to track lag.
+	 */
+	Index			timestampedLsnRead;
+	Index			timestampedLsnWrite;
+	XLogRecPtr		timestampedLsn[MAX_TIMESTAMPED_LSNS];
+	TimestampTz		timestampedLsnTime[MAX_TIMESTAMPED_LSNS];
+
 	/*
 	 * timestamp of when we started replaying the current chunk of WAL data,
 	 * only relevant for replication or archive recovery
@@ -6874,20 +6898,51 @@ StartupXLOG(void)
 				error_context_stack = errcallback.previous;
 
 				/*
-				 * Update lastReplayedEndRecPtr after this record has been
-				 * successfully replayed.
+				 * Update lastReplayedEndRecPtr and lastReplayedTimestamp
+				 * after this record has been successfully replayed.
 				 */
 				SpinLockAcquire(&XLogCtl->info_lck);
 				XLogCtl->lastReplayedEndRecPtr = EndRecPtr;
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
+				if (LastReplayedTimestamp != 0)
+				{
+					/* If replaying a record produced a timestamp, use that. */
+					XLogCtl->lastReplayedTimestamp = LastReplayedTimestamp;
+					LastReplayedTimestamp = 0;
+				}
+				else
+				{
+					/*
+					 * If we have applied LSNs associated with timestamps
+					 * received by walreceiver, then use the recorded
+					 * timestamp.  We consume from the read end of the
+					 * circular buffer.
+					 */
+					while (XLogCtl->timestampedLsnRead !=
+						   XLogCtl->timestampedLsnWrite &&
+						   XLogCtl->timestampedLsn[XLogCtl->timestampedLsnRead]
+						   <= EndRecPtr)
+					{
+						if (XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead] >
+							XLogCtl->lastReplayedTimestamp)
+						{
+							XLogCtl->lastReplayedTimestamp =
+								XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead];
+							doRequestWalReceiverReply = true;
+						}
+						XLogCtl->timestampedLsnRead =
+							(XLogCtl->timestampedLsnRead + 1) % MAX_TIMESTAMPED_LSNS;
+					}
+				}
 				SpinLockRelease(&XLogCtl->info_lck);
 
 				/*
 				 * If rm_redo reported that it applied a commit record that
 				 * the master is waiting for by calling
-				 * XLogRequestWalReceiverReply, then we wake up the receiver
-				 * so that it notices the updated lastReplayedEndRecPtr and
-				 * sends a reply to the master.
+				 * XLogRequestWalReceiverReply, or we encountered a WAL
+				 * location that was associated with a timestamp above, then
+				 * we wake up the receiver so that it notices the updated
+				 * lastReplayedEndRecPtr and sends a reply to the master.
 				 */
 				if (doRequestWalReceiverReply)
 				{
@@ -11619,3 +11674,91 @@ XLogRequestWalReceiverReply(void)
 {
 	doRequestWalReceiverReply = true;
 }
+
+/*
+ * Record the timestamp that is associated with a WAL position.
+ *
+ * This is called by walreceiver on standby servers when keepalive messages
+ * arrive, using timestamps generated on the primary server.  The timestamp
+ * will be sent back to the primary server when the standby had applied this
+ * WAL position.  The primary can use the elapsed time to estimate the replay
+ * lag.
+ */
+void
+SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn)
+{
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn == XLogCtl->lastReplayedEndRecPtr)
+	{
+		/*
+		 * That is the last replayed LSN: we are fully replayed, so we can
+		 * update the replay timestamp immediately.
+		 */
+		XLogCtl->lastReplayedTimestamp = timestamp;
+	}
+	else
+	{
+		/*
+		 * There is WAL still to be applied.  We will associate the timestamp
+		 * with this WAL position and wait for it to be replayed.  We add it
+		 * at the 'write' end of the circular buffer of LSN/timestamp
+		 * mappings, which the replay loop will eventually read.
+		 */
+		Index w = XLogCtl->timestampedLsnWrite;
+		Index r = XLogCtl->timestampedLsnRead;
+
+		XLogCtl->timestampedLsn[w] = lsn;
+		XLogCtl->timestampedLsnTime[w] = timestamp;
+
+		/* Advance the write point. */
+		w = (w + 1) % MAX_TIMESTAMPED_LSNS;
+		XLogCtl->timestampedLsnWrite = w;
+		if (w == r)
+		{
+			/*
+			 * The buffer is full.  Advance the read point (throwing away
+			 * oldest values; we will begin to overestimate replay lag, until
+			 * lag decreases to a size our buffer can manage, or the next
+			 * commit record is replayed).
+			 */
+			r = (r + 1) % MAX_TIMESTAMPED_LSNS;
+			XLogCtl->timestampedLsnRead = r;
+		}
+	}
+	SpinLockRelease(&XLogCtl->info_lck);
+}
+
+/*
+ * Set the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary.  This can be called by redo handlers that have
+ * an appropriate timestamp (currently only commit records).  Updating the
+ * shared memory value is deferred until after the redo handler returns.
+ */
+void
+SetXLogReplayTimestamp(TimestampTz timestamp)
+{
+	LastReplayedTimestamp = timestamp;
+}
+
+/*
+ * Get the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary, and also the most recently applied LSN.  (Note
+ * that the timestamp and the LSN don't necessarily relate to the same
+ * record.)
+ *
+ * This is similar to GetLatestXTime, except that it is not only advanced by
+ * commit records (see SetXLogReplayTimestampAtLsn).
+ */
+TimestampTz
+GetXLogReplayTimestamp(XLogRecPtr *lsn)
+{
+	TimestampTz result;
+
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn)
+		*lsn = XLogCtl->lastReplayedEndRecPtr;
+	result = XLogCtl->lastReplayedTimestamp;
+	SpinLockRelease(&XLogCtl->info_lck);
+
+	return result;
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 9ae1ef4..a53f07b 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -662,6 +662,7 @@ CREATE VIEW pg_stat_replication AS
             W.write_location,
             W.flush_location,
             W.replay_location,
+            W.replay_lag,
             W.sync_priority,
             W.sync_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 2fa996d..faea9ff 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -85,6 +85,8 @@ walrcv_disconnect_type walrcv_disconnect = NULL;
 
 #define NAPTIME_PER_CYCLE 100	/* max sleep time between cycles (100ms) */
 
+#define MIN_TIME_BETWEEN_TIMESTAMPED_LSNS 1000 /* 1s */
+
 /*
  * These variables are used similarly to openLogFile/SegNo/Off,
  * but for walreceiver to write the XLOG. recvFileTLI is the TimeLineID
@@ -102,6 +104,8 @@ static uint32 recvOff = 0;
 static volatile sig_atomic_t got_SIGHUP = false;
 static volatile sig_atomic_t got_SIGTERM = false;
 
+static bool recovery_active = false;
+
 /*
  * LogstreamResult indicates the byte positions that we have already
  * written/fsynced.
@@ -143,7 +147,7 @@ static void WalRcvDie(int code, Datum arg);
 static void XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len);
 static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
-static void XLogWalRcvSendReply(bool force, bool requestReply);
+static void XLogWalRcvSendReply(bool force, bool requestReply, bool includeApplyTimestamp);
 static void XLogWalRcvSendHSFeedback(bool immed);
 static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
 
@@ -443,13 +447,15 @@ WalReceiverMain(void)
 						{
 							/* The recovery process has new apply feedback to report. */
 							apply_feedback_requested = true;
+							recovery_active = true;
 							ResetLatch(&walrcv->latch);
 						}
 						len = walrcv_receive(0, &buf, &walrcv->latch);
 					}
 
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(apply_feedback_requested, false);
+					XLogWalRcvSendReply(apply_feedback_requested, false,
+										apply_feedback_requested);
 					apply_feedback_requested = false;
 
 					/*
@@ -505,7 +511,7 @@ WalReceiverMain(void)
 						}
 					}
 
-					XLogWalRcvSendReply(requestReply, requestReply);
+					XLogWalRcvSendReply(requestReply, requestReply, false);
 					XLogWalRcvSendHSFeedback(false);
 				}
 			}
@@ -836,6 +842,8 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 			}
 		case 'k':				/* Keepalive */
 			{
+				bool reportApplyTimestamp = false;
+
 				/* copy message to StringInfo */
 				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
 				if (len != hdrlen)
@@ -852,9 +860,22 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 
 				ProcessWalSndrMessage(walEnd, sendTime);
 
+				/*
+				 * If no apply timestamps have been sent at the request of the
+				 * recovery process since we last received a keepalive, then
+				 * we will send one now.  This allows us to feed back
+				 * timestamps in response to pings if we are idle or if the
+				 * recovery process is somehow blocked, but we don't want to
+				 * do that if it's actively applying and periodically waking
+				 * us up with accurate apply timestamps.
+				 */
+				if (!recovery_active)
+					reportApplyTimestamp = true;
+				recovery_active = false;
+
 				/* If the primary requested a reply, send one immediately */
-				if (replyRequested)
-					XLogWalRcvSendReply(true, false);
+				if (replyRequested || reportApplyTimestamp)
+					XLogWalRcvSendReply(true, false, reportApplyTimestamp);
 				break;
 			}
 		default:
@@ -1017,7 +1038,7 @@ XLogWalRcvFlush(bool dying)
 		/* Also let the master know that we made some progress */
 		if (!dying)
 		{
-			XLogWalRcvSendReply(false, false);
+			XLogWalRcvSendReply(false, false, false);
 			XLogWalRcvSendHSFeedback(false);
 		}
 	}
@@ -1035,15 +1056,18 @@ XLogWalRcvFlush(bool dying)
  * If 'requestReply' is true, requests the server to reply immediately upon
  * receiving this message. This is used for heartbearts, when approaching
  * wal_receiver_timeout.
+ *
+ * If 'reportApplyTimestamp' is true, the latest apply timestamp is included.
  */
 static void
-XLogWalRcvSendReply(bool force, bool requestReply)
+XLogWalRcvSendReply(bool force, bool requestReply, bool reportApplyTimestamp)
 {
 	static XLogRecPtr writePtr = 0;
 	static XLogRecPtr flushPtr = 0;
 	XLogRecPtr	applyPtr;
 	static TimestampTz sendTime = 0;
 	TimestampTz now;
+	TimestampTz applyTimestamp = 0;
 
 	/*
 	 * If the user doesn't want status to be reported to the master, be sure
@@ -1059,10 +1083,8 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	 * We can compare the write and flush positions to the last message we
 	 * sent without taking any lock, but the apply position requires a spin
 	 * lock, so we don't check that unless something else has changed or 10
-	 * seconds have passed.  This means that the apply log position will
-	 * appear, from the master's point of view, to lag slightly, but since
-	 * this is only for reporting purposes and only on idle systems, that's
-	 * probably OK.
+	 * seconds have passed, or the force flag has been set (which happens when
+	 * apply feedback has been requested by the primary).
 	 */
 	if (!force
 		&& writePtr == LogstreamResult.Write
@@ -1075,7 +1097,10 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	/* Construct a new message */
 	writePtr = LogstreamResult.Write;
 	flushPtr = LogstreamResult.Flush;
-	applyPtr = GetXLogReplayRecPtr(NULL);
+	if (reportApplyTimestamp)
+		applyTimestamp = GetXLogReplayTimestamp(&applyPtr);
+	else
+		applyPtr = GetXLogReplayRecPtr(NULL);
 
 	resetStringInfo(&reply_message);
 	pq_sendbyte(&reply_message, 'r');
@@ -1083,6 +1108,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	pq_sendint64(&reply_message, flushPtr);
 	pq_sendint64(&reply_message, applyPtr);
 	pq_sendint64(&reply_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&reply_message, TimestampTzToIntegerTimestamp(applyTimestamp));
 	pq_sendbyte(&reply_message, requestReply ? 1 : 0);
 
 	/* Send it */
@@ -1187,8 +1213,8 @@ static void
 ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 {
 	WalRcvData *walrcv = WalRcv;
-
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
+	static TimestampTz lastRecordedTimestamp = 0;
 
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
@@ -1199,6 +1225,18 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
 	SpinLockRelease(&walrcv->mutex);
 
+	/*
+	 * Remember primary's timestamp at this WAL location.  We throw away
+	 * samples if they are coming too fast because we don't want to fill up
+	 * the finite circular buffer and have to throw away older samples.
+	 */
+	if (lastRecordedTimestamp < TimestampTzPlusMilliseconds(sendTime,
+															-MIN_TIME_BETWEEN_TIMESTAMPED_LSNS))
+	{
+		SetXLogReplayTimestampAtLsn(sendTime, walEnd);
+		lastRecordedTimestamp = sendTime;
+	}
+
 	if (log_min_messages <= DEBUG2)
 	{
 		char	   *sendtime;
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index f98475c..16d7abc 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1545,15 +1545,29 @@ ProcessStandbyReplyMessage(void)
 	XLogRecPtr	writePtr,
 				flushPtr,
 				applyPtr;
+	int64		applyLagUs;
 	bool		replyRequested;
+	TimestampTz now = GetCurrentTimestamp();
+	TimestampTz applyTimestamp;
 
 	/* the caller already consumed the msgtype byte */
 	writePtr = pq_getmsgint64(&reply_message);
 	flushPtr = pq_getmsgint64(&reply_message);
 	applyPtr = pq_getmsgint64(&reply_message);
 	(void) pq_getmsgint64(&reply_message);		/* sendTime; not used ATM */
+	applyTimestamp = IntegerTimestampToTimestampTz(pq_getmsgint64(&reply_message));
 	replyRequested = pq_getmsgbyte(&reply_message);
 
+	/* Compute the apply lag in milliseconds. */
+	if (applyTimestamp == 0)
+		applyLagUs = -1;
+	else
+#ifdef HAVE_INT64_TIMESTAMP
+		applyLagUs = now - applyTimestamp;
+#else
+		applyLagUs = (now - applyTimestamp) * 1000000;
+#endif
+
 	elog(DEBUG2, "write %X/%X flush %X/%X apply %X/%X%s",
 		 (uint32) (writePtr >> 32), (uint32) writePtr,
 		 (uint32) (flushPtr >> 32), (uint32) flushPtr,
@@ -1575,6 +1589,8 @@ ProcessStandbyReplyMessage(void)
 		walsnd->write = writePtr;
 		walsnd->flush = flushPtr;
 		walsnd->apply = applyPtr;
+		if (applyLagUs >= 0)
+			walsnd->applyLagUs = applyLagUs;
 		SpinLockRelease(&walsnd->mutex);
 	}
 
@@ -2745,7 +2761,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	8
+#define PG_STAT_GET_WAL_SENDERS_COLS	9
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2793,6 +2809,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		XLogRecPtr	write;
 		XLogRecPtr	flush;
 		XLogRecPtr	apply;
+		int64		applyLagUs;
 		int			priority;
 		WalSndState state;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
@@ -2807,6 +2824,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
+		applyLagUs = walsnd->applyLagUs;
 		priority = walsnd->sync_standby_priority;
 		SpinLockRelease(&walsnd->mutex);
 
@@ -2841,6 +2859,23 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				nulls[5] = true;
 			values[5] = LSNGetDatum(apply);
 
+			if (applyLagUs < 0)
+				nulls[6] = true;
+			else
+			{
+				Interval *applyLagInterval = palloc(sizeof(Interval));
+
+				applyLagInterval->month = 0;
+				applyLagInterval->day = 0;
+#ifdef HAVE_INT64_TIMESTAMP
+				applyLagInterval->time = applyLagUs;
+#else
+				applyLagInterval->time = applyLagUs / 1000000.0;
+#endif
+				nulls[6] = false;
+				values[6] = IntervalPGetDatum(applyLagInterval);
+			}
+
 			/*
 			 * Treat a standby such as a pg_basebackup background process
 			 * which always returns an invalid flush location, as an
@@ -2848,18 +2883,18 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 			 */
 			priority = XLogRecPtrIsInvalid(walsnd->flush) ? 0 : priority;
 
-			values[6] = Int32GetDatum(priority);
+			values[7] = Int32GetDatum(priority);
 
 			/*
 			 * More easily understood version of standby state. This is purely
 			 * informational, not different from priority.
 			 */
 			if (priority == 0)
-				values[7] = CStringGetTextDatum("async");
+				values[8] = CStringGetTextDatum("async");
 			else if (walsnd == sync_standby)
-				values[7] = CStringGetTextDatum("sync");
+				values[8] = CStringGetTextDatum("sync");
 			else
-				values[7] = CStringGetTextDatum("potential");
+				values[8] = CStringGetTextDatum("potential");
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
diff --git a/src/backend/utils/adt/timestamp.c b/src/backend/utils/adt/timestamp.c
index c9e5270..f382b20 100644
--- a/src/backend/utils/adt/timestamp.c
+++ b/src/backend/utils/adt/timestamp.c
@@ -1629,6 +1629,20 @@ IntegerTimestampToTimestampTz(int64 timestamp)
 #endif
 
 /*
+ * TimestampTzToIntegerTimestamp -- convert a native timestamp to int64 format
+ *
+ * When compiled with --enable-integer-datetimes, this is implemented as a
+ * no-op macro.
+ */
+#ifndef HAVE_INT64_TIMESTAMP
+int64
+TimestampTzToIntegerTimestamp(TimestampTz timestamp)
+{
+	return timestamp * 1000000;
+}
+#endif
+
+/*
  * TimestampDifference -- convert the difference between two timestamps
  *		into integer seconds and microseconds
  *
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index a7dcdae..c8be3ce 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -235,6 +235,9 @@ extern void GetXLogReceiptTime(TimestampTz *rtime, bool *fromStream);
 extern XLogRecPtr GetXLogReplayRecPtr(TimeLineID *replayTLI);
 extern XLogRecPtr GetXLogInsertRecPtr(void);
 extern XLogRecPtr GetXLogWriteRecPtr(void);
+extern void SetXLogReplayTimestamp(TimestampTz timestamp);
+extern void SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn);
+extern TimestampTz GetXLogReplayTimestamp(XLogRecPtr *lsn);
 extern bool RecoveryIsPaused(void);
 extern void SetRecoveryPause(bool recoveryPause);
 extern TimestampTz GetLatestXTime(void);
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index a595327..4054726 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2712,7 +2712,7 @@ DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f
 DESCR("statistics: information about currently active backends");
 DATA(insert OID = 3318 (  pg_stat_get_progress_info           PGNSP PGUID 12 1 100 0 0 f f f f t t s r 1 0 2249 "25" "{25,23,26,26,20,20,20,20,20,20,20,20,20,20}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{cmdtype,pid,datid,relid,param1,param2,param3,param4,param5,param6,param7,param8,param9,param10}" _null_ _null_ pg_stat_get_progress_info _null_ _null_ _null_ ));
 DESCR("statistics: information about progress of backends running maintenance command");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,23,25}" "{o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25}" "{o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 7794aa5..4de43e8 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -46,6 +46,7 @@ typedef struct WalSnd
 	XLogRecPtr	write;
 	XLogRecPtr	flush;
 	XLogRecPtr	apply;
+	int64		applyLagUs;
 
 	/* Protects shared variables shown above. */
 	slock_t		mutex;
diff --git a/src/include/utils/timestamp.h b/src/include/utils/timestamp.h
index fbead3a..297e151 100644
--- a/src/include/utils/timestamp.h
+++ b/src/include/utils/timestamp.h
@@ -227,9 +227,11 @@ extern bool TimestampDifferenceExceeds(TimestampTz start_time,
 #ifndef HAVE_INT64_TIMESTAMP
 extern int64 GetCurrentIntegerTimestamp(void);
 extern TimestampTz IntegerTimestampToTimestampTz(int64 timestamp);
+extern int64 TimestampTzToIntegerTimestamp(TimestampTz timestamp);
 #else
 #define GetCurrentIntegerTimestamp()	GetCurrentTimestamp()
 #define IntegerTimestampToTimestampTz(timestamp) (timestamp)
+#define TimestampTzToIntegerTimestamp(timestamp) (timestamp)
 #endif
 
 extern TimestampTz time_t_to_timestamptz(pg_time_t tm);
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 79f9b23..fc4b765 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1783,11 +1783,12 @@ pg_stat_replication| SELECT s.pid,
     w.write_location,
     w.flush_location,
     w.replay_location,
+    w.replay_lag,
     w.sync_priority,
     w.sync_state
    FROM pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, wait_event_type, wait_event, xact_start, query_start, backend_start, state_change, client_addr, client_hostname, client_port, backend_xid, backend_xmin, ssl, sslversion, sslcipher, sslbits, sslcompression, sslclientdn),
     pg_authid u,
-    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, sync_priority, sync_state)
+    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state)
   WHERE ((s.usesysid = u.oid) AND (s.pid = w.pid));
 pg_stat_ssl| SELECT s.pid,
     s.ssl,
0003-refactor-syncrep-exit-v6.patchapplication/octet-stream; name=0003-refactor-syncrep-exit-v6.patchDownload
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 1ee1bc5..376ddf4 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -83,6 +83,64 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * ===========================================================
  */
 
+static bool
+SyncRepCheckEarlyExit(void)
+{
+	/*
+	 * If a wait for synchronous replication is pending, we can neither
+	 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
+	 * lead the client to believe that the transaction aborted, which
+	 * is not true: it's already committed locally. The former is no good
+	 * either: the client has requested synchronous replication, and is
+	 * entitled to assume that an acknowledged commit is also replicated,
+	 * which might not be true. So in this case we issue a WARNING (which
+	 * some clients may be able to interpret) and shut off further output.
+	 * We do NOT reset ProcDiePending, so that the process will die after
+	 * the commit is cleaned up.
+	 */
+	if (ProcDiePending)
+	{
+		ereport(WARNING,
+				(errcode(ERRCODE_ADMIN_SHUTDOWN),
+				 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * It's unclear what to do if a query cancel interrupt arrives.  We
+	 * can't actually abort at this point, but ignoring the interrupt
+	 * altogether is not helpful, so we just terminate the wait with a
+	 * suitable warning.
+	 */
+	if (QueryCancelPending)
+	{
+		QueryCancelPending = false;
+		ereport(WARNING,
+				(errmsg("canceling wait for synchronous replication due to user request"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * If the postmaster dies, we'll probably never get an
+	 * acknowledgement, because all the wal sender processes will exit. So
+	 * just bail out.
+	 */
+	if (!PostmasterIsAlive())
+	{
+		ProcDiePending = true;
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	return false;
+}
+
 /*
  * Wait for synchronous replication, if requested by user.
  *
@@ -192,57 +250,9 @@ SyncRepWaitForLSN(XLogRecPtr lsn, bool commit)
 		if (syncRepState == SYNC_REP_WAIT_COMPLETE)
 			break;
 
-		/*
-		 * If a wait for synchronous replication is pending, we can neither
-		 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
-		 * lead the client to believe that the transaction aborted, which
-		 * is not true: it's already committed locally. The former is no good
-		 * either: the client has requested synchronous replication, and is
-		 * entitled to assume that an acknowledged commit is also replicated,
-		 * which might not be true. So in this case we issue a WARNING (which
-		 * some clients may be able to interpret) and shut off further output.
-		 * We do NOT reset ProcDiePending, so that the process will die after
-		 * the commit is cleaned up.
-		 */
-		if (ProcDiePending)
-		{
-			ereport(WARNING,
-					(errcode(ERRCODE_ADMIN_SHUTDOWN),
-					 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit())
 			break;
-		}
-
-		/*
-		 * It's unclear what to do if a query cancel interrupt arrives.  We
-		 * can't actually abort at this point, but ignoring the interrupt
-		 * altogether is not helpful, so we just terminate the wait with a
-		 * suitable warning.
-		 */
-		if (QueryCancelPending)
-		{
-			QueryCancelPending = false;
-			ereport(WARNING,
-					(errmsg("canceling wait for synchronous replication due to user request"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * If the postmaster dies, we'll probably never get an
-		 * acknowledgement, because all the wal sender processes will exit. So
-		 * just bail out.
-		 */
-		if (!PostmasterIsAlive())
-		{
-			ProcDiePending = true;
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
-			break;
-		}
 
 		/*
 		 * Wait on latch.  Any condition that should wake us up will set the
0004-causal-reads-v6.patchapplication/octet-stream; name=0004-causal-reads-v6.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 6ab65a7..ed6f07c 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -2749,6 +2749,35 @@ include_dir 'conf.d'
      across the cluster without problems if that is required.
     </para>
 
+    <sect2 id="runtime-config-replication-all">
+     <title>All Servers</title>
+     <para>
+      These parameters can be set on the primary or any standby.
+     </para>
+     <variablelist>
+      <varlistentry id="guc-causal-reads" xreflabel="causal_reads">
+       <term><varname>causal_reads</varname> (<type>boolean</type>)
+       <indexterm>
+        <primary><varname>causal_reads</> configuration parameter</primary>
+       </indexterm>
+       </term>
+       <listitem>
+        <para>
+         Enables causal consistency between transactions run on different
+         servers.  A transaction that is run on a standby
+         with <varname>causal_reads</> set to <literal>on</> is guaranteed
+         either to see the effects of all completed transactions run on the
+         primary with the setting on, or to receive an error "standby is not
+         available for causal reads".  Note that both transactions involved in
+         a causal dependency (a write on the primary followed by a read on any
+         server which must see the write) must be run with the setting on.
+         See <xref linkend="causal-reads"> for more details.
+        </para>
+       </listitem>
+      </varlistentry>
+     </variablelist>     
+    </sect2>
+
     <sect2 id="runtime-config-replication-sender">
      <title>Sending Server(s)</title>
 
@@ -2980,6 +3009,48 @@ include_dir 'conf.d'
       </listitem>
      </varlistentry>
 
+     <varlistentry>
+      <term><varname>causal_reads_timeout</varname> (<type>integer</type>)
+       <indexterm>
+        <primary><varname>causal_reads_timeout</> configuration parameter</primary>
+       </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies the maximum replay lag the primary will tolerate from a
+        standby before dropping it from the set of standbys available for
+        causal reads.
+       </para>
+       <para>
+        This setting is also used to control the <firstterm>leases</> used to
+        maintain the causal reads guarantee.  It must be set to a value which
+        is at least 4 times the maximum possible difference in system clocks
+        between the primary and standby servers, as described
+        in <xref linkend="causal-reads">.
+       </para>
+      </listitem>
+     </varlistentry>
+
+     <varlistentry id="guc-causal-reads-standby-names" xreflabel="causal-reads-standby-names">
+      <term><varname>causal_reads_standby_names</varname> (<type>string</type>)
+      <indexterm>
+       <primary><varname>causal_reads_standby_names</> configuration parameter</primary>
+      </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies a comma-separated list of standby names that can support
+        <firstterm>causal reads</>, as described in
+        <xref linkend="causal-reads">.  Follows the same convention
+        as <link linkend="guc-synchronous-standby-names"><literal>synchronous_standby_name</></>.
+        The default is <literal>*</>, matching all standbys.
+       </para>
+       <para>
+        This setting has no effect if <varname>causal_reads_timeout</> is not set.
+       </para>
+      </listitem>
+     </varlistentry>
+
      </variablelist>
     </sect2>
 
diff --git a/doc/src/sgml/high-availability.sgml b/doc/src/sgml/high-availability.sgml
index 03c6c30..7a0910d 100644
--- a/doc/src/sgml/high-availability.sgml
+++ b/doc/src/sgml/high-availability.sgml
@@ -1115,6 +1115,9 @@ primary_slot_name = 'node_a_slot'
     that it has replayed the transaction, making it visible to user queries.
     In simple cases, this allows for load balancing with causal consistency
     on a single hot standby.
+    (See also
+    <xref linkend="causal-reads"> which deals with multiple standbys and
+    standby failure.)
    </para>
 
    <para>
@@ -1233,6 +1236,119 @@ primary_slot_name = 'node_a_slot'
    </sect3>
   </sect2>
 
+  <sect2 id="causal-reads">
+   <title>Causal reads</title>
+   <indexterm>
+    <primary>causal reads</primary>
+    <secondary>in standby</secondary>
+   </indexterm>
+
+   <para>
+    The causal reads feature allows read-only queries to run on hot standby
+    servers without exposing stale data to the client, providing a form of
+    causal consistency.  Transactions can run on any standby with the
+    following guarantee about the visibility of preceding transactions: If you
+    set <varname>causal_reads</> to <literal>on</> in any pair of consecutive
+    transactions tx1, tx2 where tx2 begins after tx1 successfully returns,
+    then tx2 will either see tx1 or fail with a new error "standby is not
+    available for causal reads", no matter which server it runs on.  Although
+    the guarantee is expressed in terms of two individual transactions, the
+    GUC can also be set at session, role or system level to make the guarantee
+    generally, allowing for load balancing of applications that were not
+    designed with load balancing in mind.
+   </para>
+
+   <para>
+    In order to enable the feature, <varname>causal_reads_timeout</> must be
+    set to a non-zero value on the primary server.  The
+    GUC <varname>causal_reads_standby_names</> can be used to limit the set of
+    standbys that can join the dynamic set of causal reads standbys by
+    providing a comma-separated list of application names.  By default, all
+    standbys are candidates, if the feature is enabled.
+   </para>
+
+   <para>
+    The current set of servers that the primary considers to be available for
+    causal reads can be seen in
+    the <link linkend="monitoring-stats-views-table"> <literal>pg_stat_replication</></>
+    view.  Administrators, applications and load balancing middleware can use
+    this view to discover standbys that can currently handle causal reads
+    transactions without raising the error.  Since that information is only an
+    instantantaneous snapshot, clients should still be prepared for the error
+    to be raised at any time, and consider redirecting transactions to another
+    standby.
+   </para>
+
+   <para>
+    The advantages of the causal reads feature over simply
+    setting <varname>synchronous_commit</> to <literal>remote_apply</> are:
+    <orderedlist>
+      <listitem>
+       <para>
+        It allows the primary to wait for multiple standbys to replay
+        transactions.
+       </para>
+      </listitem>
+      <listitem>
+       <para>
+        It places a configurable limit on how much replay lag (and therefore
+        delay at commit time) the primary tolerates from standbys before it
+        drops them from the dynamic set of standbys it waits for.
+       </para>   
+      </listitem>
+      <listitem>
+       <para>
+        It upholds the causal reads guarantee during the transitions that
+        occur when new standbys are added or removed from the set of standbys,
+        including scenarios where contact has been lost between the primary
+        and standbys but the standby is still alive and running client
+        queries.
+       </para>
+      </listitem>
+    </orderedlist>
+   </para>
+
+   <para>
+    The protocol used to uphold the guarantee even in the case of network
+    failure depends on the system clocks of the primary and standby servers
+    being synchronized, with an allowance for a difference up to one quarter
+    of <varname>causal_reads_timeout</>.  For example,
+    if <varname>causal_reads_timeout</> is set to <literal>4s</>, then the
+    clocks must not be further than 1 second apart for the guarantee to be
+    upheld reliably during transitions.  The ubiquity of the Network Time
+    Protocol (NTP) on modern operating systems and availability of high
+    quality time servers makes it possible to choose a tolerance significantly
+    higher than the maximum expected clock difference.  An effort is
+    nevertheless made to detect and report misconfigured and faulty systems
+    with clock differences greater than the configured tolerance.
+   </para>
+
+   <note>
+    <para>
+     Current hardware clocks, NTP implementations and public time servers are
+     unlikely to allow the system clocks to differ more than tens or hundreds
+     of milliseconds, and systems synchronized with dedicated local time
+     servers may be considerably more accurate, but you should only consider
+     setting <varname>causal_reads_timeout</> below 4 seconds (allowing up to
+     1 second of clock difference) after researching your time synchronization
+     infrastructure thoroughly.
+    </para>  
+   </note>
+
+   <note>
+    <para>
+      While similar to synchronous replication in the sense that both involve
+      the primary server waiting for responses from standby servers, the
+      causal reads feature is not concerned with avoiding data loss.  A
+      primary configured for causal reads will drop all standbys that stop
+      responding or replay too slowly from the dynamic set that it waits for,
+      so you should consider configuring both synchronous replication and
+      causal reads if you need data loss avoidance guarantees and causal
+      consistency guarantees for load balancing.
+    </para>
+   </note>
+  </sect2>
+
   <sect2 id="continuous-archiving-in-standby">
    <title>Continuous archiving in standby</title>
 
@@ -1581,7 +1697,16 @@ if (!triggered)
     so there will be a measurable delay between primary and standby. Running the
     same query nearly simultaneously on both primary and standby might therefore
     return differing results. We say that data on the standby is
-    <firstterm>eventually consistent</firstterm> with the primary.  Once the
+    <firstterm>eventually consistent</firstterm> with the primary by default.
+    The data visible to a transaction running on a standby can be
+    made <firstterm>causally consistent</> with respect to a transaction that
+    has completed on the primary by setting <varname>causal_reads</>
+    to <literal>on</> in both transactions.  For more details,
+    see <xref linkend="causal-reads">.
+   </para>
+
+   <para>
+    Once the    
     commit record for a transaction is replayed on the standby, the changes
     made by that transaction will be visible to any new snapshots taken on
     the standby.  Snapshots may be taken at the start of each query or at the
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 7d63782..23d68d5 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1224,6 +1224,17 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
      <entry><type>text</></entry>
      <entry>Synchronous state of this standby server</entry>
     </row>
+    <row>
+     <entry><structfield>causal_reads_state</></entry>
+     <entry><type>text</></entry>
+     <entry>Causal reads state of this standby server.  This field will be
+     non-null only if <varname>cause_reads_timeout</> is set.  If a standby is
+     in <literal>available</> state, then it can currently serve causal reads
+     queries.  If it is not replaying fast enough or not responding to
+     keepalive messages, it will be in <literal>unavailable</> state, and if
+     it is currently transitioning to availability it will be
+     in <literal>joining</> state for a short time.</entry>
+    </row>
    </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 893c2fa..111198a 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -2098,11 +2098,12 @@ RecordTransactionCommitPrepared(TransactionId xid,
 	END_CRIT_SECTION();
 
 	/*
-	 * Wait for synchronous replication, if required.
+	 * Wait for causal reads and synchronous replication, if required.
 	 *
 	 * Note that at this stage we have marked clog, but still show as running
 	 * in the procarray and continue to hold locks.
 	 */
+	CausalReadsWaitForLSN(recptr);
 	SyncRepWaitForLSN(recptr, true);
 }
 
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 48a5950..4530a6e 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,10 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
+	{
+		CausalReadsWaitForLSN(XactLastRecEnd);
 		SyncRepWaitForLSN(XactLastRecEnd, true);
+	}
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5126,7 +5129,7 @@ XactLogCommitRecord(TimestampTz commit_time,
 	 * Check if the caller would like to ask standbys for immediate feedback
 	 * once this commit is applied.
 	 */
-	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY || causal_reads)
 		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
 
 	/*
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index a53f07b..276ac12 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -664,7 +664,8 @@ CREATE VIEW pg_stat_replication AS
             W.replay_location,
             W.replay_lag,
             W.sync_priority,
-            W.sync_state
+            W.sync_state,
+            W.causal_reads_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
             pg_stat_get_wal_senders() AS W
     WHERE S.usesysid = U.oid AND
diff --git a/src/backend/replication/README.causal_reads b/src/backend/replication/README.causal_reads
new file mode 100644
index 0000000..1fddd62
--- /dev/null
+++ b/src/backend/replication/README.causal_reads
@@ -0,0 +1,193 @@
+The causal reads guarantee says: If you run any two consecutive
+transactions tx1, tx2 where tx1 completes before tx2 begins, with
+causal_reads set to "on" in both transactions, tx2 will see tx1 or
+raise an error to complain that it can't guarantee causal consistency,
+no matter which servers (primary or any standby) you run each
+transaction on.
+
+When both transactions run on the primary, the guarantee is trivially
+upheld.
+
+To deal with read-only physical streaming standbys, the primary keeps
+track of a set of standbys that it considers to be currently
+"available" for causal reads, and sends a stream of "leases" to those
+standbys granting them the right to handle causal reads transactions
+for a short time without any further communication with the primary.
+
+In general, the primary provides the guarantee by waiting for all of
+the "available" standbys to report that they have applied a
+transaction.  However, the set of available standbys is dynamic, and
+things get more complicated during state transitions.  There are two
+types of transitions to consider:
+
+1.  unavailable->joining->available
+
+Standbys start out as "unavailable".  If a standby is unavailable and
+is applying fast enough and matches causal_reads_standby_names, the
+primary transitions it to "available", but first it sets it to
+"joining" until it is sure that any transaction committed while it was
+unavailable has definitely been applied on the standby.  This closes a
+race that would otherwise exist if we moved directly to available
+state: tx1 might not wait for a given standby because it's
+unavailable, then a lease might be granted, and then tx2 might run a
+causal reads transaction without error but see stale data.  The
+joining state acts as an airlock: while in joining state, the primary
+waits for that standby to replay causal reads transactions in
+anticipation of the move to available, but it doesn't progress to
+available state and grant a lease to the standby until everything
+preceding joining state has also been applied.
+
+2.  available->unavailable
+
+If a standby is not applying fast enough or not responding to
+keepalive messages, then the primary kicks that standby out of the
+dynamic set of available standbys, that is, marks it as "unavailable".
+In order to make sure that the standby has started rejecting causal
+reads transactions, it needs to revoke the lease it most recently
+granted.  It does that by waiting for the lease to expire before
+allowing any causal reads commits to return.  (In future there could
+be a fast-path revocation message which waits for a serial-numbered
+acknowledgement to reduce waiting in the case where the standby is
+lagging but still reachable and responding).
+
+The rest of this document illustrates how clock skew affects the
+available->unavailable transition.
+
+The following 4 variables are derived from a single GUC, and these
+values will be used in the following illustrations:
+
+causal_reads_timeout = 4s
+lease_time           = 4s (= causal_reads_timeout)
+keepalive_time       = 2s (= lease_time / 2)
+max_clock_skew       = 1s (= lease_time / 4)
+
+Every keepalive_time, the primary transmits a lease that expires at
+local_clock_time + lease_time - max_clock_skew, shown in the following
+diagram as 't' for transmission time and '|' for expiry time.  If
+contact is lost with a standby, the primary will wait until sent_time
++ lease_time for the most recently granted lease to expire, shown on
+the following diagram 'x', to be sure that the standby's clock has
+reached the expiry time even if its clock differs by up to
+max_clock_skew.  In other words, the primary tells the standby that
+the expiry time is at one time, but it trusts that the standby will
+surely agree if it gives it some extra time.  The extra time is
+max_clock_skew.  If the clocks differ by more than max_clock_skew, all
+bets are off (but see below for attempt to detect obvious cases).
+
+0     1     2     3     4     5     6     7     8     9
+t-----------------|-----x
+            t-----------------|-----x
+                        t-----------------|-----x
+                                    t-----------------|...
+                                                t------...
+
+A standby whose clock is 2 seconds ahead of the primary's clock
+perceives gaps in the stream of leases, and will reject causal_reads
+transactions in those intervals.  The causal reads guarantee is
+upheld, but spurious errors are raised between leases, as a
+consequence of the clock skew being greater than max_clock_skew.  In
+the following diagram 'r' shows reception time, and the timeline along
+the top shows the standby's local clock time.
+
+2     3     4     5     6     7     8     9    10    11
+r-----|
+            r-----|
+                        r-----|
+                                    r-----|
+                                                r-----|
+
+If there were no network latency, a standby whose clock is exactly 1
+second ahead of the primary's clock would perceive the stream of
+leases as being replaced just in time, so there is no gap.  Since in
+reality the time of receipt is some time after the time of
+transmission due to network latency, if the standby's clock is exactly
+1 second behind, then there will be small network-latency-sized gaps
+before the next lease arrives, but still no correctness problem with
+respect to the causal reads guarantee.
+
+1     2     3     4     5     6     7     8     9    10
+r-----------|
+            r-----------|
+                        r-----------|
+                                    r-----------|
+                                                r------...
+
+A standby whose clock is perfectly in sync with the primary's
+perceives the stream of leases overlapping (this matches the primary's
+perception of the leases it sent):
+
+0     1     2     3     4     5     6     7     8     9
+r-----------------|
+            r-----------------|
+                        r-----------------|
+                                    r-----------------|
+                                                r------...
+
+A standby whose clock is exactly 1 second behind the primary's
+perceives the stream of leases as overlapping even more, but the time
+of expiry as judged by the standby is no later than the time the
+primary will wait for if required ('x').  That is, if contact is lost
+with the standby, the primary can still reliably hold up causal reads
+commits until the standby has started raising the error in
+causal_reads transactions.
+
+-1    0     1     2     3     4     5     6     7     8
+r-----------------------|
+            r-----------------------|
+                        r-----------------------|
+                                    r------------------...
+                                                r------...
+
+
+A standby whose clock is 2 seconds behind the primary's would perceive
+the stream of leases overlapping even more, and the primary would no
+longer be able to wait for a lease to expire if it wanted to revoke
+it.  But because the expiry time is after local_clock_time +
+lease_time, the standby can immediately see that its own clock must be
+more than 1 second behind the primary's, so it ignores the lease and
+logs a clock skew warning.  In the following diagram a lease expiry
+time that is obviously generated by a primary with a clock set too far
+in the future compared to the local clock is shown with a '!'.
+
+-2    -1    0     1     2     3     4     5     6     7
+r-----------------------------!
+            r-----------------------------!
+                        r-----------------------------!
+                                    r------------------...
+                                                r------...
+
+A danger window exists when the standby's clock is more than
+max_clock_skew behind the primary's clock, but not more than
+max_clock_skew + network latency time behind.  If the clock difference
+is in that range, then the algorithm presented above which is based on
+time of receipt cannot detect that the local clock is too far behind.
+The consequence of this problem could be as follows:
+
+1.  The standby loses contact with the primary due to a network fault.
+
+2.  The primary decides to drop the standby from the set of available
+    causal reads standbys due to lack of keepalive responses or
+    excessive lag, which necessitates holding up commits of causal
+    reads transactions until the most recently sent lease expires, in
+    the belief that the standby will definitely have started raising
+    the 'causal reads unavailable' error in causal reads transactions
+    by that time, if it is still alive and servicing requests.
+
+3.  The standby still has clients connected and running queries.
+
+4.  Due to clock skew in the problematic range, in the standby's
+    opinion the lease lasts slightly longer than the primary waits.
+
+5.  For a short window at most the duration of the network latency
+    time, clients running causal reads transactions are allowed to see
+    potentially stale data.
+
+For this reason we say that the causal reads guarantee only holds as
+long as the absolute difference between the system clocks of the
+machines is no more than max_clock_skew.  The theory is that NTP makes
+it possible to reason about the maximum possible clock difference
+between machines and choose a value that allows for a much larger
+difference.  However, we do make a best effort attempt to detect
+wildly divergent systems as described above, to catch the case of
+servers not running a correctly configured ntp daemon, or with a clock
+so far out of whack that ntp refuses to fix it.
\ No newline at end of file
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 376ddf4..8240d0d 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -57,6 +57,11 @@
 #include "utils/builtins.h"
 #include "utils/ps_status.h"
 
+/* GUC variables */
+int causal_reads_timeout;
+bool causal_reads;
+char *causal_reads_standby_names;
+
 /* User-settable parameters for sync rep */
 char	   *SyncRepStandbyNames;
 
@@ -69,7 +74,7 @@ static int	SyncRepWaitMode = SYNC_REP_NO_WAIT;
 
 static void SyncRepQueueInsert(int mode);
 static void SyncRepCancelWait(void);
-static int	SyncRepWakeQueue(bool all, int mode);
+static int	SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn);
 
 static int	SyncRepGetStandbyPriority(void);
 
@@ -142,6 +147,198 @@ SyncRepCheckEarlyExit(void)
 }
 
 /*
+ * Check if we can stop waiting for causal consistency.  We can stop waiting
+ * when the following conditions are met:
+ *
+ * 1.  All walsenders currently in 'joining' or 'available' state have
+ * applied the target LSN.
+ *
+ * 2.  Any stall periods caused by standbys dropping out of 'available' state
+ * have passed, so that we can be sure that their leases have expired and they
+ * have started rejecting causal reads transactions.
+ *
+ * The output parameter 'waitingFor' is set to the number of nodes we are
+ * currently waiting for.  The output parameters 'stallTimeMillis' is set to
+ * the number of milliseconds we need to wait for to observe any current
+ * commit stall.
+ *
+ * Returns true if commit can return control, because every standby has either
+ * applied the LSN or started rejecting causal_reads transactions.
+ */
+static bool
+CausalReadsCommitCanReturn(XLogRecPtr XactCommitLSN,
+						   int *waitingFor,
+						   long *stallTimeMillis)
+{
+	int i;
+	TimestampTz now;
+
+	/* Count how many joining/available nodes we are waiting for. */
+	*waitingFor = 0;
+	for (i = 0; i < max_wal_senders; ++i)
+	{
+		WalSnd *walsnd = &WalSndCtl->walsnds[i];
+
+		/*
+		 * Assuming atomic read of pid_t, we can check walsnd->pid without
+		 * acquiring the spinlock to avoid memory synchronization costs for
+		 * unused walsender slots.  We see a value that existed sometime at
+		 * least as recently as the last memory barrier.
+		 */
+		if (walsnd->pid != 0)
+		{
+			/*
+			 * We need to hold the spinlock to read LSNs, because we can't be
+			 * sure they can be read atomically.
+			 */
+			SpinLockAcquire(&walsnd->mutex);
+			if (walsnd->pid != 0 && walsnd->causal_reads_state >= WALSNDCRSTATE_JOINING)
+			{
+				if (walsnd->apply < XactCommitLSN)
+					++*waitingFor;
+			}
+			SpinLockRelease(&walsnd->mutex);
+		}
+	}
+
+	/* Check if there is a stall in progress that we need to observe. */
+	now = GetCurrentTimestamp();
+	LWLockAcquire(SyncRepLock, LW_SHARED);
+	if (WalSndCtl->stall_causal_reads_until > now)
+	{
+		long seconds;
+		int usecs;
+
+		/* Compute how long we have to wait, rounded up to nearest ms. */
+		TimestampDifference(now, WalSndCtl->stall_causal_reads_until,
+							&seconds, &usecs);
+		*stallTimeMillis = seconds * 1000 + (usecs + 999) / 1000;
+	}
+	else
+		*stallTimeMillis = 0;
+	LWLockRelease(SyncRepLock);
+
+	/* We are done if we are not waiting for any nodes or stalls. */
+	return *waitingFor == 0 && *stallTimeMillis == 0;
+}
+
+/*
+ * Wait for causal consistency in causal_reads mode, if requested by user.
+ */
+void
+CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN)
+{
+	long stallTimeMillis;
+	int waitingFor;
+	char *ps_display_buffer = NULL;
+
+	/* Leave if we aren't in causal_reads mode. */
+	if (!causal_reads)
+		return;
+
+	for (;;)
+	{
+		/* Reset latch before checking state. */
+		ResetLatch(MyLatch);
+
+		/*
+		 * Join the queue to be woken up if any causal reads joining/available
+		 * standby applies XactCommitLSN or the set of causal reads standbys
+		 * changes (if we aren't already in the queue).  We don't actually know
+		 * if we need to wait for any peers to reach the target LSN yet, but
+		 * we have to register just in case before checking the walsenders'
+		 * state to avoid a race condition that could occur if we did it after
+		 * calling CausalReadsCommitCanReturn.  (SyncRepWaitForLSN doesn't
+		 * have to do this because it can check the highest-seen LSN in
+		 * walsndctl->lsn[mode] which is protected by SyncRepLock, the same
+		 * lock as the queues.  We can't do that here, because there is no
+		 * single highest-seen LSN that is useful.  We must check
+		 * walsnd->apply for all relevant walsenders.  Therefore we must
+		 * register for notifications first, so that we can be notified via
+		 * our latch of any standby applying the LSN we're interested in after
+		 * we check but before we start waiting, or we could wait forever for
+		 * something that has already happened.)
+		 */
+		LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+		if (MyProc->syncRepState != SYNC_REP_WAITING)
+		{
+			MyProc->waitLSN = XactCommitLSN;
+			MyProc->syncRepState = SYNC_REP_WAITING;
+			SyncRepQueueInsert(SYNC_REP_WAIT_CAUSAL_READS);
+			Assert(SyncRepQueueIsOrderedByLSN(SYNC_REP_WAIT_CAUSAL_READS));
+		}
+		LWLockRelease(SyncRepLock);
+
+		/* Check if we're done. */
+		if (CausalReadsCommitCanReturn(XactCommitLSN, &waitingFor, &stallTimeMillis))
+		{
+			SyncRepCancelWait();
+			break;
+		}
+
+		Assert(waitingFor > 0 || stallTimeMillis > 0);
+
+		/* If we aren't actually waiting for any standbys, leave the queue. */
+		if (waitingFor == 0)
+			SyncRepCancelWait();
+
+		/* Update the ps title. */
+		if (update_process_title)
+		{
+			char buffer[80];
+
+			/* Remember the old value if this is our first update. */
+			if (ps_display_buffer == NULL)
+			{
+				int len;
+				const char *ps_display = get_ps_display(&len);
+
+				ps_display_buffer = palloc(len + 1);
+				memcpy(ps_display_buffer, ps_display, len);
+				ps_display_buffer[len] = '\0';
+			}
+
+			snprintf(buffer, sizeof(buffer),
+					 "waiting for %d peer(s) to apply %X/%X%s",
+					 waitingFor,
+					 (uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN,
+					 stallTimeMillis > 0 ? " (stalling)" : "");
+			set_ps_display(buffer, false);
+		}
+
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit()) /* Calls SyncRepCancelWait() if true. */
+			break;
+
+		/*
+		 * If are still waiting for peers, then we wait for any joining or
+		 * available peer to reach the LSN (or possibly stop being in one of
+		 * those states or go away).
+		 *
+		 * If not, there must be a non-zero stall time, so we wait for that to
+		 * elapse.
+		 */
+		if (waitingFor > 0)
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH, -1);
+		else
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH | WL_TIMEOUT,
+					  stallTimeMillis);
+	}
+
+	/* There is no way out of the loop that could leave us in the queue. */
+	Assert(SHMQueueIsDetached(&(MyProc->syncRepLinks)));
+	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
+	MyProc->waitLSN = 0;
+
+	/* Restore the ps display. */
+	if (ps_display_buffer != NULL)
+	{
+		set_ps_display(ps_display_buffer, false);
+		pfree(ps_display_buffer);
+	}
+}
+
+/*
  * Wait for synchronous replication, if requested by user.
  *
  * Initially backends start in state SYNC_REP_NOT_WAITING and then
@@ -425,6 +622,53 @@ SyncRepGetSynchronousStandby(void)
 }
 
 /*
+ * Check if the current WALSender process's application_name matches a name in
+ * causal_reads_standby_names (including '*' for wildcard).
+ */
+bool
+CausalReadsPotentialStandby(void)
+{
+	char *rawstring;
+	List	   *elemlist;
+	ListCell   *l;
+	bool		found = false;
+
+	/* If the feature is disable, then no. */
+	if (causal_reads_timeout == 0)
+		return false;
+
+	/* Need a modifiable copy of string */
+	rawstring = pstrdup(causal_reads_standby_names);
+
+	/* Parse string into list of identifiers */
+	if (!SplitIdentifierString(rawstring, ',', &elemlist))
+	{
+		/* syntax error in list */
+		pfree(rawstring);
+		list_free(elemlist);
+		/* GUC machinery will have already complained - no need to do again */
+		return 0;
+	}
+
+	foreach(l, elemlist)
+	{
+		char	   *standby_name = (char *) lfirst(l);
+
+		if (pg_strcasecmp(standby_name, application_name) == 0 ||
+			pg_strcasecmp(standby_name, "*") == 0)
+		{
+			found = true;
+			break;
+		}
+	}
+
+	pfree(rawstring);
+	list_free(elemlist);
+
+	return found;
+}
+
+/*
  * Update the LSNs on each queue based upon our latest state. This
  * implements a simple policy of first-valid-standby-releases-waiter.
  *
@@ -432,23 +676,27 @@ SyncRepGetSynchronousStandby(void)
  * perhaps also which information we store as well.
  */
 void
-SyncRepReleaseWaiters(void)
+SyncRepReleaseWaiters(bool walsender_cr_available_or_joining)
 {
 	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
 	int			numapply = 0;
+	int			numcausalreadsapply = 0;
+	bool		is_highest_priority_sync_standby;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
-	 * potential sync standbys then we have nothing to do. If we are still
-	 * starting up, still running base backup or the current flush position
-	 * is still invalid, then leave quickly also.
+	 * potential sync standbys and not in a state that causal_reads waits for,
+	 * then we have nothing to do. If we are still starting up, still running
+	 * base backup or the current flush position is still invalid, then leave
+	 * quickly also.
 	 */
-	if (MyWalSnd->sync_standby_priority == 0 ||
-		MyWalSnd->state < WALSNDSTATE_STREAMING ||
-		XLogRecPtrIsInvalid(MyWalSnd->flush))
+	if (!walsender_cr_available_or_joining &&
+		(MyWalSnd->sync_standby_priority == 0 ||
+		 MyWalSnd->state < WALSNDSTATE_STREAMING ||
+		 XLogRecPtrIsInvalid(MyWalSnd->flush)))
 		return;
 
 	/*
@@ -458,13 +706,19 @@ SyncRepReleaseWaiters(void)
 	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
 	syncWalSnd = SyncRepGetSynchronousStandby();
 
-	/* We should have found ourselves at least */
-	Assert(syncWalSnd != NULL);
+	/*
+	 * If we aren't managing the highest priority standby then make a note of
+	 * that so we can announce a takeover in the log if we ever get that job.
+	 */
+	is_highest_priority_sync_standby = syncWalSnd == MyWalSnd;
+	if (!is_highest_priority_sync_standby)
+		announce_next_takeover = true;
 
 	/*
-	 * If we aren't managing the highest priority standby then just leave.
+	 * If we aren't managing the highest priority standby or a standby in
+	 * causal reads 'joining' or 'available' state, then just leave.
 	 */
-	if (syncWalSnd != MyWalSnd)
+	if (!is_highest_priority_sync_standby && !walsender_cr_available_or_joining)
 	{
 		LWLockRelease(SyncRepLock);
 		announce_next_takeover = true;
@@ -473,24 +727,45 @@ SyncRepReleaseWaiters(void)
 
 	/*
 	 * Set the lsn first so that when we wake backends they will release up to
-	 * this location.
+	 * this location.  For the single-standby synchronous commit levels, we
+	 * only do this if we are the current synchronous standby and we are
+	 * advancing the LSN further than it has been advanced before, so that
+	 * SyncRepWaitForLSN can skip waiting in some cases.
 	 */
-	if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
-		numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
-		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+	if (is_highest_priority_sync_standby)
 	{
-		walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
-		numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY);
+		if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
+			numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE,
+										MyWalSnd->write);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
+			numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH,
+										MyWalSnd->flush);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+			numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY,
+										MyWalSnd->apply);
+		}
 	}
 
+	/*
+	 * For causal_reads, all walsenders currently in available or joining
+	 * state must reach the LSN on their own, and standbys will reach LSNs in
+	 * any order.  It doesn't make sense to keep the highest seen LSN in a
+	 * single walsndctl->lsn element.  (CausalReadsWaitForLSN has handling for
+	 * LSNs that have already been reached).
+	 */
+	if (walsender_cr_available_or_joining)
+		numcausalreadsapply =
+			SyncRepWakeQueue(false, SYNC_REP_WAIT_CAUSAL_READS,
+							 MyWalSnd->apply);
+
 	LWLockRelease(SyncRepLock);
 
 	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%x",
@@ -502,7 +777,7 @@ SyncRepReleaseWaiters(void)
 	 * If we are managing the highest priority standby, though we weren't
 	 * prior to this, then announce we are now the sync standby.
 	 */
-	if (announce_next_takeover)
+	if (is_highest_priority_sync_standby && announce_next_takeover)
 	{
 		announce_next_takeover = false;
 		ereport(LOG,
@@ -577,9 +852,8 @@ SyncRepGetStandbyPriority(void)
  * Must hold SyncRepLock.
  */
 static int
-SyncRepWakeQueue(bool all, int mode)
+SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn)
 {
-	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	PGPROC	   *proc = NULL;
 	PGPROC	   *thisproc = NULL;
 	int			numprocs = 0;
@@ -596,7 +870,7 @@ SyncRepWakeQueue(bool all, int mode)
 		/*
 		 * Assume the queue is ordered by LSN
 		 */
-		if (!all && walsndctl->lsn[mode] < proc->waitLSN)
+		if (!all && lsn < proc->waitLSN)
 			return numprocs;
 
 		/*
@@ -656,7 +930,7 @@ SyncRepUpdateSyncStandbysDefined(void)
 			int			i;
 
 			for (i = 0; i < NUM_SYNC_REP_WAIT_MODE; i++)
-				SyncRepWakeQueue(true, i);
+				SyncRepWakeQueue(true, i, InvalidXLogRecPtr);
 		}
 
 		/*
@@ -708,13 +982,31 @@ SyncRepQueueIsOrderedByLSN(int mode)
 #endif
 
 /*
+ * Make sure that CausalReadsWaitForLSN can't return until after the given
+ * lease expiry time has been reached.  In other words, revoke the lease.
+ *
+ * Wake up all backends waiting in CausalReadsWaitForLSN, because the set of
+ * available/joining peers has changed, and there is a new stall time they
+ * need to observe.
+ */
+void
+CausalReadsBeginStall(TimestampTz lease_expiry_time)
+{
+	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+	WalSndCtl->stall_causal_reads_until =
+		Max(WalSndCtl->stall_causal_reads_until, lease_expiry_time);
+	SyncRepWakeQueue(true, SYNC_REP_WAIT_CAUSAL_READS, InvalidXLogRecPtr);
+	LWLockRelease(SyncRepLock);
+}
+
+/*
  * ===========================================================
  * Synchronous Replication functions executed by any process
  * ===========================================================
  */
 
 bool
-check_synchronous_standby_names(char **newval, void **extra, GucSource source)
+check_standby_names(char **newval, void **extra, GucSource source)
 {
 	char	   *rawstring;
 	List	   *elemlist;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index faea9ff..ff3a4db 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -55,6 +55,7 @@
 #include "libpq/pqformat.h"
 #include "libpq/pqsignal.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "replication/walsender.h"
 #include "storage/ipc.h"
@@ -149,7 +150,8 @@ static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
 static void XLogWalRcvSendReply(bool force, bool requestReply, bool includeApplyTimestamp);
 static void XLogWalRcvSendHSFeedback(bool immed);
-static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
+static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+								  TimestampTz *causalReadsUntil);
 
 /* Signal handlers */
 static void WalRcvSigHupHandler(SIGNAL_ARGS);
@@ -813,6 +815,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 	XLogRecPtr	walEnd;
 	TimestampTz sendTime;
 	bool		replyRequested;
+	TimestampTz causalReadsLease;
 
 	resetStringInfo(&incoming_message);
 
@@ -833,7 +836,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				walEnd = pq_getmsgint64(&incoming_message);
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, NULL);
 
 				buf += hdrlen;
 				len -= hdrlen;
@@ -845,7 +848,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				bool reportApplyTimestamp = false;
 
 				/* copy message to StringInfo */
-				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
+				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char) + sizeof(int64);
 				if (len != hdrlen)
 					ereport(ERROR,
 							(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -857,8 +860,10 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
 				replyRequested = pq_getmsgbyte(&incoming_message);
+				causalReadsLease = IntegerTimestampToTimestampTz(
+					pq_getmsgint64(&incoming_message));
 
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, &causalReadsLease);
 
 				/*
 				 * If no apply timestamps have been sent at the request of the
@@ -1207,15 +1212,52 @@ XLogWalRcvSendHSFeedback(bool immed)
  * Update shared memory status upon receiving a message from primary.
  *
  * 'walEnd' and 'sendTime' are the end-of-WAL and timestamp of the latest
- * message, reported by primary.
+ * message, reported by primary.  'causalReadsLease' is a pointer to
+ * the time the primary promises that this standby can safely claim to be
+ * causally consistent, to 0 if it cannot, or a NULL pointer for no change.
  */
 static void
-ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
+ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+					  TimestampTz *causalReadsLease)
 {
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
 	static TimestampTz lastRecordedTimestamp = 0;
 
+	/* Sanity check for the causalReadsLease time. */
+	if (causalReadsLease != NULL && *causalReadsLease != 0)
+	{
+		/* Deduce max_clock_skew from the causalReadsLease and sendTime. */
+#ifdef HAVE_INT64_TIMESTAMP
+		int64 diffMillis = (*causalReadsLease - sendTime) / 1000;
+#else
+		int64 diffMillis = (*causalReadsLease - sendTime) * 1000;
+#endif
+		int64 max_clock_skew = diffMillis / (CAUSAL_READS_CLOCK_SKEW_RATIO - 1);
+
+		if (sendTime > TimestampTzPlusMilliseconds(lastMsgReceiptTime, max_clock_skew))
+		{
+			/*
+			 * The primary's clock is more than max_clock_skew + network
+			 * latency ahead of the standby's clock.  (If the primary's clock
+			 * is more than max_clock_skew ahead of the standby's clock, but
+			 * by less than the network latency, then there isn't much we can
+			 * do to detect that; but it still seems useful to have this basic
+			 * sanity check for wildly misconfigured servers.)
+			 */
+			elog(LOG, "the primary server's clock time is too far ahead");
+			causalReadsLease = NULL;
+		}
+		/*
+		 * We could also try to detect cases where sendTime is more than
+		 * max_clock_skew in the past according to the standby's clock, but
+		 * that is indistinguishable from network latency/buffering, so we
+		 * could produce misleading error messages; if we do nothing, the
+		 * consequence is 'standby is not available for causal reads' errors
+		 * which should cause the user to investigate.
+		 */
+	}
+
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
 	if (walrcv->latestWalEnd < walEnd)
@@ -1223,6 +1265,8 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->latestWalEnd = walEnd;
 	walrcv->lastMsgSendTime = sendTime;
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
+	if (causalReadsLease != NULL)
+		walrcv->causalReadsLease = *causalReadsLease;
 	SpinLockRelease(&walrcv->mutex);
 
 	/*
diff --git a/src/backend/replication/walreceiverfuncs.c b/src/backend/replication/walreceiverfuncs.c
index 5f6e423..e502f74 100644
--- a/src/backend/replication/walreceiverfuncs.c
+++ b/src/backend/replication/walreceiverfuncs.c
@@ -28,6 +28,7 @@
 #include "replication/walreceiver.h"
 #include "storage/pmsignal.h"
 #include "storage/shmem.h"
+#include "utils/guc.h"
 #include "utils/timestamp.h"
 
 WalRcvData *WalRcv = NULL;
@@ -374,3 +375,21 @@ GetReplicationTransferLatency(void)
 
 	return ms;
 }
+
+/*
+ * Used by snapmgr to check if this standby has a valid lease, granting it the
+ * right to consider itself available for causal reads.
+ */
+bool
+WalRcvCausalReadsAvailable(void)
+{
+	WalRcvData *walrcv = WalRcv;
+	TimestampTz now = GetCurrentTimestamp();
+	bool result;
+
+	SpinLockAcquire(&walrcv->mutex);
+	result = walrcv->causalReadsLease != 0 && now <= walrcv->causalReadsLease;
+	SpinLockRelease(&walrcv->mutex);
+
+	return result;
+}
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 16d7abc..b4dad72 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -154,9 +154,20 @@ static StringInfoData tmpbuf;
  */
 static TimestampTz last_reply_timestamp = 0;
 
+static TimestampTz last_keepalive_timestamp = 0;
+
 /* Have we sent a heartbeat message asking for reply, since last reply? */
 static bool waiting_for_ping_response = false;
 
+/* How long do need to stay in JOINING state? */
+static XLogRecPtr causal_reads_joining_until = 0;
+
+/* The last causal reads lease sent to the standby. */
+static TimestampTz causal_reads_last_lease = 0;
+
+/* Is this WALSender listed in causal_reads_standby_names? */
+static bool am_potential_causal_reads_standby = false;
+
 /*
  * While streaming WAL in Copy mode, streamingDoneSending is set to true
  * after we have sent CopyDone. We should not send any more CopyData messages
@@ -242,6 +253,57 @@ InitWalSender(void)
 	SendPostmasterSignal(PMSIGNAL_ADVANCE_STATE_MACHINE);
 }
 
+ /*
+ * If we are exiting unexpectedly, we may need to communicate with concurrent
+ * causal_reads commits to maintain the causal consistency guarantee.
+ */
+static void
+PrepareUncleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * We've lost contact with the standby, but it may still be alive.  We
+		 * can't let any causal_reads transactions return until we've stalled
+		 * for long enough for a zombie standby to start raising errors
+		 * because its lease has expired.
+		 */
+		elog(LOG, "standby \"%s\" is lost (no longer available for causal reads)", application_name);
+		CausalReadsBeginStall(causal_reads_last_lease);
+
+		/*
+		 * We set the state to a lower level _after_ beginning the stall,
+		 * otherwise there would be a tiny window where commits could return
+		 * without observing the stall.
+		 */
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
+ * We are shutting down because we received a goodbye message from the
+ * walreceiver.
+ */
+static void
+PrepareCleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * The standby is shutting down, so it won't be running any more
+		 * transactions.  It is therefore safe to stop waiting for it, and no
+		 * stall is necessary.
+		 */
+		elog(LOG, "standby \"%s\" is leaving (no longer available for causal reads)", application_name);
+
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
 /*
  * Clean up after an error.
  *
@@ -266,7 +328,10 @@ WalSndErrorCleanup(void)
 
 	replication_active = false;
 	if (walsender_ready_to_stop)
+	{
+		PrepareUncleanExit();
 		proc_exit(0);
+	}
 
 	/* Revert back to startup state */
 	WalSndSetState(WALSNDSTATE_STARTUP);
@@ -278,6 +343,8 @@ WalSndErrorCleanup(void)
 static void
 WalSndShutdown(void)
 {
+	PrepareUncleanExit();
+
 	/*
 	 * Reset whereToSendOutput to prevent ereport from attempting to send any
 	 * more messages to the standby.
@@ -1388,6 +1455,7 @@ ProcessRepliesIfAny(void)
 		if (r < 0)
 		{
 			/* unexpected error or EOF */
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1404,6 +1472,7 @@ ProcessRepliesIfAny(void)
 		resetStringInfo(&reply_message);
 		if (pq_getmessage(&reply_message, 0))
 		{
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1453,6 +1522,7 @@ ProcessRepliesIfAny(void)
 				 * 'X' means that the standby is closing down the socket.
 				 */
 			case 'X':
+				PrepareCleanExit();
 				proc_exit(0);
 
 			default:
@@ -1584,6 +1654,83 @@ ProcessStandbyReplyMessage(void)
 	 */
 	{
 		WalSnd *walsnd = MyWalSnd;
+		WalSndCausalReadsState causal_reads_state = walsnd->causal_reads_state;
+		bool causal_reads_state_changed = false;
+		bool causal_reads_set_joining_until = false;
+
+		/*
+		 * Handle causal reads state transitions, if a causal_reads_timeout is
+		 * configured, this standby is listed in causal_reads_standby_names,
+		 * and we are a primary database (not a cascading standby).
+		 */
+		if (am_potential_causal_reads_standby &&
+			!am_cascading_walsender &&
+			applyLagUs >= 0)
+		{
+			if (applyLagUs / 1000 < causal_reads_timeout)
+			{
+				if (causal_reads_state == WALSNDCRSTATE_UNAVAILABLE)
+				{
+					/*
+					 * The standby is applying fast enough.  We can't grant a
+					 * lease yet though, we need to wait for everything that
+					 * was committed while this standby was unavailable to be
+					 * applied first.  We move to joining state while we wait
+					 * for the standby to catch up.
+					 */
+					causal_reads_state = WALSNDCRSTATE_JOINING;
+					causal_reads_set_joining_until = true;
+					causal_reads_state_changed = true;
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING &&
+						 applyPtr >= causal_reads_joining_until)
+				{
+					/*
+					 * The standby has applied everything committed before we
+					 * reached joining state, and has been waiting for remote
+					 * apply on this standby while it's been in joining state,
+					 * so it is safe to move to available state and send a
+					 * lease.
+					 */
+					causal_reads_state = WALSNDCRSTATE_AVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+			else
+			{
+				if (causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+					/*
+					 * We are dropping a causal reads available standby, so we
+					 * mustn't let any commit command that is waiting in
+					 * CausalReadsWaitForLSN return until we are sure that the
+					 * standby definitely knows that it's not available and
+					 * starts raising errors for causal_reads transactions.
+					 * TODO: We could just wait until the standby acks that
+					 * its lease has been cancelled, and start numbering
+					 * keepalives and sending the number back in replies, so
+					 * we know it's acking the right message; then lagging
+					 * standbys would be less disruptive, but for now we just
+					 * wait for the lease to expire, as we do when we lose
+					 * contact with a standby, for the sake of simplicity.
+					 */
+					CausalReadsBeginStall(causal_reads_last_lease);
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING)
+				{
+					/*
+					 * Dropping a joining standby doesn't require a stall,
+					 * because the standby doesn't think it's available, so
+					 * it's already raising the error for causal_reads
+					 * transactions.
+					 */
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+		}
 
 		SpinLockAcquire(&walsnd->mutex);
 		walsnd->write = writePtr;
@@ -1591,11 +1738,33 @@ ProcessStandbyReplyMessage(void)
 		walsnd->apply = applyPtr;
 		if (applyLagUs >= 0)
 			walsnd->applyLagUs = applyLagUs;
+		walsnd->causal_reads_state = causal_reads_state;
 		SpinLockRelease(&walsnd->mutex);
+
+		if (causal_reads_set_joining_until)
+		{
+			/*
+			 * Record the end of the primary's WAL at some arbitrary point
+			 * observed _after_ we moved to joining state (so that causal
+			 * reads commits start waiting, closing a race).  The standby
+			 * won't become available until it has replayed up to here.
+			 */
+			causal_reads_joining_until = GetFlushRecPtr();
+		}
+
+		if (causal_reads_state_changed)
+		{
+			WalSndKeepalive(true);
+			elog(LOG, "standby \"%s\" is %s", application_name,
+				 causal_reads_state == WALSNDCRSTATE_UNAVAILABLE ? "unavailable for causal reads" :
+				 causal_reads_state == WALSNDCRSTATE_JOINING ? "joining as a causal reads standby..." :
+				 causal_reads_state == WALSNDCRSTATE_AVAILABLE ? "available for causal reads" :
+				 "UNKNOWN");
+		}
 	}
 
 	if (!am_cascading_walsender)
-		SyncRepReleaseWaiters();
+		SyncRepReleaseWaiters(MyWalSnd->causal_reads_state >= WALSNDCRSTATE_JOINING);
 
 	/*
 	 * Advance our local xmin horizon when the client confirmed a flush.
@@ -1736,33 +1905,53 @@ ProcessStandbyHSFeedbackMessage(void)
  * If wal_sender_timeout is enabled we want to wake up in time to send
  * keepalives and to abort the connection if wal_sender_timeout has been
  * reached.
+ *
+ * But if causal_reads_timeout is enabled, we override that and send
+ * keepalives at a constant rate to replace expiring leases.
  */
 static long
 WalSndComputeSleeptime(TimestampTz now)
 {
 	long		sleeptime = 10000;		/* 10 s */
 
-	if (wal_sender_timeout > 0 && last_reply_timestamp > 0)
+	if ((wal_sender_timeout > 0 && last_reply_timestamp > 0) ||
+		am_potential_causal_reads_standby)
 	{
 		TimestampTz wakeup_time;
 		long		sec_to_timeout;
 		int			microsec_to_timeout;
 
-		/*
-		 * At the latest stop sleeping once wal_sender_timeout has been
-		 * reached.
-		 */
-		wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-												  wal_sender_timeout);
-
-		/*
-		 * If no ping has been sent yet, wakeup when it's time to do so.
-		 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
-		 * the timeout passed without a response.
-		 */
-		if (!waiting_for_ping_response)
+		if (am_potential_causal_reads_standby)
+		{
+			/*
+			 * Leases last for a period of between 50% and 100% of
+			 * causal_reads_timeout, depending on clock skew, assuming clock
+			 * skew is under the 25% of causal_reads_timeout.  We send new
+			 * leases every half a lease, so that there are no gaps between
+			 * leases.
+			 */
+			wakeup_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+													  causal_reads_timeout /
+													  CAUSAL_READS_KEEPALIVE_RATIO);
+		}
+		else
+		{
+			/*
+			 * At the latest stop sleeping once wal_sender_timeout has been
+			 * reached.
+			 */
 			wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-													  wal_sender_timeout / 2);
+													  wal_sender_timeout);
+
+			/*
+			 * If no ping has been sent yet, wakeup when it's time to do so.
+			 * WalSndKeepaliveIfNecessary() wants to send a keepalive once
+			 * half of the timeout passed without a response.
+			 */
+			if (!waiting_for_ping_response)
+				wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+														  wal_sender_timeout / 2);
+		}
 
 		/* Compute relative time until wakeup. */
 		TimestampDifference(now, wakeup_time,
@@ -1778,20 +1967,33 @@ WalSndComputeSleeptime(TimestampTz now)
 /*
  * Check whether there have been responses by the client within
  * wal_sender_timeout and shutdown if not.
+ *
+ * If causal_reads_timeout is configured we override that, so that
+ * unresponsive standbys are detected sooner.
  */
 static void
 WalSndCheckTimeOut(TimestampTz now)
 {
 	TimestampTz timeout;
+	int allowed_time;
 
 	/* don't bail out if we're doing something that doesn't require timeouts */
 	if (last_reply_timestamp <= 0)
 		return;
 
-	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
-										  wal_sender_timeout);
+	/*
+	 * If a causal_reads_timeout is configured, it is used instead of
+	 * wal_sender_timeout, to limit the time before an unresponsive causal
+	 * reads standby is dropped.
+	 */
+	if (am_potential_causal_reads_standby)
+		allowed_time = causal_reads_timeout;
+	else
+		allowed_time = wal_sender_timeout;
 
-	if (wal_sender_timeout > 0 && now >= timeout)
+	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
+										  allowed_time);
+	if (allowed_time > 0 && now >= timeout)
 	{
 		/*
 		 * Since typically expiration of replication timeout means
@@ -1824,6 +2026,9 @@ WalSndLoop(WalSndSendDataCallback send_data)
 	last_reply_timestamp = GetCurrentTimestamp();
 	waiting_for_ping_response = false;
 
+	/* Check if we are managing potential causal_reads standby. */
+	am_potential_causal_reads_standby = CausalReadsPotentialStandby();
+
 	/*
 	 * Loop until we reach the end of this timeline or the client requests to
 	 * stop streaming.
@@ -1984,6 +2189,7 @@ InitWalSenderSlot(void)
 			walsnd->flush = InvalidXLogRecPtr;
 			walsnd->apply = InvalidXLogRecPtr;
 			walsnd->state = WALSNDSTATE_STARTUP;
+			walsnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
 			walsnd->latch = &MyProc->procLatch;
 			SpinLockRelease(&walsnd->mutex);
 			/* don't need the lock anymore */
@@ -2753,6 +2959,24 @@ WalSndGetStateString(WalSndState state)
 	return "UNKNOWN";
 }
 
+/*
+ * Return a string constant representing the causal reads state. This is used
+ * in system views, and should *not* be translated.
+ */
+static const char *
+WalSndGetCausalReadsStateString(WalSndCausalReadsState causal_reads_state)
+{
+	switch (causal_reads_state)
+	{
+		case WALSNDCRSTATE_UNAVAILABLE:
+			return "unavailable";
+		case WALSNDCRSTATE_JOINING:
+			return "joining";
+		case WALSNDCRSTATE_AVAILABLE:
+			return "available";
+	}
+	return "UNKNOWN";
+}
 
 /*
  * Returns activity of walsenders, including pids and xlog locations sent to
@@ -2761,7 +2985,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	9
+#define PG_STAT_GET_WAL_SENDERS_COLS	10
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2812,6 +3036,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		int64		applyLagUs;
 		int			priority;
 		WalSndState state;
+		WalSndCausalReadsState causalReadsState;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
 		bool		nulls[PG_STAT_GET_WAL_SENDERS_COLS];
 
@@ -2821,6 +3046,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		SpinLockAcquire(&walsnd->mutex);
 		sentPtr = walsnd->sentPtr;
 		state = walsnd->state;
+		causalReadsState = walsnd->causal_reads_state;
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
@@ -2895,6 +3121,9 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				values[8] = CStringGetTextDatum("sync");
 			else
 				values[8] = CStringGetTextDatum("potential");
+
+			values[9] =
+				CStringGetTextDatum(WalSndGetCausalReadsStateString(causalReadsState));
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
@@ -2914,14 +3143,52 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 static void
 WalSndKeepalive(bool requestReply)
 {
+	TimestampTz now;
+	TimestampTz causal_reads_lease;
+
 	elog(DEBUG2, "sending replication keepalive");
 
+	/*
+	 * If the walsender currently deems the standby to be available for causal
+	 * reads, then it grants a causal reads lease.  The lease authorizes the
+	 * standby to consider itself available for causal reads until a short
+	 * time in the future.  The primary promises to uphold the causal reads
+	 * guarantee until that time, by stalling commits until the the lease has
+	 * expired if necessary.
+	 */
+	now = GetCurrentTimestamp();
+	if (MyWalSnd->causal_reads_state < WALSNDCRSTATE_AVAILABLE)
+		causal_reads_lease = 0; /* Not available, no lease granted. */
+	else
+	{
+		/*
+		 * Since this timestamp is being sent to the standby where it will be
+		 * compared against a time generated by the standby's system clock, we
+		 * must consider clock skew.  First, we decide on a maximum tolerable
+		 * difference between system clocks.  If the primary's clock is ahead
+		 * of the standby's by more than this, then all bets are off (the
+		 * standby could falsely believe it has a valid lease).  If the
+		 * primary's clock is behind the standby's by more than this, then the
+		 * standby will err the other way and generate spurious errors in
+		 * causal_reads mode.  Rather than having a separate GUC for this, we
+		 * derive it from causal_reads_timeout.
+		 */
+		int max_clock_skew = causal_reads_timeout / CAUSAL_READS_CLOCK_SKEW_RATIO;
+
+		/* Compute and remember the expiry time of the lease we're granting. */
+		causal_reads_last_lease = TimestampTzPlusMilliseconds(now, causal_reads_timeout);
+		/* The version we'll send to the standby is adjusted to tolerate clock skew. */
+		causal_reads_lease =
+			TimestampTzPlusMilliseconds(causal_reads_last_lease, -max_clock_skew);
+	}
+
 	/* construct the message... */
 	resetStringInfo(&output_message);
 	pq_sendbyte(&output_message, 'k');
 	pq_sendint64(&output_message, sentPtr);
-	pq_sendint64(&output_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(now));
 	pq_sendbyte(&output_message, requestReply ? 1 : 0);
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(causal_reads_lease));
 
 	/* ... and send it wrapped in CopyData */
 	pq_putmessage_noblock('d', output_message.data, output_message.len);
@@ -2939,23 +3206,35 @@ WalSndKeepaliveIfNecessary(TimestampTz now)
 	 * Don't send keepalive messages if timeouts are globally disabled or
 	 * we're doing something not partaking in timeouts.
 	 */
-	if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
-		return;
-
-	if (waiting_for_ping_response)
-		return;
+	if (!am_potential_causal_reads_standby)
+	{
+		if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
+			return;
+		if (waiting_for_ping_response)
+			return;
+	}
 
 	/*
 	 * If half of wal_sender_timeout has lapsed without receiving any reply
 	 * from the standby, send a keep-alive message to the standby requesting
 	 * an immediate reply.
+	 *
+	 * If causal_reads_timeout has been configured, use it to control
+	 * keepalive intervals rather than wal_sender_timeout, so that we can keep
+	 * replacing leases at the right frequency.
 	 */
-	ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-											wal_sender_timeout / 2);
+	if (am_potential_causal_reads_standby)
+		ping_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+												causal_reads_timeout /
+												CAUSAL_READS_KEEPALIVE_RATIO);
+	else
+		ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+												wal_sender_timeout / 2);
 	if (now >= ping_time)
 	{
 		WalSndKeepalive(true);
 		waiting_for_ping_response = true;
+		last_keepalive_timestamp = now;
 
 		/* Try to flush pending output to the client */
 		if (pq_flush_if_writable() != 0)
diff --git a/src/backend/utils/errcodes.txt b/src/backend/utils/errcodes.txt
index 1a920e8..9fdc992 100644
--- a/src/backend/utils/errcodes.txt
+++ b/src/backend/utils/errcodes.txt
@@ -303,6 +303,7 @@ Section: Class 40 - Transaction Rollback
 40001    E    ERRCODE_T_R_SERIALIZATION_FAILURE                              serialization_failure
 40003    E    ERRCODE_T_R_STATEMENT_COMPLETION_UNKNOWN                       statement_completion_unknown
 40P01    E    ERRCODE_T_R_DEADLOCK_DETECTED                                  deadlock_detected
+40P02    E    ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE                         causal_reads_not_available
 
 Section: Class 42 - Syntax Error or Access Rule Violation
 
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 06cb166..ac422e7 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -1634,6 +1634,16 @@ static struct config_bool ConfigureNamesBool[] =
 	},
 
 	{
+		{"causal_reads", PGC_USERSET, REPLICATION_STANDBY,
+		 gettext_noop("Enables causal reads."),
+		 NULL
+		},
+		&causal_reads,
+		false,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"syslog_sequence_numbers", PGC_SIGHUP, LOGGING_WHERE,
 			gettext_noop("Add sequence number to syslog messages to avoid duplicate suppression."),
 			NULL
@@ -1811,6 +1821,17 @@ static struct config_int ConfigureNamesInt[] =
 	},
 
 	{
+		{"causal_reads_timeout", PGC_SIGHUP, REPLICATION_STANDBY,
+			gettext_noop("Sets the maximum apply lag before causal reads standbys are no longer available."),
+			NULL,
+			GUC_UNIT_MS
+		},
+		&causal_reads_timeout,
+		0, 0, INT_MAX,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"max_connections", PGC_POSTMASTER, CONN_AUTH_SETTINGS,
 			gettext_noop("Sets the maximum number of concurrent connections."),
 			NULL
@@ -3454,7 +3475,18 @@ static struct config_string ConfigureNamesString[] =
 		},
 		&SyncRepStandbyNames,
 		"",
-		check_synchronous_standby_names, NULL, NULL
+		check_standby_names, NULL, NULL
+	},
+
+	{
+		{"causal_reads_standby_names", PGC_SIGHUP, REPLICATION_MASTER,
+			gettext_noop("List of names of potential causal reads standbys."),
+			NULL,
+			GUC_LIST_INPUT
+		},
+		&causal_reads_standby_names,
+		"*",
+		check_standby_names, NULL, NULL
 	},
 
 	{
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index ec4427f..fcc2c35 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -244,6 +244,15 @@
 				# from standby(s); '*' = all
 #vacuum_defer_cleanup_age = 0	# number of xacts by which cleanup is delayed
 
+#causal_reads_timeout = 0s      # maximum replication delay to tolerate from
+                                # standbys before dropping them from the set of
+				# available causal reads peers; 0 to disable
+				# causal reads
+
+#causal_reads_standy_names = '*'
+                                # standby servers that can potentially become
+				# available for causal reads; '*' = all
+
 # - Standby Servers -
 
 # These settings are ignored on a master server.
@@ -266,6 +275,14 @@
 #wal_retrieve_retry_interval = 5s	# time to wait before retrying to
 					# retrieve WAL after a failed attempt
 
+# - All Servers -
+
+#causal_reads = off                     # "on" in any pair of consecutive
+                                        # transactions guarantees that the second
+					# can see the first (even if the second
+					# is run on a standby), or will raise an
+					# error to report that the standby is
+					# unavailable for causal reads
 
 #------------------------------------------------------------------------------
 # QUERY TUNING
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index b88e012..6336240 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -46,8 +46,11 @@
 
 #include "access/transam.h"
 #include "access/xact.h"
+#include "access/xlog.h"
 #include "lib/pairingheap.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
+#include "replication/walreceiver.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
 #include "storage/procarray.h"
@@ -209,6 +212,16 @@ GetTransactionSnapshot(void)
 				 "cannot take query snapshot during a parallel operation");
 
 		/*
+		 * In causal_reads mode on a standby, check if we have definitely
+		 * applied WAL for any COMMIT that returned successfully on the
+		 * primary.
+		 */
+		if (causal_reads && RecoveryInProgress() && !WalRcvCausalReadsAvailable())
+			ereport(ERROR,
+					(errcode(ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE),
+					 errmsg("standby is not available for causal reads")));
+
+		/*
 		 * In transaction-snapshot mode, the first snapshot must live until
 		 * end of xact regardless of what the caller does with it, so we must
 		 * make a copy of it rather than returning CurrentSnapshotData
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 4054726..c0d7173 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2712,7 +2712,7 @@ DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f
 DESCR("statistics: information about currently active backends");
 DATA(insert OID = 3318 (  pg_stat_get_progress_info           PGNSP PGUID 12 1 100 0 0 f f f f t t s r 1 0 2249 "25" "{25,23,26,26,20,20,20,20,20,20,20,20,20,20}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{cmdtype,pid,datid,relid,param1,param2,param3,param4,param5,param6,param7,param8,param9,param10}" _null_ _null_ pg_stat_get_progress_info _null_ _null_ _null_ ));
 DESCR("statistics: information about progress of backends running maintenance command");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25}" "{o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25,25}" "{o,o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state,causal_reads_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index c005a42..dbfd601 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -24,14 +24,33 @@
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
 #define SYNC_REP_WAIT_APPLY		2
+#define SYNC_REP_WAIT_CAUSAL_READS 3
 
-#define NUM_SYNC_REP_WAIT_MODE	3
+#define NUM_SYNC_REP_WAIT_MODE	4
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
 #define SYNC_REP_WAITING			1
 #define SYNC_REP_WAIT_COMPLETE		2
 
+/*
+ * ratio of causal_read_timeout to max_clock_skew (4 means than the maximum
+ * tolerated clock difference between primary and standbys using causal_reads
+ * is 1/4 of causal_reads_timeout)
+ */
+#define CAUSAL_READS_CLOCK_SKEW_RATIO 4
+
+/*
+ * ratio of causal_reads_timeout to keepalive time (2 means that the effective
+ * keepalive time is 1/2 of the causal_reads_timeout GUC when it is non-zero)
+ */
+#define CAUSAL_READS_KEEPALIVE_RATIO 2
+
+/* GUC variables */
+extern int causal_reads_timeout;
+extern bool causal_reads;
+extern char *causal_reads_standby_names;
+
 /* user-settable parameters for synchronous replication */
 extern char *SyncRepStandbyNames;
 
@@ -43,16 +62,23 @@ extern void SyncRepCleanupAtProcExit(void);
 
 /* called by wal sender */
 extern void SyncRepInitConfig(void);
-extern void SyncRepReleaseWaiters(void);
+extern void SyncRepReleaseWaiters(bool walsender_cr_available_or_joining);
 
 /* called by checkpointer */
 extern void SyncRepUpdateSyncStandbysDefined(void);
 
+/* called by user backend (xact.c) */
+extern void CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN);
+
+/* called by wal sender */
+extern void CausalReadsBeginStall(TimestampTz lease_expiry_time);
+extern bool CausalReadsPotentialStandby(void);
+
 /* forward declaration to avoid pulling in walsender_private.h */
 struct WalSnd;
 extern struct WalSnd *SyncRepGetSynchronousStandby(void);
 
-extern bool check_synchronous_standby_names(char **newval, void **extra, GucSource source);
+extern bool check_standby_names(char **newval, void **extra, GucSource source);
 extern void assign_synchronous_commit(int newval, void *extra);
 
 #endif   /* _SYNCREP_H */
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index ee60dd6..b6fa9eb 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -80,6 +80,13 @@ typedef struct
 	TimeLineID	receivedTLI;
 
 	/*
+	 * causalReadsLease is the time until which the primary has authorized
+	 * this standby to consider itself available for causal_reads mode, or 0
+	 * for not authorized.
+	 */
+	TimestampTz causalReadsLease;
+
+	/*
 	 * latestChunkStart is the starting byte position of the current "batch"
 	 * of received WAL.  It's actually the same as the previous value of
 	 * receivedUpto before the last flush to disk.  Startup process can use
@@ -168,4 +175,6 @@ extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
 extern void WalRcvWakeup(void);
 
+extern bool WalRcvCausalReadsAvailable(void);
+
 #endif   /* _WALRECEIVER_H */
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 4de43e8..f6e0e9e 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -27,6 +27,13 @@ typedef enum WalSndState
 	WALSNDSTATE_STREAMING
 } WalSndState;
 
+typedef enum WalSndCausalReadsState
+{
+	WALSNDCRSTATE_UNAVAILABLE = 0,
+	WALSNDCRSTATE_JOINING,
+	WALSNDCRSTATE_AVAILABLE
+} WalSndCausalReadsState;
+
 /*
  * Each walsender has a WalSnd struct in shared memory.
  */
@@ -34,6 +41,7 @@ typedef struct WalSnd
 {
 	pid_t		pid;			/* this walsender's process id, or 0 */
 	WalSndState state;			/* this walsender's state */
+	WalSndCausalReadsState causal_reads_state; /* the walsender's causal reads state */
 	XLogRecPtr	sentPtr;		/* WAL has been sent up to this point */
 	bool		needreload;		/* does currently-open file need to be
 								 * reloaded? */
@@ -89,6 +97,12 @@ typedef struct
 	 */
 	bool		sync_standbys_defined;
 
+	/*
+	 * Until when must commits in causal_reads stall?  This is used to wait
+	 * for causal reads leases to expire.
+	 */
+	TimestampTz	stall_causal_reads_until;
+
 	WalSnd		walsnds[FLEXIBLE_ARRAY_MEMBER];
 } WalSndCtlData;
 
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index fc4b765..44f826f 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1785,10 +1785,11 @@ pg_stat_replication| SELECT s.pid,
     w.replay_location,
     w.replay_lag,
     w.sync_priority,
-    w.sync_state
+    w.sync_state,
+    w.causal_reads_state
    FROM pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, wait_event_type, wait_event, xact_start, query_start, backend_start, state_change, client_addr, client_hostname, client_port, backend_xid, backend_xmin, ssl, sslversion, sslcipher, sslbits, sslcompression, sslclientdn),
     pg_authid u,
-    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state)
+    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state, causal_reads_state)
   WHERE ((s.usesysid = u.oid) AND (s.pid = w.pid));
 pg_stat_ssl| SELECT s.pid,
     s.ssl,
#60Michael Paquier
michael.paquier@gmail.com
In reply to: Thomas Munro (#59)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Fri, Mar 25, 2016 at 4:51 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Thu, Mar 24, 2016 at 12:34 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Wed, Mar 23, 2016 at 12:37 PM, Robert Haas <robertmhaas@gmail.com> wrote:

+static void WalRcvUnblockSigUsr2(void)

And again here.

Fixed.

+                WalRcvUnblockSigUsr2();
len = walrcv_receive(NAPTIME_PER_CYCLE, &buf);
+                WalRcvBlockSigUsr2();

This does not seem like it will be cheap on all operating systems. I
think you should try to rejigger this somehow so that it can just set
the process latch and the wal receiver figures it out from looking at
shared memory. Like maybe a flag in WalRcvData? An advantage of this
is that it should cut down on the number of signals significantly,
because it won't need to send SIGUSR1 when the latch is already set.

Still experimenting with a latch here. I will come back on this point soon.

Here is a latch-based version.

Thanks for the updated version. This looks pretty nice.

I find the routine name libpqrcv_wait to be a bit confusing. This is
not a routine aimed at being exposed externally as walrcv_send or
walrcv_receive. I would recommend changing the name, to something like
waitForWALStream or similar.

Should we worried about potential backward-incompatibilities with the
new return values of walrcv_receive?

Do you have numbers to share regarding how is performing the
latch-based approach and the approach that used SIGUSR2 when
remote_apply is used?
--
Michael

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#61Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Michael Paquier (#60)
5 attachment(s)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Sat, Mar 26, 2016 at 2:48 AM, Michael Paquier
<michael.paquier@gmail.com> wrote:

On Fri, Mar 25, 2016 at 4:51 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Thu, Mar 24, 2016 at 12:34 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Wed, Mar 23, 2016 at 12:37 PM, Robert Haas <robertmhaas@gmail.com> wrote:

+static void WalRcvUnblockSigUsr2(void)

And again here.

Fixed.

+                WalRcvUnblockSigUsr2();
len = walrcv_receive(NAPTIME_PER_CYCLE, &buf);
+                WalRcvBlockSigUsr2();

This does not seem like it will be cheap on all operating systems. I
think you should try to rejigger this somehow so that it can just set
the process latch and the wal receiver figures it out from looking at
shared memory. Like maybe a flag in WalRcvData? An advantage of this
is that it should cut down on the number of signals significantly,
because it won't need to send SIGUSR1 when the latch is already set.

Still experimenting with a latch here. I will come back on this point soon.

Here is a latch-based version.

Thanks for the updated version. This looks pretty nice.

I find the routine name libpqrcv_wait to be a bit confusing. This is
not a routine aimed at being exposed externally as walrcv_send or
walrcv_receive. I would recommend changing the name, to something like
waitForWALStream or similar.

Done (as "wait_for_wal_stream", following the case convention of nearby stuff).

Should we worried about potential backward-incompatibilities with the
new return values of walrcv_receive?

There are three changes to the walrcv_receive interface:

1. It now takes a latch pointer, which may be NULL.

2. If the latch pointer is non-NULL, the existing function might
return a new sentinel value WALRCV_RECEIVE_LATCH_SET. (The
pre-existing sentinel value -1 is still in use and has the same value
and meaning as before, but it now has a name:
WALRCV_RECEIVE_COPY_ENDED.)

3. It will no longer return when the process is signalled (a latch
should be used to ask it to return instead).

Effectively, any client code would need to change at least to add NULL
or possibly a latch if it needs to ask it to return, and any
alternative implementation of the WAL receiver interface would need to
use WaitEventSet (or WaitLatchOrSocket) as its event loop instead of
whatever it might be using now so that it can detect a latch's state.
But in any case, any such code would fail to compile against 9.6 due
to the new argument, and then you'd only be able to get the new return
value if you asked for it by passing in a latch. What affected code
are we aware of -- either users of libpqwalreceiver.so or other WAL
receiver implementations?

Do you have numbers to share regarding how is performing the
latch-based approach and the approach that used SIGUSR2 when
remote_apply is used?

I couldn't measure any significant change (Linux, all on localhost, 128 cores):

pgbench -c 1 -N bench2 -T 600

0001-remote-apply-v5.patch (signals), remote_apply -> 449 TPS
0001-remote-apply-v6.patch (latches), remote_apply -> 452 TPS

pgbench -c 64 -j 32 -N bench2 -T 600

0001-remote-apply-v5.patch (signals), remote_apply -> 8536 TPS
0001-remote-apply-v6.patch (latches), remote_apply -> 8534 TPS

Incidentally, I also did some testing on what happens when you signal
a process that is busily writing and fsyncing. I tested a few
different kernels, write sizes and disk latencies and saw that things
were fine on all of them up to 10k signals/sec but after that some
systems' fsync performance started to reduce. Only Linux on Power was
still happily fsyncing at around 3/4 of full speed when signalled with
a 2MHz+ tight kill loop (!), while FreeBSD and Linux on Intel weren't
able to make much progress at all under such adversity. So I suppose
that if you could get the commit rate up into 5 figures you might be
able to measure an improvement for the latch version due to
latch-collapsing, though I noticed a large amount of signal-collapsing
going on at the OS level on all systems I tested anyway, so maybe it
wouldn't make a difference. I attach that test program for interest.

Also, I updated the comment for the declaration of the latch in
walreceiver.h to say something about the new usage.

New patch series attached.

--
Thomas Munro
http://www.enterprisedb.com

Attachments:

0001-remote-apply-v7.patchapplication/octet-stream; name=0001-remote-apply-v7.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index d48a13f..6ab65a7 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -2143,8 +2143,8 @@ include_dir 'conf.d'
         Specifies whether transaction commit will wait for WAL records
         to be written to disk before the command returns a <quote>success</>
         indication to the client.  Valid values are <literal>on</>,
-        <literal>remote_write</>, <literal>local</>, and <literal>off</>.
-        The default, and safe, setting
+        <literal>remote_write</>, <literal>remote_apply</>, <literal>local</>,
+        and <literal>off</>.  The default, and safe, setting
         is <literal>on</>.  When <literal>off</>, there can be a delay between
         when success is reported to the client and when the transaction is
         really guaranteed to be safe against a server crash.  (The maximum
@@ -2177,6 +2177,10 @@ include_dir 'conf.d'
         ensure data preservation even if the standby instance of
         <productname>PostgreSQL</> were to crash, but not if the standby
         suffers an operating-system-level crash.
+        When set to <literal>remote_apply</>, commits will wait until a reply
+        from the current synchronous standby indicates it has received the
+        commit record of the transaction and applied it, so that it has become
+        visible to queries.
        </para>
        <para>
         When synchronous
diff --git a/doc/src/sgml/high-availability.sgml b/doc/src/sgml/high-availability.sgml
index 19d613e..03c6c30 100644
--- a/doc/src/sgml/high-availability.sgml
+++ b/doc/src/sgml/high-availability.sgml
@@ -1081,6 +1081,9 @@ primary_slot_name = 'node_a_slot'
     WAL record is then sent to the standby. The standby sends reply
     messages each time a new batch of WAL data is written to disk, unless
     <varname>wal_receiver_status_interval</> is set to zero on the standby.
+    In the case that <varname>synchronous_commit</> is set to
+    <literal>remote_apply</>, the standby sends reply messages when the commit
+    record is replayed, making the transaction visible.
     If the standby is the first matching standby, as specified in
     <varname>synchronous_standby_names</> on the primary, the reply
     messages from that standby will be used to wake users waiting for
@@ -1107,6 +1110,14 @@ primary_slot_name = 'node_a_slot'
    </para>
 
    <para>
+    Setting <varname>synchronous_commit</> to <literal>remote_apply</> will
+    cause each commit to wait until the current synchronous standby reports
+    that it has replayed the transaction, making it visible to user queries.
+    In simple cases, this allows for load balancing with causal consistency
+    on a single hot standby.
+   </para>
+
+   <para>
     Users will stop waiting if a fast shutdown is requested.  However, as
     when using asynchronous replication, the server will not fully
     shutdown until all outstanding WAL records are transferred to the currently
@@ -1160,9 +1171,10 @@ primary_slot_name = 'node_a_slot'
     <title>Planning for High Availability</title>
 
    <para>
-    Commits made when <varname>synchronous_commit</> is set to <literal>on</>
-    or <literal>remote_write</> will wait until the synchronous standby responds. The response
-    may never occur if the last, or only, standby should crash.
+    Commits made when <varname>synchronous_commit</> is set to <literal>on</>,
+    <literal>remote_write</> or <literal>remote_apply</> will wait until the
+    synchronous standby responds. The response may never occur if the last, or
+    only, standby should crash.
    </para>
 
    <para>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index e7234c8..893c2fa 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -1107,7 +1107,7 @@ EndPrepare(GlobalTransaction gxact)
 	 * Note that at this stage we have marked the prepare, but still show as
 	 * running in the procarray (twice!) and continue to hold locks.
 	 */
-	SyncRepWaitForLSN(gxact->prepare_end_lsn);
+	SyncRepWaitForLSN(gxact->prepare_end_lsn, false);
 
 	records.tail = records.head = NULL;
 	records.num_chunks = 0;
@@ -2103,7 +2103,7 @@ RecordTransactionCommitPrepared(TransactionId xid,
 	 * Note that at this stage we have marked clog, but still show as running
 	 * in the procarray and continue to hold locks.
 	 */
-	SyncRepWaitForLSN(recptr);
+	SyncRepWaitForLSN(recptr, true);
 }
 
 /*
@@ -2156,5 +2156,5 @@ RecordTransactionAbortPrepared(TransactionId xid,
 	 * Note that at this stage we have marked clog, but still show as running
 	 * in the procarray and continue to hold locks.
 	 */
-	SyncRepWaitForLSN(recptr);
+	SyncRepWaitForLSN(recptr, true);
 }
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 89a14b4..130b56b 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,7 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
-		SyncRepWaitForLSN(XactLastRecEnd);
+		SyncRepWaitForLSN(XactLastRecEnd, true);
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5123,6 +5123,13 @@ XactLogCommitRecord(TimestampTz commit_time,
 		xl_xinfo.xinfo |= XACT_COMPLETION_FORCE_SYNC_COMMIT;
 
 	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this commit is applied.
+	 */
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
+	/*
 	 * Relcache invalidations requires information about the current database
 	 * and so does logical decoding.
 	 */
@@ -5300,6 +5307,13 @@ XactLogAbortRecord(TimestampTz abort_time,
 	if (xl_xinfo.xinfo & XACT_XINFO_HAS_TWOPHASE)
 		XLogRegisterData((char *) (&xl_twophase), sizeof(xl_xact_twophase));
 
+	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this abort is applied.
+	 */
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
 	return XLogInsert(RM_XACT_ID, info);
 }
 
@@ -5458,6 +5472,13 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 	if (XactCompletionForceSyncCommit(parsed->xinfo))
 		XLogFlush(lsn);
 
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit = remote_apply), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 /*
@@ -5544,6 +5565,14 @@ xact_redo_abort(xl_xact_parsed_abort *parsed, TransactionId xid)
 		smgrdounlink(srel, true);
 		smgrclose(srel);
 	}
+
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit = remote_apply), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 void
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index b119a47..3e454f5 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -345,6 +345,9 @@ static XLogRecPtr RedoRecPtr;
  */
 static bool doPageWrites;
 
+/* Has the recovery code requested a walreceiver wakeup? */
+static bool doRequestWalReceiverReply;
+
 /*
  * RedoStartLSN points to the checkpoint's REDO location which is specified
  * in a backup label file, backup history file or control file. In standby
@@ -6879,6 +6882,19 @@ StartupXLOG(void)
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
 				SpinLockRelease(&XLogCtl->info_lck);
 
+				/*
+				 * If rm_redo reported that it applied a commit record that
+				 * the master is waiting for by calling
+				 * XLogRequestWalReceiverReply, then we wake up the receiver
+				 * so that it notices the updated lastReplayedEndRecPtr and
+				 * sends a reply to the master.
+				 */
+				if (doRequestWalReceiverReply)
+				{
+					doRequestWalReceiverReply = false;
+					WalRcvWakeup();
+				}
+
 				/* Remember this record as the last-applied one */
 				LastRec = ReadRecPtr;
 
@@ -11594,3 +11610,12 @@ SetWalWriterSleeping(bool sleeping)
 	XLogCtl->WalWriterSleeping = sleeping;
 	SpinLockRelease(&XLogCtl->info_lck);
 }
+
+/*
+ * Schedule a walreceiver wakeup in the main recovery loop.
+ */
+void
+XLogRequestWalReceiverReply(void)
+{
+	doRequestWalReceiverReply = true;
+}
diff --git a/src/backend/replication/README b/src/backend/replication/README
index 8e5bf0d..8ec8ef2 100644
--- a/src/backend/replication/README
+++ b/src/backend/replication/README
@@ -16,14 +16,15 @@ bool walrcv_connect(char *conninfo, XLogRecPtr startpoint)
 Establish connection to the primary, and starts streaming from 'startpoint'.
 Returns true on success.
 
-bool walrcv_receive(int timeout, unsigned char *type, char **buffer, int *len)
+int walrcv_receive(int timeout, char **buffer, Latch *latch)
 
 Retrieve any message available through the connection, blocking for
-maximum of 'timeout' ms. If a message was successfully read, returns true,
-otherwise false. On success, a pointer to the message payload is stored in
-*buffer, length in *len, and the type of message received in *type. The
-returned buffer is valid until the next call to walrcv_* functions, the
-caller should not attempt freeing it.
+maximum of 'timeout' ms. If a message was successfully read, returns
+its length. Otherwise returns 0 for timeout, WALRCV_RECEIVE_COPY_ENDED
+for disconnection or WALRCV_RECEIVE_LATCH_SET. On success, a pointer
+to the message payload is stored in *buffer. The returned buffer is
+valid until the next call to walrcv_* functions, and the caller should
+not attempt to free it.
 
 void walrcv_send(const char *buffer, int nbytes)
 
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index 4ee4d71..78cee9f 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -25,16 +25,6 @@
 #include "replication/walreceiver.h"
 #include "utils/builtins.h"
 
-#ifdef HAVE_POLL_H
-#include <poll.h>
-#endif
-#ifdef HAVE_SYS_POLL_H
-#include <sys/poll.h>
-#endif
-#ifdef HAVE_SYS_SELECT_H
-#include <sys/select.h>
-#endif
-
 PG_MODULE_MAGIC;
 
 void		_PG_init(void);
@@ -52,12 +42,12 @@ static void libpqrcv_readtimelinehistoryfile(TimeLineID tli, char **filename, ch
 static bool libpqrcv_startstreaming(TimeLineID tli, XLogRecPtr startpoint,
 						char *slotname);
 static void libpqrcv_endstreaming(TimeLineID *next_tli);
-static int	libpqrcv_receive(int timeout, char **buffer);
+static int	libpqrcv_receive(int timeout, char **buffer, Latch *latch);
 static void libpqrcv_send(const char *buffer, int nbytes);
 static void libpqrcv_disconnect(void);
 
 /* Prototypes for private functions */
-static bool libpq_select(int timeout_ms);
+static int wait_for_wal_stream(int timeout_ms, Latch *latch);
 static PGresult *libpqrcv_PQexec(const char *query);
 
 /*
@@ -315,17 +305,16 @@ libpqrcv_readtimelinehistoryfile(TimeLineID tli,
 }
 
 /*
- * Wait until we can read WAL stream, or timeout.
- *
- * Returns true if data has become available for reading, false if timed out
- * or interrupted by signal.
+ * Wait until we can read WAL stream, a timeout is reached or (optionally) a
+ * latch is set.
  *
- * This is based on pqSocketCheck.
+ * Returns WL_SOCKET_READABLE, WL_TIMEOUT or WL_LATCH_SET.
  */
-static bool
-libpq_select(int timeout_ms)
+static int
+wait_for_wal_stream(int timeout_ms, Latch *latch)
 {
 	int			ret;
+	int			events;
 
 	Assert(streamConn != NULL);
 	if (PQsocket(streamConn) < 0)
@@ -333,51 +322,39 @@ libpq_select(int timeout_ms)
 				(errcode_for_socket_access(),
 				 errmsg("invalid socket: %s", PQerrorMessage(streamConn))));
 
-	/* We use poll(2) if available, otherwise select(2) */
-	{
-#ifdef HAVE_POLL
-		struct pollfd input_fd;
-
-		input_fd.fd = PQsocket(streamConn);
-		input_fd.events = POLLIN | POLLERR;
-		input_fd.revents = 0;
-
-		ret = poll(&input_fd, 1, timeout_ms);
-#else							/* !HAVE_POLL */
-
-		fd_set		input_mask;
-		struct timeval timeout;
-		struct timeval *ptr_timeout;
-
-		FD_ZERO(&input_mask);
-		FD_SET(PQsocket(streamConn), &input_mask);
-
-		if (timeout_ms < 0)
-			ptr_timeout = NULL;
-		else
-		{
-			timeout.tv_sec = timeout_ms / 1000;
-			timeout.tv_usec = (timeout_ms % 1000) * 1000;
-			ptr_timeout = &timeout;
-		}
+	events = WL_SOCKET_READABLE | WL_POSTMASTER_DEATH;
+	if (timeout_ms > 0)
+		events |= WL_TIMEOUT;
+	if (latch != NULL)
+		events |= WL_LATCH_SET;
 
-		ret = select(PQsocket(streamConn) + 1, &input_mask,
-					 NULL, NULL, ptr_timeout);
-#endif   /* HAVE_POLL */
+	/*
+	 * Ideally we would reuse a WaitEventSet object repeatedly here to avoid
+	 * the overheads of WaitLatchOrSocket on epoll systems, but we can't be
+	 * sure that libpq has the same socket (even if the fd is the same number,
+	 * it may have been closed and reopened since the last call to
+	 * wait_for_wal_stream).  In future, if there is a function for removing
+	 * sockets from WaitEventSet, then we could add and remove just the socket
+	 * each time, potentially avoiding some system calls.
+	 */
+	ret = WaitLatchOrSocket(latch, events, PQsocket(streamConn), timeout_ms);
+
+	if (ret & WL_POSTMASTER_DEATH)
+		exit(0);
+	else if (ret & WL_LATCH_SET)
+		return WL_LATCH_SET;
+	else if (ret & WL_SOCKET_READABLE)
+		return WL_SOCKET_READABLE;
+	else
+	{
+		Assert(ret & WL_TIMEOUT);
+		return WL_TIMEOUT;
 	}
-
-	if (ret == 0 || (ret < 0 && errno == EINTR))
-		return false;
-	if (ret < 0)
-		ereport(ERROR,
-				(errcode_for_socket_access(),
-				 errmsg("select() failed: %m")));
-	return true;
 }
 
 /*
  * Send a query and wait for the results by using the asynchronous libpq
- * functions and the backend version of select().
+ * functions and WaitLatchOrSocket.
  *
  * We must not use the regular blocking libpq functions like PQexec()
  * since they are uninterruptible by signals on some platforms, such as
@@ -424,8 +401,7 @@ libpqrcv_PQexec(const char *query)
 			 * elog(FATAL) within SIGTERM signal handler if the signal arrives
 			 * in the middle of establishment of replication connection.
 			 */
-			if (!libpq_select(-1))
-				continue;		/* interrupted */
+			wait_for_wal_stream(-1, NULL); /* WL_SOCKET_READABLE */
 			if (PQconsumeInput(streamConn) == 0)
 				return NULL;	/* trouble */
 		}
@@ -463,8 +439,9 @@ libpqrcv_disconnect(void)
 }
 
 /*
- * Receive a message available from XLOG stream, blocking for
- * maximum of 'timeout' ms.
+ * Receive a message available from XLOG stream, blocking for maximum of
+ * 'timeout' ms.  If latch is not NULL, return early if the latch it points to
+ * is set.
  *
  * Returns:
  *
@@ -472,15 +449,16 @@ libpqrcv_disconnect(void)
  *	 point to a buffer holding the received message. The buffer is only valid
  *	 until the next libpqrcv_* call.
  *
- *	 0 if no data was available within timeout, or wait was interrupted
- *	 by signal.
+ *	 0 if no data was available within timeout.
  *
- *	 -1 if the server ended the COPY.
+ *	 WALRCV_RECEIVE_COPY_ENDED if the server ended the COPY.
+ *
+ *	 WALRCV_RECEIVE_LATCH_SET if the latch was set.
  *
  * ereports on error.
  */
 static int
-libpqrcv_receive(int timeout, char **buffer)
+libpqrcv_receive(int timeout, char **buffer, Latch *latch)
 {
 	int			rawlen;
 
@@ -498,8 +476,14 @@ libpqrcv_receive(int timeout, char **buffer)
 		 */
 		if (timeout > 0)
 		{
-			if (!libpq_select(timeout))
+			int ready = wait_for_wal_stream(timeout, latch);
+
+			if (ready == WL_TIMEOUT)
 				return 0;
+			else if (ready == WL_LATCH_SET)
+				return WALRCV_RECEIVE_LATCH_SET;
+
+			Assert(ready == WL_SOCKET_READABLE);
 		}
 
 		if (PQconsumeInput(streamConn) == 0)
@@ -521,7 +505,7 @@ libpqrcv_receive(int timeout, char **buffer)
 			PQresultStatus(res) == PGRES_COPY_IN)
 		{
 			PQclear(res);
-			return -1;
+			return WALRCV_RECEIVE_COPY_ENDED;
 		}
 		else
 		{
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 92faf4e..1ee1bc5 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -91,13 +91,25 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * to the wait queue. During SyncRepWakeQueue() a WALSender changes
  * the state to SYNC_REP_WAIT_COMPLETE once replication is confirmed.
  * This backend then resets its state to SYNC_REP_NOT_WAITING.
+ *
+ * 'lsn' represents the LSN to wait for.  'commit' indicates whether this LSN
+ * represents a commit/abort record.  If it's not, then we wait only for the
+ * WAL to be flushed if synchronous_commit is set to the higher level of
+ * remote_apply, because standbys only send apply feedback for commit/abort
+ * records.
  */
 void
-SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
+SyncRepWaitForLSN(XLogRecPtr lsn, bool commit)
 {
 	char	   *new_status = NULL;
 	const char *old_status;
-	int			mode = SyncRepWaitMode;
+	int			mode;
+
+	/* Cap the level for non-commit records to remote flush only. */
+	if (commit)
+		mode = SyncRepWaitMode;
+	else
+		mode = Max(SyncRepWaitMode, SYNC_REP_WAIT_FLUSH);
 
 	/*
 	 * Fast exit if user has not requested sync replication, or there are no
@@ -122,7 +134,7 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 	 * to be a low cost check.
 	 */
 	if (!WalSndCtl->sync_standbys_defined ||
-		XactCommitLSN <= WalSndCtl->lsn[mode])
+		lsn <= WalSndCtl->lsn[mode])
 	{
 		LWLockRelease(SyncRepLock);
 		return;
@@ -132,7 +144,7 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 	 * Set our waitLSN so WALSender will know when to wake us, and add
 	 * ourselves to the queue.
 	 */
-	MyProc->waitLSN = XactCommitLSN;
+	MyProc->waitLSN = lsn;
 	MyProc->syncRepState = SYNC_REP_WAITING;
 	SyncRepQueueInsert(mode);
 	Assert(SyncRepQueueIsOrderedByLSN(mode));
@@ -147,7 +159,7 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 		new_status = (char *) palloc(len + 32 + 1);
 		memcpy(new_status, old_status, len);
 		sprintf(new_status + len, " waiting for %X/%X",
-				(uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN);
+				(uint32) (lsn >> 32), (uint32) lsn);
 		set_ps_display(new_status, false);
 		new_status[len] = '\0'; /* truncate off " waiting ..." */
 	}
@@ -416,6 +428,7 @@ SyncRepReleaseWaiters(void)
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
+	int			numapply = 0;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
@@ -462,12 +475,18 @@ SyncRepReleaseWaiters(void)
 		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
 		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
 	}
+	if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+	{
+		walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+		numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY);
+	}
 
 	LWLockRelease(SyncRepLock);
 
-	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X",
+	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%x",
 		 numwrite, (uint32) (MyWalSnd->write >> 32), (uint32) MyWalSnd->write,
-	   numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush);
+		 numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush,
+		 numapply, (uint32) (MyWalSnd->apply >> 32), (uint32) MyWalSnd->apply);
 
 	/*
 	 * If we are managing the highest priority standby, though we weren't
@@ -728,6 +747,9 @@ assign_synchronous_commit(int newval, void *extra)
 		case SYNCHRONOUS_COMMIT_REMOTE_FLUSH:
 			SyncRepWaitMode = SYNC_REP_WAIT_FLUSH;
 			break;
+		case SYNCHRONOUS_COMMIT_REMOTE_APPLY:
+			SyncRepWaitMode = SYNC_REP_WAIT_APPLY;
+			break;
 		default:
 			SyncRepWaitMode = SYNC_REP_NO_WAIT;
 			break;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 7b36e02..2fa996d 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -153,7 +153,6 @@ static void WalRcvSigUsr1Handler(SIGNAL_ARGS);
 static void WalRcvShutdownHandler(SIGNAL_ARGS);
 static void WalRcvQuickDieHandler(SIGNAL_ARGS);
 
-
 static void
 ProcessWalRcvInterrupts(void)
 {
@@ -200,6 +199,7 @@ WalReceiverMain(void)
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz last_recv_timestamp;
 	bool		ping_sent;
+	bool		apply_feedback_requested = false;
 
 	/*
 	 * WalRcv should be set up already (if we are a backend, we inherit this
@@ -407,8 +407,8 @@ WalReceiverMain(void)
 					XLogWalRcvSendHSFeedback(true);
 				}
 
-				/* Wait a while for data to arrive */
-				len = walrcv_receive(NAPTIME_PER_CYCLE, &buf);
+				/* Wait a while for data to arrive or our latch to be set */
+				len = walrcv_receive(NAPTIME_PER_CYCLE, &buf, &walrcv->latch);
 				if (len != 0)
 				{
 					/*
@@ -429,7 +429,7 @@ WalReceiverMain(void)
 						}
 						else if (len == 0)
 							break;
-						else if (len < 0)
+						else if (len == WALRCV_RECEIVE_COPY_ENDED)
 						{
 							ereport(LOG,
 									(errmsg("replication terminated by primary server"),
@@ -439,11 +439,18 @@ WalReceiverMain(void)
 							endofwal = true;
 							break;
 						}
-						len = walrcv_receive(0, &buf);
+						else if (len == WALRCV_RECEIVE_LATCH_SET)
+						{
+							/* The recovery process has new apply feedback to report. */
+							apply_feedback_requested = true;
+							ResetLatch(&walrcv->latch);
+						}
+						len = walrcv_receive(0, &buf, &walrcv->latch);
 					}
 
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(false, false);
+					XLogWalRcvSendReply(apply_feedback_requested, false);
+					apply_feedback_requested = false;
 
 					/*
 					 * If we've written some records, flush them to disk and
@@ -1222,6 +1229,21 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 }
 
 /*
+ * Wake up the walreceiver if it happens to be blocked in walrcv_receive,
+ * and tell it that a commit record has been applied.
+ *
+ * This is called by the startup process whenever interesting xlog records
+ * are applied, so that walreceiver can check if it needs to send an apply
+ * notification back to the master which may be waiting in a COMMIT with
+ * synchronous_commit = remote_apply.
+ */
+void
+WalRcvWakeup(void)
+{
+	SetLatch(&WalRcv->latch);
+}
+
+/*
  * Return a string constant representing the state. This is used
  * in system functions and views, and should *not* be translated.
  */
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 65a6cd4..06cb166 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -345,12 +345,13 @@ static const struct config_enum_entry constraint_exclusion_options[] = {
 };
 
 /*
- * Although only "on", "off", "remote_write", and "local" are documented, we
- * accept all the likely variants of "on" and "off".
+ * Although only "on", "off", "remote_apply", "remote_write", and "local" are
+ * documented, we accept all the likely variants of "on" and "off".
  */
 static const struct config_enum_entry synchronous_commit_options[] = {
 	{"local", SYNCHRONOUS_COMMIT_LOCAL_FLUSH, false},
 	{"remote_write", SYNCHRONOUS_COMMIT_REMOTE_WRITE, false},
+	{"remote_apply", SYNCHRONOUS_COMMIT_REMOTE_APPLY, false},
 	{"on", SYNCHRONOUS_COMMIT_ON, false},
 	{"off", SYNCHRONOUS_COMMIT_OFF, false},
 	{"true", SYNCHRONOUS_COMMIT_ON, true},
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index 5536012..ec4427f 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -177,7 +177,7 @@
 					# (change requires restart)
 #fsync = on				# turns forced synchronization on or off
 #synchronous_commit = on		# synchronization level;
-					# off, local, remote_write, or on
+					# off, local, remote_write, remote_apply, or on
 #wal_sync_method = fsync		# the default is the first option
 					# supported by the operating system:
 					#   open_datasync
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index ebeb582..6b706db 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -60,7 +60,9 @@ typedef enum
 	SYNCHRONOUS_COMMIT_LOCAL_FLUSH,		/* wait for local flush only */
 	SYNCHRONOUS_COMMIT_REMOTE_WRITE,	/* wait for local flush and remote
 										 * write */
-	SYNCHRONOUS_COMMIT_REMOTE_FLUSH		/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_FLUSH,	/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_APPLY		/* wait for local flush and remote
+										 * apply */
 }	SyncCommitLevel;
 
 /* Define the default setting for synchonous_commit */
@@ -144,10 +146,13 @@ typedef void (*SubXactCallback) (SubXactEvent event, SubTransactionId mySubid,
  * EOXact... routines which run at the end of the original transaction
  * completion.
  */
+#define XACT_COMPLETION_SYNC_APPLY_FEEDBACK		(1U << 29)
 #define XACT_COMPLETION_UPDATE_RELCACHE_FILE	(1U << 30)
 #define XACT_COMPLETION_FORCE_SYNC_COMMIT		(1U << 31)
 
 /* Access macros for above flags */
+#define XactCompletionSyncApplyFeedback(xinfo) \
+	((xinfo & XACT_COMPLETION_SYNC_APPLY_FEEDBACK) != 0)
 #define XactCompletionRelcacheInitFileInval(xinfo) \
 	(!!(xinfo & XACT_COMPLETION_UPDATE_RELCACHE_FILE))
 #define XactCompletionForceSyncCommit(xinfo) \
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 74a1394..a7dcdae 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -267,6 +267,8 @@ extern bool CheckPromoteSignal(void);
 extern void WakeupRecovery(void);
 extern void SetWalWriterSleeping(bool sleeping);
 
+extern void XLogRequestWalReceiverReply(void);
+
 extern void assign_max_wal_size(int newval, void *extra);
 extern void assign_checkpoint_completion_target(double newval, void *extra);
 
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index 96e059b..c005a42 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -23,8 +23,9 @@
 #define SYNC_REP_NO_WAIT		-1
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
+#define SYNC_REP_WAIT_APPLY		2
 
-#define NUM_SYNC_REP_WAIT_MODE	2
+#define NUM_SYNC_REP_WAIT_MODE	3
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
@@ -35,7 +36,7 @@
 extern char *SyncRepStandbyNames;
 
 /* called by user backend */
-extern void SyncRepWaitForLSN(XLogRecPtr XactCommitLSN);
+extern void SyncRepWaitForLSN(XLogRecPtr lsn, bool commit);
 
 /* called at backend exit */
 extern void SyncRepCleanupAtProcExit(void);
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 6eacb09..3ffa9e3 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -115,7 +115,8 @@ typedef struct
 	/*
 	 * Latch used by startup process to wake up walreceiver after telling it
 	 * where to start streaming (after setting receiveStart and
-	 * receiveStartTLI).
+	 * receiveStartTLI), and then to tell it to send apply feedback to the
+	 * primary whenever specially marked commit records are applied.
 	 */
 	Latch		latch;
 } WalRcvData;
@@ -138,7 +139,7 @@ extern PGDLLIMPORT walrcv_startstreaming_type walrcv_startstreaming;
 typedef void (*walrcv_endstreaming_type) (TimeLineID *next_tli);
 extern PGDLLIMPORT walrcv_endstreaming_type walrcv_endstreaming;
 
-typedef int (*walrcv_receive_type) (int timeout, char **buffer);
+typedef int (*walrcv_receive_type) (int timeout, char **buffer, Latch *latch);
 extern PGDLLIMPORT walrcv_receive_type walrcv_receive;
 
 typedef void (*walrcv_send_type) (const char *buffer, int nbytes);
@@ -147,6 +148,10 @@ extern PGDLLIMPORT walrcv_send_type walrcv_send;
 typedef void (*walrcv_disconnect_type) (void);
 extern PGDLLIMPORT walrcv_disconnect_type walrcv_disconnect;
 
+/* special values returned by walrcv_receive */
+#define WALRCV_RECEIVE_COPY_ENDED -1
+#define WALRCV_RECEIVE_LATCH_SET -2
+
 /* prototypes for functions in walreceiver.c */
 extern void WalReceiverMain(void) pg_attribute_noreturn();
 extern Datum pg_stat_get_wal_receiver(PG_FUNCTION_ARGS);
@@ -162,5 +167,6 @@ extern void RequestXLogStreaming(TimeLineID tli, XLogRecPtr recptr,
 extern XLogRecPtr GetWalRcvWriteRecPtr(XLogRecPtr *latestChunkStart, TimeLineID *receiveTLI);
 extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
+extern void WalRcvWakeup(void);
 
 #endif   /* _WALRECEIVER_H */
0002-replay-lag-v7.patchapplication/octet-stream; name=0002-replay-lag-v7.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 105d541..7d63782 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1208,6 +1208,12 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
       standby server</entry>
     </row>
     <row>
+     <entry><structfield>replay_lag</></entry>
+     <entry><type>interval</></entry>
+     <entry>Estimated time taken for recent WAL records to be replayed on this
+      standby server</entry>
+    </row>
+    <row>
      <entry><structfield>sync_priority</></entry>
      <entry><type>integer</></entry>
      <entry>Priority of this standby server for being chosen as the
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 130b56b..48a5950 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -5473,6 +5473,12 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 		XLogFlush(lsn);
 
 	/*
+	 * Record the primary's timestamp for the commit record, so it can be used
+	 * for tracking replay lag.
+	 */
+	SetXLogReplayTimestamp(parsed->xact_time);
+
+	/*
 	 * If asked by the primary (because someone is waiting for a synchronous
 	 * commit = remote_apply), we will need to ask walreceiver to send a
 	 * reply immediately.
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 3e454f5..504b4df 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -81,6 +81,8 @@ extern uint32 bootstrap_data_checksum_version;
 #define PROMOTE_SIGNAL_FILE		"promote"
 #define FALLBACK_PROMOTE_SIGNAL_FILE "fallback_promote"
 
+/* Size of the circular buffer of timestamped LSNs. */
+#define MAX_TIMESTAMPED_LSNS 8192
 
 /* User-settable parameters */
 int			max_wal_size = 64;	/* 1 GB */
@@ -360,6 +362,13 @@ static bool doRequestWalReceiverReply;
  */
 static XLogRecPtr RedoStartLSN = InvalidXLogRecPtr;
 
+/*
+ * LastReplayedTimestamp can be set by redo handlers when they apply a record
+ * that carries a timestamp, by calling SetXLogReplayedTimestamp.  The xlog
+ * apply loop can then update the value in shared memory.
+ */
+static TimestampTz LastReplayedTimestamp = 0;
+
 /*----------
  * Shared-memory data structures for XLOG control
  *
@@ -634,6 +643,21 @@ typedef struct XLogCtlData
 	/* current effective recovery target timeline */
 	TimeLineID	RecoveryTargetTLI;
 
+	/* timestamp from the most recently applied record associated with a timestamp. */
+	TimestampTz lastReplayedTimestamp;
+
+	/*
+	 * We maintain a circular buffer of LSNs and associated timestamps.
+	 * Walreceiver writes into it using information from timestamps, and the
+	 * startup recovery process reads from it and notifies walreceiver when
+	 * LSNs are replayed so that the timestamps can eventually be fed back to
+	 * the upstream server, to track lag.
+	 */
+	Index			timestampedLsnRead;
+	Index			timestampedLsnWrite;
+	XLogRecPtr		timestampedLsn[MAX_TIMESTAMPED_LSNS];
+	TimestampTz		timestampedLsnTime[MAX_TIMESTAMPED_LSNS];
+
 	/*
 	 * timestamp of when we started replaying the current chunk of WAL data,
 	 * only relevant for replication or archive recovery
@@ -6874,20 +6898,51 @@ StartupXLOG(void)
 				error_context_stack = errcallback.previous;
 
 				/*
-				 * Update lastReplayedEndRecPtr after this record has been
-				 * successfully replayed.
+				 * Update lastReplayedEndRecPtr and lastReplayedTimestamp
+				 * after this record has been successfully replayed.
 				 */
 				SpinLockAcquire(&XLogCtl->info_lck);
 				XLogCtl->lastReplayedEndRecPtr = EndRecPtr;
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
+				if (LastReplayedTimestamp != 0)
+				{
+					/* If replaying a record produced a timestamp, use that. */
+					XLogCtl->lastReplayedTimestamp = LastReplayedTimestamp;
+					LastReplayedTimestamp = 0;
+				}
+				else
+				{
+					/*
+					 * If we have applied LSNs associated with timestamps
+					 * received by walreceiver, then use the recorded
+					 * timestamp.  We consume from the read end of the
+					 * circular buffer.
+					 */
+					while (XLogCtl->timestampedLsnRead !=
+						   XLogCtl->timestampedLsnWrite &&
+						   XLogCtl->timestampedLsn[XLogCtl->timestampedLsnRead]
+						   <= EndRecPtr)
+					{
+						if (XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead] >
+							XLogCtl->lastReplayedTimestamp)
+						{
+							XLogCtl->lastReplayedTimestamp =
+								XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead];
+							doRequestWalReceiverReply = true;
+						}
+						XLogCtl->timestampedLsnRead =
+							(XLogCtl->timestampedLsnRead + 1) % MAX_TIMESTAMPED_LSNS;
+					}
+				}
 				SpinLockRelease(&XLogCtl->info_lck);
 
 				/*
 				 * If rm_redo reported that it applied a commit record that
 				 * the master is waiting for by calling
-				 * XLogRequestWalReceiverReply, then we wake up the receiver
-				 * so that it notices the updated lastReplayedEndRecPtr and
-				 * sends a reply to the master.
+				 * XLogRequestWalReceiverReply, or we encountered a WAL
+				 * location that was associated with a timestamp above, then
+				 * we wake up the receiver so that it notices the updated
+				 * lastReplayedEndRecPtr and sends a reply to the master.
 				 */
 				if (doRequestWalReceiverReply)
 				{
@@ -11619,3 +11674,91 @@ XLogRequestWalReceiverReply(void)
 {
 	doRequestWalReceiverReply = true;
 }
+
+/*
+ * Record the timestamp that is associated with a WAL position.
+ *
+ * This is called by walreceiver on standby servers when keepalive messages
+ * arrive, using timestamps generated on the primary server.  The timestamp
+ * will be sent back to the primary server when the standby had applied this
+ * WAL position.  The primary can use the elapsed time to estimate the replay
+ * lag.
+ */
+void
+SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn)
+{
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn == XLogCtl->lastReplayedEndRecPtr)
+	{
+		/*
+		 * That is the last replayed LSN: we are fully replayed, so we can
+		 * update the replay timestamp immediately.
+		 */
+		XLogCtl->lastReplayedTimestamp = timestamp;
+	}
+	else
+	{
+		/*
+		 * There is WAL still to be applied.  We will associate the timestamp
+		 * with this WAL position and wait for it to be replayed.  We add it
+		 * at the 'write' end of the circular buffer of LSN/timestamp
+		 * mappings, which the replay loop will eventually read.
+		 */
+		Index w = XLogCtl->timestampedLsnWrite;
+		Index r = XLogCtl->timestampedLsnRead;
+
+		XLogCtl->timestampedLsn[w] = lsn;
+		XLogCtl->timestampedLsnTime[w] = timestamp;
+
+		/* Advance the write point. */
+		w = (w + 1) % MAX_TIMESTAMPED_LSNS;
+		XLogCtl->timestampedLsnWrite = w;
+		if (w == r)
+		{
+			/*
+			 * The buffer is full.  Advance the read point (throwing away
+			 * oldest values; we will begin to overestimate replay lag, until
+			 * lag decreases to a size our buffer can manage, or the next
+			 * commit record is replayed).
+			 */
+			r = (r + 1) % MAX_TIMESTAMPED_LSNS;
+			XLogCtl->timestampedLsnRead = r;
+		}
+	}
+	SpinLockRelease(&XLogCtl->info_lck);
+}
+
+/*
+ * Set the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary.  This can be called by redo handlers that have
+ * an appropriate timestamp (currently only commit records).  Updating the
+ * shared memory value is deferred until after the redo handler returns.
+ */
+void
+SetXLogReplayTimestamp(TimestampTz timestamp)
+{
+	LastReplayedTimestamp = timestamp;
+}
+
+/*
+ * Get the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary, and also the most recently applied LSN.  (Note
+ * that the timestamp and the LSN don't necessarily relate to the same
+ * record.)
+ *
+ * This is similar to GetLatestXTime, except that it is not only advanced by
+ * commit records (see SetXLogReplayTimestampAtLsn).
+ */
+TimestampTz
+GetXLogReplayTimestamp(XLogRecPtr *lsn)
+{
+	TimestampTz result;
+
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn)
+		*lsn = XLogCtl->lastReplayedEndRecPtr;
+	result = XLogCtl->lastReplayedTimestamp;
+	SpinLockRelease(&XLogCtl->info_lck);
+
+	return result;
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 9ae1ef4..a53f07b 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -662,6 +662,7 @@ CREATE VIEW pg_stat_replication AS
             W.write_location,
             W.flush_location,
             W.replay_location,
+            W.replay_lag,
             W.sync_priority,
             W.sync_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 2fa996d..faea9ff 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -85,6 +85,8 @@ walrcv_disconnect_type walrcv_disconnect = NULL;
 
 #define NAPTIME_PER_CYCLE 100	/* max sleep time between cycles (100ms) */
 
+#define MIN_TIME_BETWEEN_TIMESTAMPED_LSNS 1000 /* 1s */
+
 /*
  * These variables are used similarly to openLogFile/SegNo/Off,
  * but for walreceiver to write the XLOG. recvFileTLI is the TimeLineID
@@ -102,6 +104,8 @@ static uint32 recvOff = 0;
 static volatile sig_atomic_t got_SIGHUP = false;
 static volatile sig_atomic_t got_SIGTERM = false;
 
+static bool recovery_active = false;
+
 /*
  * LogstreamResult indicates the byte positions that we have already
  * written/fsynced.
@@ -143,7 +147,7 @@ static void WalRcvDie(int code, Datum arg);
 static void XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len);
 static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
-static void XLogWalRcvSendReply(bool force, bool requestReply);
+static void XLogWalRcvSendReply(bool force, bool requestReply, bool includeApplyTimestamp);
 static void XLogWalRcvSendHSFeedback(bool immed);
 static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
 
@@ -443,13 +447,15 @@ WalReceiverMain(void)
 						{
 							/* The recovery process has new apply feedback to report. */
 							apply_feedback_requested = true;
+							recovery_active = true;
 							ResetLatch(&walrcv->latch);
 						}
 						len = walrcv_receive(0, &buf, &walrcv->latch);
 					}
 
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(apply_feedback_requested, false);
+					XLogWalRcvSendReply(apply_feedback_requested, false,
+										apply_feedback_requested);
 					apply_feedback_requested = false;
 
 					/*
@@ -505,7 +511,7 @@ WalReceiverMain(void)
 						}
 					}
 
-					XLogWalRcvSendReply(requestReply, requestReply);
+					XLogWalRcvSendReply(requestReply, requestReply, false);
 					XLogWalRcvSendHSFeedback(false);
 				}
 			}
@@ -836,6 +842,8 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 			}
 		case 'k':				/* Keepalive */
 			{
+				bool reportApplyTimestamp = false;
+
 				/* copy message to StringInfo */
 				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
 				if (len != hdrlen)
@@ -852,9 +860,22 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 
 				ProcessWalSndrMessage(walEnd, sendTime);
 
+				/*
+				 * If no apply timestamps have been sent at the request of the
+				 * recovery process since we last received a keepalive, then
+				 * we will send one now.  This allows us to feed back
+				 * timestamps in response to pings if we are idle or if the
+				 * recovery process is somehow blocked, but we don't want to
+				 * do that if it's actively applying and periodically waking
+				 * us up with accurate apply timestamps.
+				 */
+				if (!recovery_active)
+					reportApplyTimestamp = true;
+				recovery_active = false;
+
 				/* If the primary requested a reply, send one immediately */
-				if (replyRequested)
-					XLogWalRcvSendReply(true, false);
+				if (replyRequested || reportApplyTimestamp)
+					XLogWalRcvSendReply(true, false, reportApplyTimestamp);
 				break;
 			}
 		default:
@@ -1017,7 +1038,7 @@ XLogWalRcvFlush(bool dying)
 		/* Also let the master know that we made some progress */
 		if (!dying)
 		{
-			XLogWalRcvSendReply(false, false);
+			XLogWalRcvSendReply(false, false, false);
 			XLogWalRcvSendHSFeedback(false);
 		}
 	}
@@ -1035,15 +1056,18 @@ XLogWalRcvFlush(bool dying)
  * If 'requestReply' is true, requests the server to reply immediately upon
  * receiving this message. This is used for heartbearts, when approaching
  * wal_receiver_timeout.
+ *
+ * If 'reportApplyTimestamp' is true, the latest apply timestamp is included.
  */
 static void
-XLogWalRcvSendReply(bool force, bool requestReply)
+XLogWalRcvSendReply(bool force, bool requestReply, bool reportApplyTimestamp)
 {
 	static XLogRecPtr writePtr = 0;
 	static XLogRecPtr flushPtr = 0;
 	XLogRecPtr	applyPtr;
 	static TimestampTz sendTime = 0;
 	TimestampTz now;
+	TimestampTz applyTimestamp = 0;
 
 	/*
 	 * If the user doesn't want status to be reported to the master, be sure
@@ -1059,10 +1083,8 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	 * We can compare the write and flush positions to the last message we
 	 * sent without taking any lock, but the apply position requires a spin
 	 * lock, so we don't check that unless something else has changed or 10
-	 * seconds have passed.  This means that the apply log position will
-	 * appear, from the master's point of view, to lag slightly, but since
-	 * this is only for reporting purposes and only on idle systems, that's
-	 * probably OK.
+	 * seconds have passed, or the force flag has been set (which happens when
+	 * apply feedback has been requested by the primary).
 	 */
 	if (!force
 		&& writePtr == LogstreamResult.Write
@@ -1075,7 +1097,10 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	/* Construct a new message */
 	writePtr = LogstreamResult.Write;
 	flushPtr = LogstreamResult.Flush;
-	applyPtr = GetXLogReplayRecPtr(NULL);
+	if (reportApplyTimestamp)
+		applyTimestamp = GetXLogReplayTimestamp(&applyPtr);
+	else
+		applyPtr = GetXLogReplayRecPtr(NULL);
 
 	resetStringInfo(&reply_message);
 	pq_sendbyte(&reply_message, 'r');
@@ -1083,6 +1108,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	pq_sendint64(&reply_message, flushPtr);
 	pq_sendint64(&reply_message, applyPtr);
 	pq_sendint64(&reply_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&reply_message, TimestampTzToIntegerTimestamp(applyTimestamp));
 	pq_sendbyte(&reply_message, requestReply ? 1 : 0);
 
 	/* Send it */
@@ -1187,8 +1213,8 @@ static void
 ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 {
 	WalRcvData *walrcv = WalRcv;
-
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
+	static TimestampTz lastRecordedTimestamp = 0;
 
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
@@ -1199,6 +1225,18 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
 	SpinLockRelease(&walrcv->mutex);
 
+	/*
+	 * Remember primary's timestamp at this WAL location.  We throw away
+	 * samples if they are coming too fast because we don't want to fill up
+	 * the finite circular buffer and have to throw away older samples.
+	 */
+	if (lastRecordedTimestamp < TimestampTzPlusMilliseconds(sendTime,
+															-MIN_TIME_BETWEEN_TIMESTAMPED_LSNS))
+	{
+		SetXLogReplayTimestampAtLsn(sendTime, walEnd);
+		lastRecordedTimestamp = sendTime;
+	}
+
 	if (log_min_messages <= DEBUG2)
 	{
 		char	   *sendtime;
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index f98475c..16d7abc 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1545,15 +1545,29 @@ ProcessStandbyReplyMessage(void)
 	XLogRecPtr	writePtr,
 				flushPtr,
 				applyPtr;
+	int64		applyLagUs;
 	bool		replyRequested;
+	TimestampTz now = GetCurrentTimestamp();
+	TimestampTz applyTimestamp;
 
 	/* the caller already consumed the msgtype byte */
 	writePtr = pq_getmsgint64(&reply_message);
 	flushPtr = pq_getmsgint64(&reply_message);
 	applyPtr = pq_getmsgint64(&reply_message);
 	(void) pq_getmsgint64(&reply_message);		/* sendTime; not used ATM */
+	applyTimestamp = IntegerTimestampToTimestampTz(pq_getmsgint64(&reply_message));
 	replyRequested = pq_getmsgbyte(&reply_message);
 
+	/* Compute the apply lag in milliseconds. */
+	if (applyTimestamp == 0)
+		applyLagUs = -1;
+	else
+#ifdef HAVE_INT64_TIMESTAMP
+		applyLagUs = now - applyTimestamp;
+#else
+		applyLagUs = (now - applyTimestamp) * 1000000;
+#endif
+
 	elog(DEBUG2, "write %X/%X flush %X/%X apply %X/%X%s",
 		 (uint32) (writePtr >> 32), (uint32) writePtr,
 		 (uint32) (flushPtr >> 32), (uint32) flushPtr,
@@ -1575,6 +1589,8 @@ ProcessStandbyReplyMessage(void)
 		walsnd->write = writePtr;
 		walsnd->flush = flushPtr;
 		walsnd->apply = applyPtr;
+		if (applyLagUs >= 0)
+			walsnd->applyLagUs = applyLagUs;
 		SpinLockRelease(&walsnd->mutex);
 	}
 
@@ -2745,7 +2761,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	8
+#define PG_STAT_GET_WAL_SENDERS_COLS	9
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2793,6 +2809,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		XLogRecPtr	write;
 		XLogRecPtr	flush;
 		XLogRecPtr	apply;
+		int64		applyLagUs;
 		int			priority;
 		WalSndState state;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
@@ -2807,6 +2824,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
+		applyLagUs = walsnd->applyLagUs;
 		priority = walsnd->sync_standby_priority;
 		SpinLockRelease(&walsnd->mutex);
 
@@ -2841,6 +2859,23 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				nulls[5] = true;
 			values[5] = LSNGetDatum(apply);
 
+			if (applyLagUs < 0)
+				nulls[6] = true;
+			else
+			{
+				Interval *applyLagInterval = palloc(sizeof(Interval));
+
+				applyLagInterval->month = 0;
+				applyLagInterval->day = 0;
+#ifdef HAVE_INT64_TIMESTAMP
+				applyLagInterval->time = applyLagUs;
+#else
+				applyLagInterval->time = applyLagUs / 1000000.0;
+#endif
+				nulls[6] = false;
+				values[6] = IntervalPGetDatum(applyLagInterval);
+			}
+
 			/*
 			 * Treat a standby such as a pg_basebackup background process
 			 * which always returns an invalid flush location, as an
@@ -2848,18 +2883,18 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 			 */
 			priority = XLogRecPtrIsInvalid(walsnd->flush) ? 0 : priority;
 
-			values[6] = Int32GetDatum(priority);
+			values[7] = Int32GetDatum(priority);
 
 			/*
 			 * More easily understood version of standby state. This is purely
 			 * informational, not different from priority.
 			 */
 			if (priority == 0)
-				values[7] = CStringGetTextDatum("async");
+				values[8] = CStringGetTextDatum("async");
 			else if (walsnd == sync_standby)
-				values[7] = CStringGetTextDatum("sync");
+				values[8] = CStringGetTextDatum("sync");
 			else
-				values[7] = CStringGetTextDatum("potential");
+				values[8] = CStringGetTextDatum("potential");
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
diff --git a/src/backend/utils/adt/timestamp.c b/src/backend/utils/adt/timestamp.c
index c9e5270..f382b20 100644
--- a/src/backend/utils/adt/timestamp.c
+++ b/src/backend/utils/adt/timestamp.c
@@ -1629,6 +1629,20 @@ IntegerTimestampToTimestampTz(int64 timestamp)
 #endif
 
 /*
+ * TimestampTzToIntegerTimestamp -- convert a native timestamp to int64 format
+ *
+ * When compiled with --enable-integer-datetimes, this is implemented as a
+ * no-op macro.
+ */
+#ifndef HAVE_INT64_TIMESTAMP
+int64
+TimestampTzToIntegerTimestamp(TimestampTz timestamp)
+{
+	return timestamp * 1000000;
+}
+#endif
+
+/*
  * TimestampDifference -- convert the difference between two timestamps
  *		into integer seconds and microseconds
  *
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index a7dcdae..c8be3ce 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -235,6 +235,9 @@ extern void GetXLogReceiptTime(TimestampTz *rtime, bool *fromStream);
 extern XLogRecPtr GetXLogReplayRecPtr(TimeLineID *replayTLI);
 extern XLogRecPtr GetXLogInsertRecPtr(void);
 extern XLogRecPtr GetXLogWriteRecPtr(void);
+extern void SetXLogReplayTimestamp(TimestampTz timestamp);
+extern void SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn);
+extern TimestampTz GetXLogReplayTimestamp(XLogRecPtr *lsn);
 extern bool RecoveryIsPaused(void);
 extern void SetRecoveryPause(bool recoveryPause);
 extern TimestampTz GetLatestXTime(void);
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index a595327..4054726 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2712,7 +2712,7 @@ DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f
 DESCR("statistics: information about currently active backends");
 DATA(insert OID = 3318 (  pg_stat_get_progress_info           PGNSP PGUID 12 1 100 0 0 f f f f t t s r 1 0 2249 "25" "{25,23,26,26,20,20,20,20,20,20,20,20,20,20}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{cmdtype,pid,datid,relid,param1,param2,param3,param4,param5,param6,param7,param8,param9,param10}" _null_ _null_ pg_stat_get_progress_info _null_ _null_ _null_ ));
 DESCR("statistics: information about progress of backends running maintenance command");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,23,25}" "{o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25}" "{o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 7794aa5..4de43e8 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -46,6 +46,7 @@ typedef struct WalSnd
 	XLogRecPtr	write;
 	XLogRecPtr	flush;
 	XLogRecPtr	apply;
+	int64		applyLagUs;
 
 	/* Protects shared variables shown above. */
 	slock_t		mutex;
diff --git a/src/include/utils/timestamp.h b/src/include/utils/timestamp.h
index fbead3a..297e151 100644
--- a/src/include/utils/timestamp.h
+++ b/src/include/utils/timestamp.h
@@ -227,9 +227,11 @@ extern bool TimestampDifferenceExceeds(TimestampTz start_time,
 #ifndef HAVE_INT64_TIMESTAMP
 extern int64 GetCurrentIntegerTimestamp(void);
 extern TimestampTz IntegerTimestampToTimestampTz(int64 timestamp);
+extern int64 TimestampTzToIntegerTimestamp(TimestampTz timestamp);
 #else
 #define GetCurrentIntegerTimestamp()	GetCurrentTimestamp()
 #define IntegerTimestampToTimestampTz(timestamp) (timestamp)
+#define TimestampTzToIntegerTimestamp(timestamp) (timestamp)
 #endif
 
 extern TimestampTz time_t_to_timestamptz(pg_time_t tm);
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 79f9b23..fc4b765 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1783,11 +1783,12 @@ pg_stat_replication| SELECT s.pid,
     w.write_location,
     w.flush_location,
     w.replay_location,
+    w.replay_lag,
     w.sync_priority,
     w.sync_state
    FROM pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, wait_event_type, wait_event, xact_start, query_start, backend_start, state_change, client_addr, client_hostname, client_port, backend_xid, backend_xmin, ssl, sslversion, sslcipher, sslbits, sslcompression, sslclientdn),
     pg_authid u,
-    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, sync_priority, sync_state)
+    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state)
   WHERE ((s.usesysid = u.oid) AND (s.pid = w.pid));
 pg_stat_ssl| SELECT s.pid,
     s.ssl,
0003-refactor-syncrep-exit-v7.patchapplication/octet-stream; name=0003-refactor-syncrep-exit-v7.patchDownload
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 1ee1bc5..376ddf4 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -83,6 +83,64 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * ===========================================================
  */
 
+static bool
+SyncRepCheckEarlyExit(void)
+{
+	/*
+	 * If a wait for synchronous replication is pending, we can neither
+	 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
+	 * lead the client to believe that the transaction aborted, which
+	 * is not true: it's already committed locally. The former is no good
+	 * either: the client has requested synchronous replication, and is
+	 * entitled to assume that an acknowledged commit is also replicated,
+	 * which might not be true. So in this case we issue a WARNING (which
+	 * some clients may be able to interpret) and shut off further output.
+	 * We do NOT reset ProcDiePending, so that the process will die after
+	 * the commit is cleaned up.
+	 */
+	if (ProcDiePending)
+	{
+		ereport(WARNING,
+				(errcode(ERRCODE_ADMIN_SHUTDOWN),
+				 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * It's unclear what to do if a query cancel interrupt arrives.  We
+	 * can't actually abort at this point, but ignoring the interrupt
+	 * altogether is not helpful, so we just terminate the wait with a
+	 * suitable warning.
+	 */
+	if (QueryCancelPending)
+	{
+		QueryCancelPending = false;
+		ereport(WARNING,
+				(errmsg("canceling wait for synchronous replication due to user request"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * If the postmaster dies, we'll probably never get an
+	 * acknowledgement, because all the wal sender processes will exit. So
+	 * just bail out.
+	 */
+	if (!PostmasterIsAlive())
+	{
+		ProcDiePending = true;
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	return false;
+}
+
 /*
  * Wait for synchronous replication, if requested by user.
  *
@@ -192,57 +250,9 @@ SyncRepWaitForLSN(XLogRecPtr lsn, bool commit)
 		if (syncRepState == SYNC_REP_WAIT_COMPLETE)
 			break;
 
-		/*
-		 * If a wait for synchronous replication is pending, we can neither
-		 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
-		 * lead the client to believe that the transaction aborted, which
-		 * is not true: it's already committed locally. The former is no good
-		 * either: the client has requested synchronous replication, and is
-		 * entitled to assume that an acknowledged commit is also replicated,
-		 * which might not be true. So in this case we issue a WARNING (which
-		 * some clients may be able to interpret) and shut off further output.
-		 * We do NOT reset ProcDiePending, so that the process will die after
-		 * the commit is cleaned up.
-		 */
-		if (ProcDiePending)
-		{
-			ereport(WARNING,
-					(errcode(ERRCODE_ADMIN_SHUTDOWN),
-					 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit())
 			break;
-		}
-
-		/*
-		 * It's unclear what to do if a query cancel interrupt arrives.  We
-		 * can't actually abort at this point, but ignoring the interrupt
-		 * altogether is not helpful, so we just terminate the wait with a
-		 * suitable warning.
-		 */
-		if (QueryCancelPending)
-		{
-			QueryCancelPending = false;
-			ereport(WARNING,
-					(errmsg("canceling wait for synchronous replication due to user request"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * If the postmaster dies, we'll probably never get an
-		 * acknowledgement, because all the wal sender processes will exit. So
-		 * just bail out.
-		 */
-		if (!PostmasterIsAlive())
-		{
-			ProcDiePending = true;
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
-			break;
-		}
 
 		/*
 		 * Wait on latch.  Any condition that should wake us up will set the
0004-causal-reads-v7.patchapplication/octet-stream; name=0004-causal-reads-v7.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 6ab65a7..ed6f07c 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -2749,6 +2749,35 @@ include_dir 'conf.d'
      across the cluster without problems if that is required.
     </para>
 
+    <sect2 id="runtime-config-replication-all">
+     <title>All Servers</title>
+     <para>
+      These parameters can be set on the primary or any standby.
+     </para>
+     <variablelist>
+      <varlistentry id="guc-causal-reads" xreflabel="causal_reads">
+       <term><varname>causal_reads</varname> (<type>boolean</type>)
+       <indexterm>
+        <primary><varname>causal_reads</> configuration parameter</primary>
+       </indexterm>
+       </term>
+       <listitem>
+        <para>
+         Enables causal consistency between transactions run on different
+         servers.  A transaction that is run on a standby
+         with <varname>causal_reads</> set to <literal>on</> is guaranteed
+         either to see the effects of all completed transactions run on the
+         primary with the setting on, or to receive an error "standby is not
+         available for causal reads".  Note that both transactions involved in
+         a causal dependency (a write on the primary followed by a read on any
+         server which must see the write) must be run with the setting on.
+         See <xref linkend="causal-reads"> for more details.
+        </para>
+       </listitem>
+      </varlistentry>
+     </variablelist>     
+    </sect2>
+
     <sect2 id="runtime-config-replication-sender">
      <title>Sending Server(s)</title>
 
@@ -2980,6 +3009,48 @@ include_dir 'conf.d'
       </listitem>
      </varlistentry>
 
+     <varlistentry>
+      <term><varname>causal_reads_timeout</varname> (<type>integer</type>)
+       <indexterm>
+        <primary><varname>causal_reads_timeout</> configuration parameter</primary>
+       </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies the maximum replay lag the primary will tolerate from a
+        standby before dropping it from the set of standbys available for
+        causal reads.
+       </para>
+       <para>
+        This setting is also used to control the <firstterm>leases</> used to
+        maintain the causal reads guarantee.  It must be set to a value which
+        is at least 4 times the maximum possible difference in system clocks
+        between the primary and standby servers, as described
+        in <xref linkend="causal-reads">.
+       </para>
+      </listitem>
+     </varlistentry>
+
+     <varlistentry id="guc-causal-reads-standby-names" xreflabel="causal-reads-standby-names">
+      <term><varname>causal_reads_standby_names</varname> (<type>string</type>)
+      <indexterm>
+       <primary><varname>causal_reads_standby_names</> configuration parameter</primary>
+      </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies a comma-separated list of standby names that can support
+        <firstterm>causal reads</>, as described in
+        <xref linkend="causal-reads">.  Follows the same convention
+        as <link linkend="guc-synchronous-standby-names"><literal>synchronous_standby_name</></>.
+        The default is <literal>*</>, matching all standbys.
+       </para>
+       <para>
+        This setting has no effect if <varname>causal_reads_timeout</> is not set.
+       </para>
+      </listitem>
+     </varlistentry>
+
      </variablelist>
     </sect2>
 
diff --git a/doc/src/sgml/high-availability.sgml b/doc/src/sgml/high-availability.sgml
index 03c6c30..7a0910d 100644
--- a/doc/src/sgml/high-availability.sgml
+++ b/doc/src/sgml/high-availability.sgml
@@ -1115,6 +1115,9 @@ primary_slot_name = 'node_a_slot'
     that it has replayed the transaction, making it visible to user queries.
     In simple cases, this allows for load balancing with causal consistency
     on a single hot standby.
+    (See also
+    <xref linkend="causal-reads"> which deals with multiple standbys and
+    standby failure.)
    </para>
 
    <para>
@@ -1233,6 +1236,119 @@ primary_slot_name = 'node_a_slot'
    </sect3>
   </sect2>
 
+  <sect2 id="causal-reads">
+   <title>Causal reads</title>
+   <indexterm>
+    <primary>causal reads</primary>
+    <secondary>in standby</secondary>
+   </indexterm>
+
+   <para>
+    The causal reads feature allows read-only queries to run on hot standby
+    servers without exposing stale data to the client, providing a form of
+    causal consistency.  Transactions can run on any standby with the
+    following guarantee about the visibility of preceding transactions: If you
+    set <varname>causal_reads</> to <literal>on</> in any pair of consecutive
+    transactions tx1, tx2 where tx2 begins after tx1 successfully returns,
+    then tx2 will either see tx1 or fail with a new error "standby is not
+    available for causal reads", no matter which server it runs on.  Although
+    the guarantee is expressed in terms of two individual transactions, the
+    GUC can also be set at session, role or system level to make the guarantee
+    generally, allowing for load balancing of applications that were not
+    designed with load balancing in mind.
+   </para>
+
+   <para>
+    In order to enable the feature, <varname>causal_reads_timeout</> must be
+    set to a non-zero value on the primary server.  The
+    GUC <varname>causal_reads_standby_names</> can be used to limit the set of
+    standbys that can join the dynamic set of causal reads standbys by
+    providing a comma-separated list of application names.  By default, all
+    standbys are candidates, if the feature is enabled.
+   </para>
+
+   <para>
+    The current set of servers that the primary considers to be available for
+    causal reads can be seen in
+    the <link linkend="monitoring-stats-views-table"> <literal>pg_stat_replication</></>
+    view.  Administrators, applications and load balancing middleware can use
+    this view to discover standbys that can currently handle causal reads
+    transactions without raising the error.  Since that information is only an
+    instantantaneous snapshot, clients should still be prepared for the error
+    to be raised at any time, and consider redirecting transactions to another
+    standby.
+   </para>
+
+   <para>
+    The advantages of the causal reads feature over simply
+    setting <varname>synchronous_commit</> to <literal>remote_apply</> are:
+    <orderedlist>
+      <listitem>
+       <para>
+        It allows the primary to wait for multiple standbys to replay
+        transactions.
+       </para>
+      </listitem>
+      <listitem>
+       <para>
+        It places a configurable limit on how much replay lag (and therefore
+        delay at commit time) the primary tolerates from standbys before it
+        drops them from the dynamic set of standbys it waits for.
+       </para>   
+      </listitem>
+      <listitem>
+       <para>
+        It upholds the causal reads guarantee during the transitions that
+        occur when new standbys are added or removed from the set of standbys,
+        including scenarios where contact has been lost between the primary
+        and standbys but the standby is still alive and running client
+        queries.
+       </para>
+      </listitem>
+    </orderedlist>
+   </para>
+
+   <para>
+    The protocol used to uphold the guarantee even in the case of network
+    failure depends on the system clocks of the primary and standby servers
+    being synchronized, with an allowance for a difference up to one quarter
+    of <varname>causal_reads_timeout</>.  For example,
+    if <varname>causal_reads_timeout</> is set to <literal>4s</>, then the
+    clocks must not be further than 1 second apart for the guarantee to be
+    upheld reliably during transitions.  The ubiquity of the Network Time
+    Protocol (NTP) on modern operating systems and availability of high
+    quality time servers makes it possible to choose a tolerance significantly
+    higher than the maximum expected clock difference.  An effort is
+    nevertheless made to detect and report misconfigured and faulty systems
+    with clock differences greater than the configured tolerance.
+   </para>
+
+   <note>
+    <para>
+     Current hardware clocks, NTP implementations and public time servers are
+     unlikely to allow the system clocks to differ more than tens or hundreds
+     of milliseconds, and systems synchronized with dedicated local time
+     servers may be considerably more accurate, but you should only consider
+     setting <varname>causal_reads_timeout</> below 4 seconds (allowing up to
+     1 second of clock difference) after researching your time synchronization
+     infrastructure thoroughly.
+    </para>  
+   </note>
+
+   <note>
+    <para>
+      While similar to synchronous replication in the sense that both involve
+      the primary server waiting for responses from standby servers, the
+      causal reads feature is not concerned with avoiding data loss.  A
+      primary configured for causal reads will drop all standbys that stop
+      responding or replay too slowly from the dynamic set that it waits for,
+      so you should consider configuring both synchronous replication and
+      causal reads if you need data loss avoidance guarantees and causal
+      consistency guarantees for load balancing.
+    </para>
+   </note>
+  </sect2>
+
   <sect2 id="continuous-archiving-in-standby">
    <title>Continuous archiving in standby</title>
 
@@ -1581,7 +1697,16 @@ if (!triggered)
     so there will be a measurable delay between primary and standby. Running the
     same query nearly simultaneously on both primary and standby might therefore
     return differing results. We say that data on the standby is
-    <firstterm>eventually consistent</firstterm> with the primary.  Once the
+    <firstterm>eventually consistent</firstterm> with the primary by default.
+    The data visible to a transaction running on a standby can be
+    made <firstterm>causally consistent</> with respect to a transaction that
+    has completed on the primary by setting <varname>causal_reads</>
+    to <literal>on</> in both transactions.  For more details,
+    see <xref linkend="causal-reads">.
+   </para>
+
+   <para>
+    Once the    
     commit record for a transaction is replayed on the standby, the changes
     made by that transaction will be visible to any new snapshots taken on
     the standby.  Snapshots may be taken at the start of each query or at the
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 7d63782..23d68d5 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1224,6 +1224,17 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
      <entry><type>text</></entry>
      <entry>Synchronous state of this standby server</entry>
     </row>
+    <row>
+     <entry><structfield>causal_reads_state</></entry>
+     <entry><type>text</></entry>
+     <entry>Causal reads state of this standby server.  This field will be
+     non-null only if <varname>cause_reads_timeout</> is set.  If a standby is
+     in <literal>available</> state, then it can currently serve causal reads
+     queries.  If it is not replaying fast enough or not responding to
+     keepalive messages, it will be in <literal>unavailable</> state, and if
+     it is currently transitioning to availability it will be
+     in <literal>joining</> state for a short time.</entry>
+    </row>
    </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 893c2fa..111198a 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -2098,11 +2098,12 @@ RecordTransactionCommitPrepared(TransactionId xid,
 	END_CRIT_SECTION();
 
 	/*
-	 * Wait for synchronous replication, if required.
+	 * Wait for causal reads and synchronous replication, if required.
 	 *
 	 * Note that at this stage we have marked clog, but still show as running
 	 * in the procarray and continue to hold locks.
 	 */
+	CausalReadsWaitForLSN(recptr);
 	SyncRepWaitForLSN(recptr, true);
 }
 
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 48a5950..4530a6e 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,10 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
+	{
+		CausalReadsWaitForLSN(XactLastRecEnd);
 		SyncRepWaitForLSN(XactLastRecEnd, true);
+	}
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5126,7 +5129,7 @@ XactLogCommitRecord(TimestampTz commit_time,
 	 * Check if the caller would like to ask standbys for immediate feedback
 	 * once this commit is applied.
 	 */
-	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY || causal_reads)
 		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
 
 	/*
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index a53f07b..276ac12 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -664,7 +664,8 @@ CREATE VIEW pg_stat_replication AS
             W.replay_location,
             W.replay_lag,
             W.sync_priority,
-            W.sync_state
+            W.sync_state,
+            W.causal_reads_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
             pg_stat_get_wal_senders() AS W
     WHERE S.usesysid = U.oid AND
diff --git a/src/backend/replication/README.causal_reads b/src/backend/replication/README.causal_reads
new file mode 100644
index 0000000..1fddd62
--- /dev/null
+++ b/src/backend/replication/README.causal_reads
@@ -0,0 +1,193 @@
+The causal reads guarantee says: If you run any two consecutive
+transactions tx1, tx2 where tx1 completes before tx2 begins, with
+causal_reads set to "on" in both transactions, tx2 will see tx1 or
+raise an error to complain that it can't guarantee causal consistency,
+no matter which servers (primary or any standby) you run each
+transaction on.
+
+When both transactions run on the primary, the guarantee is trivially
+upheld.
+
+To deal with read-only physical streaming standbys, the primary keeps
+track of a set of standbys that it considers to be currently
+"available" for causal reads, and sends a stream of "leases" to those
+standbys granting them the right to handle causal reads transactions
+for a short time without any further communication with the primary.
+
+In general, the primary provides the guarantee by waiting for all of
+the "available" standbys to report that they have applied a
+transaction.  However, the set of available standbys is dynamic, and
+things get more complicated during state transitions.  There are two
+types of transitions to consider:
+
+1.  unavailable->joining->available
+
+Standbys start out as "unavailable".  If a standby is unavailable and
+is applying fast enough and matches causal_reads_standby_names, the
+primary transitions it to "available", but first it sets it to
+"joining" until it is sure that any transaction committed while it was
+unavailable has definitely been applied on the standby.  This closes a
+race that would otherwise exist if we moved directly to available
+state: tx1 might not wait for a given standby because it's
+unavailable, then a lease might be granted, and then tx2 might run a
+causal reads transaction without error but see stale data.  The
+joining state acts as an airlock: while in joining state, the primary
+waits for that standby to replay causal reads transactions in
+anticipation of the move to available, but it doesn't progress to
+available state and grant a lease to the standby until everything
+preceding joining state has also been applied.
+
+2.  available->unavailable
+
+If a standby is not applying fast enough or not responding to
+keepalive messages, then the primary kicks that standby out of the
+dynamic set of available standbys, that is, marks it as "unavailable".
+In order to make sure that the standby has started rejecting causal
+reads transactions, it needs to revoke the lease it most recently
+granted.  It does that by waiting for the lease to expire before
+allowing any causal reads commits to return.  (In future there could
+be a fast-path revocation message which waits for a serial-numbered
+acknowledgement to reduce waiting in the case where the standby is
+lagging but still reachable and responding).
+
+The rest of this document illustrates how clock skew affects the
+available->unavailable transition.
+
+The following 4 variables are derived from a single GUC, and these
+values will be used in the following illustrations:
+
+causal_reads_timeout = 4s
+lease_time           = 4s (= causal_reads_timeout)
+keepalive_time       = 2s (= lease_time / 2)
+max_clock_skew       = 1s (= lease_time / 4)
+
+Every keepalive_time, the primary transmits a lease that expires at
+local_clock_time + lease_time - max_clock_skew, shown in the following
+diagram as 't' for transmission time and '|' for expiry time.  If
+contact is lost with a standby, the primary will wait until sent_time
++ lease_time for the most recently granted lease to expire, shown on
+the following diagram 'x', to be sure that the standby's clock has
+reached the expiry time even if its clock differs by up to
+max_clock_skew.  In other words, the primary tells the standby that
+the expiry time is at one time, but it trusts that the standby will
+surely agree if it gives it some extra time.  The extra time is
+max_clock_skew.  If the clocks differ by more than max_clock_skew, all
+bets are off (but see below for attempt to detect obvious cases).
+
+0     1     2     3     4     5     6     7     8     9
+t-----------------|-----x
+            t-----------------|-----x
+                        t-----------------|-----x
+                                    t-----------------|...
+                                                t------...
+
+A standby whose clock is 2 seconds ahead of the primary's clock
+perceives gaps in the stream of leases, and will reject causal_reads
+transactions in those intervals.  The causal reads guarantee is
+upheld, but spurious errors are raised between leases, as a
+consequence of the clock skew being greater than max_clock_skew.  In
+the following diagram 'r' shows reception time, and the timeline along
+the top shows the standby's local clock time.
+
+2     3     4     5     6     7     8     9    10    11
+r-----|
+            r-----|
+                        r-----|
+                                    r-----|
+                                                r-----|
+
+If there were no network latency, a standby whose clock is exactly 1
+second ahead of the primary's clock would perceive the stream of
+leases as being replaced just in time, so there is no gap.  Since in
+reality the time of receipt is some time after the time of
+transmission due to network latency, if the standby's clock is exactly
+1 second behind, then there will be small network-latency-sized gaps
+before the next lease arrives, but still no correctness problem with
+respect to the causal reads guarantee.
+
+1     2     3     4     5     6     7     8     9    10
+r-----------|
+            r-----------|
+                        r-----------|
+                                    r-----------|
+                                                r------...
+
+A standby whose clock is perfectly in sync with the primary's
+perceives the stream of leases overlapping (this matches the primary's
+perception of the leases it sent):
+
+0     1     2     3     4     5     6     7     8     9
+r-----------------|
+            r-----------------|
+                        r-----------------|
+                                    r-----------------|
+                                                r------...
+
+A standby whose clock is exactly 1 second behind the primary's
+perceives the stream of leases as overlapping even more, but the time
+of expiry as judged by the standby is no later than the time the
+primary will wait for if required ('x').  That is, if contact is lost
+with the standby, the primary can still reliably hold up causal reads
+commits until the standby has started raising the error in
+causal_reads transactions.
+
+-1    0     1     2     3     4     5     6     7     8
+r-----------------------|
+            r-----------------------|
+                        r-----------------------|
+                                    r------------------...
+                                                r------...
+
+
+A standby whose clock is 2 seconds behind the primary's would perceive
+the stream of leases overlapping even more, and the primary would no
+longer be able to wait for a lease to expire if it wanted to revoke
+it.  But because the expiry time is after local_clock_time +
+lease_time, the standby can immediately see that its own clock must be
+more than 1 second behind the primary's, so it ignores the lease and
+logs a clock skew warning.  In the following diagram a lease expiry
+time that is obviously generated by a primary with a clock set too far
+in the future compared to the local clock is shown with a '!'.
+
+-2    -1    0     1     2     3     4     5     6     7
+r-----------------------------!
+            r-----------------------------!
+                        r-----------------------------!
+                                    r------------------...
+                                                r------...
+
+A danger window exists when the standby's clock is more than
+max_clock_skew behind the primary's clock, but not more than
+max_clock_skew + network latency time behind.  If the clock difference
+is in that range, then the algorithm presented above which is based on
+time of receipt cannot detect that the local clock is too far behind.
+The consequence of this problem could be as follows:
+
+1.  The standby loses contact with the primary due to a network fault.
+
+2.  The primary decides to drop the standby from the set of available
+    causal reads standbys due to lack of keepalive responses or
+    excessive lag, which necessitates holding up commits of causal
+    reads transactions until the most recently sent lease expires, in
+    the belief that the standby will definitely have started raising
+    the 'causal reads unavailable' error in causal reads transactions
+    by that time, if it is still alive and servicing requests.
+
+3.  The standby still has clients connected and running queries.
+
+4.  Due to clock skew in the problematic range, in the standby's
+    opinion the lease lasts slightly longer than the primary waits.
+
+5.  For a short window at most the duration of the network latency
+    time, clients running causal reads transactions are allowed to see
+    potentially stale data.
+
+For this reason we say that the causal reads guarantee only holds as
+long as the absolute difference between the system clocks of the
+machines is no more than max_clock_skew.  The theory is that NTP makes
+it possible to reason about the maximum possible clock difference
+between machines and choose a value that allows for a much larger
+difference.  However, we do make a best effort attempt to detect
+wildly divergent systems as described above, to catch the case of
+servers not running a correctly configured ntp daemon, or with a clock
+so far out of whack that ntp refuses to fix it.
\ No newline at end of file
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 376ddf4..8240d0d 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -57,6 +57,11 @@
 #include "utils/builtins.h"
 #include "utils/ps_status.h"
 
+/* GUC variables */
+int causal_reads_timeout;
+bool causal_reads;
+char *causal_reads_standby_names;
+
 /* User-settable parameters for sync rep */
 char	   *SyncRepStandbyNames;
 
@@ -69,7 +74,7 @@ static int	SyncRepWaitMode = SYNC_REP_NO_WAIT;
 
 static void SyncRepQueueInsert(int mode);
 static void SyncRepCancelWait(void);
-static int	SyncRepWakeQueue(bool all, int mode);
+static int	SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn);
 
 static int	SyncRepGetStandbyPriority(void);
 
@@ -142,6 +147,198 @@ SyncRepCheckEarlyExit(void)
 }
 
 /*
+ * Check if we can stop waiting for causal consistency.  We can stop waiting
+ * when the following conditions are met:
+ *
+ * 1.  All walsenders currently in 'joining' or 'available' state have
+ * applied the target LSN.
+ *
+ * 2.  Any stall periods caused by standbys dropping out of 'available' state
+ * have passed, so that we can be sure that their leases have expired and they
+ * have started rejecting causal reads transactions.
+ *
+ * The output parameter 'waitingFor' is set to the number of nodes we are
+ * currently waiting for.  The output parameters 'stallTimeMillis' is set to
+ * the number of milliseconds we need to wait for to observe any current
+ * commit stall.
+ *
+ * Returns true if commit can return control, because every standby has either
+ * applied the LSN or started rejecting causal_reads transactions.
+ */
+static bool
+CausalReadsCommitCanReturn(XLogRecPtr XactCommitLSN,
+						   int *waitingFor,
+						   long *stallTimeMillis)
+{
+	int i;
+	TimestampTz now;
+
+	/* Count how many joining/available nodes we are waiting for. */
+	*waitingFor = 0;
+	for (i = 0; i < max_wal_senders; ++i)
+	{
+		WalSnd *walsnd = &WalSndCtl->walsnds[i];
+
+		/*
+		 * Assuming atomic read of pid_t, we can check walsnd->pid without
+		 * acquiring the spinlock to avoid memory synchronization costs for
+		 * unused walsender slots.  We see a value that existed sometime at
+		 * least as recently as the last memory barrier.
+		 */
+		if (walsnd->pid != 0)
+		{
+			/*
+			 * We need to hold the spinlock to read LSNs, because we can't be
+			 * sure they can be read atomically.
+			 */
+			SpinLockAcquire(&walsnd->mutex);
+			if (walsnd->pid != 0 && walsnd->causal_reads_state >= WALSNDCRSTATE_JOINING)
+			{
+				if (walsnd->apply < XactCommitLSN)
+					++*waitingFor;
+			}
+			SpinLockRelease(&walsnd->mutex);
+		}
+	}
+
+	/* Check if there is a stall in progress that we need to observe. */
+	now = GetCurrentTimestamp();
+	LWLockAcquire(SyncRepLock, LW_SHARED);
+	if (WalSndCtl->stall_causal_reads_until > now)
+	{
+		long seconds;
+		int usecs;
+
+		/* Compute how long we have to wait, rounded up to nearest ms. */
+		TimestampDifference(now, WalSndCtl->stall_causal_reads_until,
+							&seconds, &usecs);
+		*stallTimeMillis = seconds * 1000 + (usecs + 999) / 1000;
+	}
+	else
+		*stallTimeMillis = 0;
+	LWLockRelease(SyncRepLock);
+
+	/* We are done if we are not waiting for any nodes or stalls. */
+	return *waitingFor == 0 && *stallTimeMillis == 0;
+}
+
+/*
+ * Wait for causal consistency in causal_reads mode, if requested by user.
+ */
+void
+CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN)
+{
+	long stallTimeMillis;
+	int waitingFor;
+	char *ps_display_buffer = NULL;
+
+	/* Leave if we aren't in causal_reads mode. */
+	if (!causal_reads)
+		return;
+
+	for (;;)
+	{
+		/* Reset latch before checking state. */
+		ResetLatch(MyLatch);
+
+		/*
+		 * Join the queue to be woken up if any causal reads joining/available
+		 * standby applies XactCommitLSN or the set of causal reads standbys
+		 * changes (if we aren't already in the queue).  We don't actually know
+		 * if we need to wait for any peers to reach the target LSN yet, but
+		 * we have to register just in case before checking the walsenders'
+		 * state to avoid a race condition that could occur if we did it after
+		 * calling CausalReadsCommitCanReturn.  (SyncRepWaitForLSN doesn't
+		 * have to do this because it can check the highest-seen LSN in
+		 * walsndctl->lsn[mode] which is protected by SyncRepLock, the same
+		 * lock as the queues.  We can't do that here, because there is no
+		 * single highest-seen LSN that is useful.  We must check
+		 * walsnd->apply for all relevant walsenders.  Therefore we must
+		 * register for notifications first, so that we can be notified via
+		 * our latch of any standby applying the LSN we're interested in after
+		 * we check but before we start waiting, or we could wait forever for
+		 * something that has already happened.)
+		 */
+		LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+		if (MyProc->syncRepState != SYNC_REP_WAITING)
+		{
+			MyProc->waitLSN = XactCommitLSN;
+			MyProc->syncRepState = SYNC_REP_WAITING;
+			SyncRepQueueInsert(SYNC_REP_WAIT_CAUSAL_READS);
+			Assert(SyncRepQueueIsOrderedByLSN(SYNC_REP_WAIT_CAUSAL_READS));
+		}
+		LWLockRelease(SyncRepLock);
+
+		/* Check if we're done. */
+		if (CausalReadsCommitCanReturn(XactCommitLSN, &waitingFor, &stallTimeMillis))
+		{
+			SyncRepCancelWait();
+			break;
+		}
+
+		Assert(waitingFor > 0 || stallTimeMillis > 0);
+
+		/* If we aren't actually waiting for any standbys, leave the queue. */
+		if (waitingFor == 0)
+			SyncRepCancelWait();
+
+		/* Update the ps title. */
+		if (update_process_title)
+		{
+			char buffer[80];
+
+			/* Remember the old value if this is our first update. */
+			if (ps_display_buffer == NULL)
+			{
+				int len;
+				const char *ps_display = get_ps_display(&len);
+
+				ps_display_buffer = palloc(len + 1);
+				memcpy(ps_display_buffer, ps_display, len);
+				ps_display_buffer[len] = '\0';
+			}
+
+			snprintf(buffer, sizeof(buffer),
+					 "waiting for %d peer(s) to apply %X/%X%s",
+					 waitingFor,
+					 (uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN,
+					 stallTimeMillis > 0 ? " (stalling)" : "");
+			set_ps_display(buffer, false);
+		}
+
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit()) /* Calls SyncRepCancelWait() if true. */
+			break;
+
+		/*
+		 * If are still waiting for peers, then we wait for any joining or
+		 * available peer to reach the LSN (or possibly stop being in one of
+		 * those states or go away).
+		 *
+		 * If not, there must be a non-zero stall time, so we wait for that to
+		 * elapse.
+		 */
+		if (waitingFor > 0)
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH, -1);
+		else
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH | WL_TIMEOUT,
+					  stallTimeMillis);
+	}
+
+	/* There is no way out of the loop that could leave us in the queue. */
+	Assert(SHMQueueIsDetached(&(MyProc->syncRepLinks)));
+	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
+	MyProc->waitLSN = 0;
+
+	/* Restore the ps display. */
+	if (ps_display_buffer != NULL)
+	{
+		set_ps_display(ps_display_buffer, false);
+		pfree(ps_display_buffer);
+	}
+}
+
+/*
  * Wait for synchronous replication, if requested by user.
  *
  * Initially backends start in state SYNC_REP_NOT_WAITING and then
@@ -425,6 +622,53 @@ SyncRepGetSynchronousStandby(void)
 }
 
 /*
+ * Check if the current WALSender process's application_name matches a name in
+ * causal_reads_standby_names (including '*' for wildcard).
+ */
+bool
+CausalReadsPotentialStandby(void)
+{
+	char *rawstring;
+	List	   *elemlist;
+	ListCell   *l;
+	bool		found = false;
+
+	/* If the feature is disable, then no. */
+	if (causal_reads_timeout == 0)
+		return false;
+
+	/* Need a modifiable copy of string */
+	rawstring = pstrdup(causal_reads_standby_names);
+
+	/* Parse string into list of identifiers */
+	if (!SplitIdentifierString(rawstring, ',', &elemlist))
+	{
+		/* syntax error in list */
+		pfree(rawstring);
+		list_free(elemlist);
+		/* GUC machinery will have already complained - no need to do again */
+		return 0;
+	}
+
+	foreach(l, elemlist)
+	{
+		char	   *standby_name = (char *) lfirst(l);
+
+		if (pg_strcasecmp(standby_name, application_name) == 0 ||
+			pg_strcasecmp(standby_name, "*") == 0)
+		{
+			found = true;
+			break;
+		}
+	}
+
+	pfree(rawstring);
+	list_free(elemlist);
+
+	return found;
+}
+
+/*
  * Update the LSNs on each queue based upon our latest state. This
  * implements a simple policy of first-valid-standby-releases-waiter.
  *
@@ -432,23 +676,27 @@ SyncRepGetSynchronousStandby(void)
  * perhaps also which information we store as well.
  */
 void
-SyncRepReleaseWaiters(void)
+SyncRepReleaseWaiters(bool walsender_cr_available_or_joining)
 {
 	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
 	int			numapply = 0;
+	int			numcausalreadsapply = 0;
+	bool		is_highest_priority_sync_standby;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
-	 * potential sync standbys then we have nothing to do. If we are still
-	 * starting up, still running base backup or the current flush position
-	 * is still invalid, then leave quickly also.
+	 * potential sync standbys and not in a state that causal_reads waits for,
+	 * then we have nothing to do. If we are still starting up, still running
+	 * base backup or the current flush position is still invalid, then leave
+	 * quickly also.
 	 */
-	if (MyWalSnd->sync_standby_priority == 0 ||
-		MyWalSnd->state < WALSNDSTATE_STREAMING ||
-		XLogRecPtrIsInvalid(MyWalSnd->flush))
+	if (!walsender_cr_available_or_joining &&
+		(MyWalSnd->sync_standby_priority == 0 ||
+		 MyWalSnd->state < WALSNDSTATE_STREAMING ||
+		 XLogRecPtrIsInvalid(MyWalSnd->flush)))
 		return;
 
 	/*
@@ -458,13 +706,19 @@ SyncRepReleaseWaiters(void)
 	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
 	syncWalSnd = SyncRepGetSynchronousStandby();
 
-	/* We should have found ourselves at least */
-	Assert(syncWalSnd != NULL);
+	/*
+	 * If we aren't managing the highest priority standby then make a note of
+	 * that so we can announce a takeover in the log if we ever get that job.
+	 */
+	is_highest_priority_sync_standby = syncWalSnd == MyWalSnd;
+	if (!is_highest_priority_sync_standby)
+		announce_next_takeover = true;
 
 	/*
-	 * If we aren't managing the highest priority standby then just leave.
+	 * If we aren't managing the highest priority standby or a standby in
+	 * causal reads 'joining' or 'available' state, then just leave.
 	 */
-	if (syncWalSnd != MyWalSnd)
+	if (!is_highest_priority_sync_standby && !walsender_cr_available_or_joining)
 	{
 		LWLockRelease(SyncRepLock);
 		announce_next_takeover = true;
@@ -473,24 +727,45 @@ SyncRepReleaseWaiters(void)
 
 	/*
 	 * Set the lsn first so that when we wake backends they will release up to
-	 * this location.
+	 * this location.  For the single-standby synchronous commit levels, we
+	 * only do this if we are the current synchronous standby and we are
+	 * advancing the LSN further than it has been advanced before, so that
+	 * SyncRepWaitForLSN can skip waiting in some cases.
 	 */
-	if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
-		numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
-		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+	if (is_highest_priority_sync_standby)
 	{
-		walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
-		numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY);
+		if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
+			numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE,
+										MyWalSnd->write);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
+			numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH,
+										MyWalSnd->flush);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+			numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY,
+										MyWalSnd->apply);
+		}
 	}
 
+	/*
+	 * For causal_reads, all walsenders currently in available or joining
+	 * state must reach the LSN on their own, and standbys will reach LSNs in
+	 * any order.  It doesn't make sense to keep the highest seen LSN in a
+	 * single walsndctl->lsn element.  (CausalReadsWaitForLSN has handling for
+	 * LSNs that have already been reached).
+	 */
+	if (walsender_cr_available_or_joining)
+		numcausalreadsapply =
+			SyncRepWakeQueue(false, SYNC_REP_WAIT_CAUSAL_READS,
+							 MyWalSnd->apply);
+
 	LWLockRelease(SyncRepLock);
 
 	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%x",
@@ -502,7 +777,7 @@ SyncRepReleaseWaiters(void)
 	 * If we are managing the highest priority standby, though we weren't
 	 * prior to this, then announce we are now the sync standby.
 	 */
-	if (announce_next_takeover)
+	if (is_highest_priority_sync_standby && announce_next_takeover)
 	{
 		announce_next_takeover = false;
 		ereport(LOG,
@@ -577,9 +852,8 @@ SyncRepGetStandbyPriority(void)
  * Must hold SyncRepLock.
  */
 static int
-SyncRepWakeQueue(bool all, int mode)
+SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn)
 {
-	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	PGPROC	   *proc = NULL;
 	PGPROC	   *thisproc = NULL;
 	int			numprocs = 0;
@@ -596,7 +870,7 @@ SyncRepWakeQueue(bool all, int mode)
 		/*
 		 * Assume the queue is ordered by LSN
 		 */
-		if (!all && walsndctl->lsn[mode] < proc->waitLSN)
+		if (!all && lsn < proc->waitLSN)
 			return numprocs;
 
 		/*
@@ -656,7 +930,7 @@ SyncRepUpdateSyncStandbysDefined(void)
 			int			i;
 
 			for (i = 0; i < NUM_SYNC_REP_WAIT_MODE; i++)
-				SyncRepWakeQueue(true, i);
+				SyncRepWakeQueue(true, i, InvalidXLogRecPtr);
 		}
 
 		/*
@@ -708,13 +982,31 @@ SyncRepQueueIsOrderedByLSN(int mode)
 #endif
 
 /*
+ * Make sure that CausalReadsWaitForLSN can't return until after the given
+ * lease expiry time has been reached.  In other words, revoke the lease.
+ *
+ * Wake up all backends waiting in CausalReadsWaitForLSN, because the set of
+ * available/joining peers has changed, and there is a new stall time they
+ * need to observe.
+ */
+void
+CausalReadsBeginStall(TimestampTz lease_expiry_time)
+{
+	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+	WalSndCtl->stall_causal_reads_until =
+		Max(WalSndCtl->stall_causal_reads_until, lease_expiry_time);
+	SyncRepWakeQueue(true, SYNC_REP_WAIT_CAUSAL_READS, InvalidXLogRecPtr);
+	LWLockRelease(SyncRepLock);
+}
+
+/*
  * ===========================================================
  * Synchronous Replication functions executed by any process
  * ===========================================================
  */
 
 bool
-check_synchronous_standby_names(char **newval, void **extra, GucSource source)
+check_standby_names(char **newval, void **extra, GucSource source)
 {
 	char	   *rawstring;
 	List	   *elemlist;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index faea9ff..ff3a4db 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -55,6 +55,7 @@
 #include "libpq/pqformat.h"
 #include "libpq/pqsignal.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "replication/walsender.h"
 #include "storage/ipc.h"
@@ -149,7 +150,8 @@ static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
 static void XLogWalRcvSendReply(bool force, bool requestReply, bool includeApplyTimestamp);
 static void XLogWalRcvSendHSFeedback(bool immed);
-static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
+static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+								  TimestampTz *causalReadsUntil);
 
 /* Signal handlers */
 static void WalRcvSigHupHandler(SIGNAL_ARGS);
@@ -813,6 +815,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 	XLogRecPtr	walEnd;
 	TimestampTz sendTime;
 	bool		replyRequested;
+	TimestampTz causalReadsLease;
 
 	resetStringInfo(&incoming_message);
 
@@ -833,7 +836,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				walEnd = pq_getmsgint64(&incoming_message);
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, NULL);
 
 				buf += hdrlen;
 				len -= hdrlen;
@@ -845,7 +848,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				bool reportApplyTimestamp = false;
 
 				/* copy message to StringInfo */
-				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
+				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char) + sizeof(int64);
 				if (len != hdrlen)
 					ereport(ERROR,
 							(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -857,8 +860,10 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
 				replyRequested = pq_getmsgbyte(&incoming_message);
+				causalReadsLease = IntegerTimestampToTimestampTz(
+					pq_getmsgint64(&incoming_message));
 
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, &causalReadsLease);
 
 				/*
 				 * If no apply timestamps have been sent at the request of the
@@ -1207,15 +1212,52 @@ XLogWalRcvSendHSFeedback(bool immed)
  * Update shared memory status upon receiving a message from primary.
  *
  * 'walEnd' and 'sendTime' are the end-of-WAL and timestamp of the latest
- * message, reported by primary.
+ * message, reported by primary.  'causalReadsLease' is a pointer to
+ * the time the primary promises that this standby can safely claim to be
+ * causally consistent, to 0 if it cannot, or a NULL pointer for no change.
  */
 static void
-ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
+ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+					  TimestampTz *causalReadsLease)
 {
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
 	static TimestampTz lastRecordedTimestamp = 0;
 
+	/* Sanity check for the causalReadsLease time. */
+	if (causalReadsLease != NULL && *causalReadsLease != 0)
+	{
+		/* Deduce max_clock_skew from the causalReadsLease and sendTime. */
+#ifdef HAVE_INT64_TIMESTAMP
+		int64 diffMillis = (*causalReadsLease - sendTime) / 1000;
+#else
+		int64 diffMillis = (*causalReadsLease - sendTime) * 1000;
+#endif
+		int64 max_clock_skew = diffMillis / (CAUSAL_READS_CLOCK_SKEW_RATIO - 1);
+
+		if (sendTime > TimestampTzPlusMilliseconds(lastMsgReceiptTime, max_clock_skew))
+		{
+			/*
+			 * The primary's clock is more than max_clock_skew + network
+			 * latency ahead of the standby's clock.  (If the primary's clock
+			 * is more than max_clock_skew ahead of the standby's clock, but
+			 * by less than the network latency, then there isn't much we can
+			 * do to detect that; but it still seems useful to have this basic
+			 * sanity check for wildly misconfigured servers.)
+			 */
+			elog(LOG, "the primary server's clock time is too far ahead");
+			causalReadsLease = NULL;
+		}
+		/*
+		 * We could also try to detect cases where sendTime is more than
+		 * max_clock_skew in the past according to the standby's clock, but
+		 * that is indistinguishable from network latency/buffering, so we
+		 * could produce misleading error messages; if we do nothing, the
+		 * consequence is 'standby is not available for causal reads' errors
+		 * which should cause the user to investigate.
+		 */
+	}
+
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
 	if (walrcv->latestWalEnd < walEnd)
@@ -1223,6 +1265,8 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->latestWalEnd = walEnd;
 	walrcv->lastMsgSendTime = sendTime;
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
+	if (causalReadsLease != NULL)
+		walrcv->causalReadsLease = *causalReadsLease;
 	SpinLockRelease(&walrcv->mutex);
 
 	/*
diff --git a/src/backend/replication/walreceiverfuncs.c b/src/backend/replication/walreceiverfuncs.c
index 5f6e423..e502f74 100644
--- a/src/backend/replication/walreceiverfuncs.c
+++ b/src/backend/replication/walreceiverfuncs.c
@@ -28,6 +28,7 @@
 #include "replication/walreceiver.h"
 #include "storage/pmsignal.h"
 #include "storage/shmem.h"
+#include "utils/guc.h"
 #include "utils/timestamp.h"
 
 WalRcvData *WalRcv = NULL;
@@ -374,3 +375,21 @@ GetReplicationTransferLatency(void)
 
 	return ms;
 }
+
+/*
+ * Used by snapmgr to check if this standby has a valid lease, granting it the
+ * right to consider itself available for causal reads.
+ */
+bool
+WalRcvCausalReadsAvailable(void)
+{
+	WalRcvData *walrcv = WalRcv;
+	TimestampTz now = GetCurrentTimestamp();
+	bool result;
+
+	SpinLockAcquire(&walrcv->mutex);
+	result = walrcv->causalReadsLease != 0 && now <= walrcv->causalReadsLease;
+	SpinLockRelease(&walrcv->mutex);
+
+	return result;
+}
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 16d7abc..b4dad72 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -154,9 +154,20 @@ static StringInfoData tmpbuf;
  */
 static TimestampTz last_reply_timestamp = 0;
 
+static TimestampTz last_keepalive_timestamp = 0;
+
 /* Have we sent a heartbeat message asking for reply, since last reply? */
 static bool waiting_for_ping_response = false;
 
+/* How long do need to stay in JOINING state? */
+static XLogRecPtr causal_reads_joining_until = 0;
+
+/* The last causal reads lease sent to the standby. */
+static TimestampTz causal_reads_last_lease = 0;
+
+/* Is this WALSender listed in causal_reads_standby_names? */
+static bool am_potential_causal_reads_standby = false;
+
 /*
  * While streaming WAL in Copy mode, streamingDoneSending is set to true
  * after we have sent CopyDone. We should not send any more CopyData messages
@@ -242,6 +253,57 @@ InitWalSender(void)
 	SendPostmasterSignal(PMSIGNAL_ADVANCE_STATE_MACHINE);
 }
 
+ /*
+ * If we are exiting unexpectedly, we may need to communicate with concurrent
+ * causal_reads commits to maintain the causal consistency guarantee.
+ */
+static void
+PrepareUncleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * We've lost contact with the standby, but it may still be alive.  We
+		 * can't let any causal_reads transactions return until we've stalled
+		 * for long enough for a zombie standby to start raising errors
+		 * because its lease has expired.
+		 */
+		elog(LOG, "standby \"%s\" is lost (no longer available for causal reads)", application_name);
+		CausalReadsBeginStall(causal_reads_last_lease);
+
+		/*
+		 * We set the state to a lower level _after_ beginning the stall,
+		 * otherwise there would be a tiny window where commits could return
+		 * without observing the stall.
+		 */
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
+ * We are shutting down because we received a goodbye message from the
+ * walreceiver.
+ */
+static void
+PrepareCleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * The standby is shutting down, so it won't be running any more
+		 * transactions.  It is therefore safe to stop waiting for it, and no
+		 * stall is necessary.
+		 */
+		elog(LOG, "standby \"%s\" is leaving (no longer available for causal reads)", application_name);
+
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
 /*
  * Clean up after an error.
  *
@@ -266,7 +328,10 @@ WalSndErrorCleanup(void)
 
 	replication_active = false;
 	if (walsender_ready_to_stop)
+	{
+		PrepareUncleanExit();
 		proc_exit(0);
+	}
 
 	/* Revert back to startup state */
 	WalSndSetState(WALSNDSTATE_STARTUP);
@@ -278,6 +343,8 @@ WalSndErrorCleanup(void)
 static void
 WalSndShutdown(void)
 {
+	PrepareUncleanExit();
+
 	/*
 	 * Reset whereToSendOutput to prevent ereport from attempting to send any
 	 * more messages to the standby.
@@ -1388,6 +1455,7 @@ ProcessRepliesIfAny(void)
 		if (r < 0)
 		{
 			/* unexpected error or EOF */
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1404,6 +1472,7 @@ ProcessRepliesIfAny(void)
 		resetStringInfo(&reply_message);
 		if (pq_getmessage(&reply_message, 0))
 		{
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1453,6 +1522,7 @@ ProcessRepliesIfAny(void)
 				 * 'X' means that the standby is closing down the socket.
 				 */
 			case 'X':
+				PrepareCleanExit();
 				proc_exit(0);
 
 			default:
@@ -1584,6 +1654,83 @@ ProcessStandbyReplyMessage(void)
 	 */
 	{
 		WalSnd *walsnd = MyWalSnd;
+		WalSndCausalReadsState causal_reads_state = walsnd->causal_reads_state;
+		bool causal_reads_state_changed = false;
+		bool causal_reads_set_joining_until = false;
+
+		/*
+		 * Handle causal reads state transitions, if a causal_reads_timeout is
+		 * configured, this standby is listed in causal_reads_standby_names,
+		 * and we are a primary database (not a cascading standby).
+		 */
+		if (am_potential_causal_reads_standby &&
+			!am_cascading_walsender &&
+			applyLagUs >= 0)
+		{
+			if (applyLagUs / 1000 < causal_reads_timeout)
+			{
+				if (causal_reads_state == WALSNDCRSTATE_UNAVAILABLE)
+				{
+					/*
+					 * The standby is applying fast enough.  We can't grant a
+					 * lease yet though, we need to wait for everything that
+					 * was committed while this standby was unavailable to be
+					 * applied first.  We move to joining state while we wait
+					 * for the standby to catch up.
+					 */
+					causal_reads_state = WALSNDCRSTATE_JOINING;
+					causal_reads_set_joining_until = true;
+					causal_reads_state_changed = true;
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING &&
+						 applyPtr >= causal_reads_joining_until)
+				{
+					/*
+					 * The standby has applied everything committed before we
+					 * reached joining state, and has been waiting for remote
+					 * apply on this standby while it's been in joining state,
+					 * so it is safe to move to available state and send a
+					 * lease.
+					 */
+					causal_reads_state = WALSNDCRSTATE_AVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+			else
+			{
+				if (causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+					/*
+					 * We are dropping a causal reads available standby, so we
+					 * mustn't let any commit command that is waiting in
+					 * CausalReadsWaitForLSN return until we are sure that the
+					 * standby definitely knows that it's not available and
+					 * starts raising errors for causal_reads transactions.
+					 * TODO: We could just wait until the standby acks that
+					 * its lease has been cancelled, and start numbering
+					 * keepalives and sending the number back in replies, so
+					 * we know it's acking the right message; then lagging
+					 * standbys would be less disruptive, but for now we just
+					 * wait for the lease to expire, as we do when we lose
+					 * contact with a standby, for the sake of simplicity.
+					 */
+					CausalReadsBeginStall(causal_reads_last_lease);
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING)
+				{
+					/*
+					 * Dropping a joining standby doesn't require a stall,
+					 * because the standby doesn't think it's available, so
+					 * it's already raising the error for causal_reads
+					 * transactions.
+					 */
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+		}
 
 		SpinLockAcquire(&walsnd->mutex);
 		walsnd->write = writePtr;
@@ -1591,11 +1738,33 @@ ProcessStandbyReplyMessage(void)
 		walsnd->apply = applyPtr;
 		if (applyLagUs >= 0)
 			walsnd->applyLagUs = applyLagUs;
+		walsnd->causal_reads_state = causal_reads_state;
 		SpinLockRelease(&walsnd->mutex);
+
+		if (causal_reads_set_joining_until)
+		{
+			/*
+			 * Record the end of the primary's WAL at some arbitrary point
+			 * observed _after_ we moved to joining state (so that causal
+			 * reads commits start waiting, closing a race).  The standby
+			 * won't become available until it has replayed up to here.
+			 */
+			causal_reads_joining_until = GetFlushRecPtr();
+		}
+
+		if (causal_reads_state_changed)
+		{
+			WalSndKeepalive(true);
+			elog(LOG, "standby \"%s\" is %s", application_name,
+				 causal_reads_state == WALSNDCRSTATE_UNAVAILABLE ? "unavailable for causal reads" :
+				 causal_reads_state == WALSNDCRSTATE_JOINING ? "joining as a causal reads standby..." :
+				 causal_reads_state == WALSNDCRSTATE_AVAILABLE ? "available for causal reads" :
+				 "UNKNOWN");
+		}
 	}
 
 	if (!am_cascading_walsender)
-		SyncRepReleaseWaiters();
+		SyncRepReleaseWaiters(MyWalSnd->causal_reads_state >= WALSNDCRSTATE_JOINING);
 
 	/*
 	 * Advance our local xmin horizon when the client confirmed a flush.
@@ -1736,33 +1905,53 @@ ProcessStandbyHSFeedbackMessage(void)
  * If wal_sender_timeout is enabled we want to wake up in time to send
  * keepalives and to abort the connection if wal_sender_timeout has been
  * reached.
+ *
+ * But if causal_reads_timeout is enabled, we override that and send
+ * keepalives at a constant rate to replace expiring leases.
  */
 static long
 WalSndComputeSleeptime(TimestampTz now)
 {
 	long		sleeptime = 10000;		/* 10 s */
 
-	if (wal_sender_timeout > 0 && last_reply_timestamp > 0)
+	if ((wal_sender_timeout > 0 && last_reply_timestamp > 0) ||
+		am_potential_causal_reads_standby)
 	{
 		TimestampTz wakeup_time;
 		long		sec_to_timeout;
 		int			microsec_to_timeout;
 
-		/*
-		 * At the latest stop sleeping once wal_sender_timeout has been
-		 * reached.
-		 */
-		wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-												  wal_sender_timeout);
-
-		/*
-		 * If no ping has been sent yet, wakeup when it's time to do so.
-		 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
-		 * the timeout passed without a response.
-		 */
-		if (!waiting_for_ping_response)
+		if (am_potential_causal_reads_standby)
+		{
+			/*
+			 * Leases last for a period of between 50% and 100% of
+			 * causal_reads_timeout, depending on clock skew, assuming clock
+			 * skew is under the 25% of causal_reads_timeout.  We send new
+			 * leases every half a lease, so that there are no gaps between
+			 * leases.
+			 */
+			wakeup_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+													  causal_reads_timeout /
+													  CAUSAL_READS_KEEPALIVE_RATIO);
+		}
+		else
+		{
+			/*
+			 * At the latest stop sleeping once wal_sender_timeout has been
+			 * reached.
+			 */
 			wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-													  wal_sender_timeout / 2);
+													  wal_sender_timeout);
+
+			/*
+			 * If no ping has been sent yet, wakeup when it's time to do so.
+			 * WalSndKeepaliveIfNecessary() wants to send a keepalive once
+			 * half of the timeout passed without a response.
+			 */
+			if (!waiting_for_ping_response)
+				wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+														  wal_sender_timeout / 2);
+		}
 
 		/* Compute relative time until wakeup. */
 		TimestampDifference(now, wakeup_time,
@@ -1778,20 +1967,33 @@ WalSndComputeSleeptime(TimestampTz now)
 /*
  * Check whether there have been responses by the client within
  * wal_sender_timeout and shutdown if not.
+ *
+ * If causal_reads_timeout is configured we override that, so that
+ * unresponsive standbys are detected sooner.
  */
 static void
 WalSndCheckTimeOut(TimestampTz now)
 {
 	TimestampTz timeout;
+	int allowed_time;
 
 	/* don't bail out if we're doing something that doesn't require timeouts */
 	if (last_reply_timestamp <= 0)
 		return;
 
-	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
-										  wal_sender_timeout);
+	/*
+	 * If a causal_reads_timeout is configured, it is used instead of
+	 * wal_sender_timeout, to limit the time before an unresponsive causal
+	 * reads standby is dropped.
+	 */
+	if (am_potential_causal_reads_standby)
+		allowed_time = causal_reads_timeout;
+	else
+		allowed_time = wal_sender_timeout;
 
-	if (wal_sender_timeout > 0 && now >= timeout)
+	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
+										  allowed_time);
+	if (allowed_time > 0 && now >= timeout)
 	{
 		/*
 		 * Since typically expiration of replication timeout means
@@ -1824,6 +2026,9 @@ WalSndLoop(WalSndSendDataCallback send_data)
 	last_reply_timestamp = GetCurrentTimestamp();
 	waiting_for_ping_response = false;
 
+	/* Check if we are managing potential causal_reads standby. */
+	am_potential_causal_reads_standby = CausalReadsPotentialStandby();
+
 	/*
 	 * Loop until we reach the end of this timeline or the client requests to
 	 * stop streaming.
@@ -1984,6 +2189,7 @@ InitWalSenderSlot(void)
 			walsnd->flush = InvalidXLogRecPtr;
 			walsnd->apply = InvalidXLogRecPtr;
 			walsnd->state = WALSNDSTATE_STARTUP;
+			walsnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
 			walsnd->latch = &MyProc->procLatch;
 			SpinLockRelease(&walsnd->mutex);
 			/* don't need the lock anymore */
@@ -2753,6 +2959,24 @@ WalSndGetStateString(WalSndState state)
 	return "UNKNOWN";
 }
 
+/*
+ * Return a string constant representing the causal reads state. This is used
+ * in system views, and should *not* be translated.
+ */
+static const char *
+WalSndGetCausalReadsStateString(WalSndCausalReadsState causal_reads_state)
+{
+	switch (causal_reads_state)
+	{
+		case WALSNDCRSTATE_UNAVAILABLE:
+			return "unavailable";
+		case WALSNDCRSTATE_JOINING:
+			return "joining";
+		case WALSNDCRSTATE_AVAILABLE:
+			return "available";
+	}
+	return "UNKNOWN";
+}
 
 /*
  * Returns activity of walsenders, including pids and xlog locations sent to
@@ -2761,7 +2985,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	9
+#define PG_STAT_GET_WAL_SENDERS_COLS	10
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2812,6 +3036,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		int64		applyLagUs;
 		int			priority;
 		WalSndState state;
+		WalSndCausalReadsState causalReadsState;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
 		bool		nulls[PG_STAT_GET_WAL_SENDERS_COLS];
 
@@ -2821,6 +3046,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		SpinLockAcquire(&walsnd->mutex);
 		sentPtr = walsnd->sentPtr;
 		state = walsnd->state;
+		causalReadsState = walsnd->causal_reads_state;
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
@@ -2895,6 +3121,9 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				values[8] = CStringGetTextDatum("sync");
 			else
 				values[8] = CStringGetTextDatum("potential");
+
+			values[9] =
+				CStringGetTextDatum(WalSndGetCausalReadsStateString(causalReadsState));
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
@@ -2914,14 +3143,52 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 static void
 WalSndKeepalive(bool requestReply)
 {
+	TimestampTz now;
+	TimestampTz causal_reads_lease;
+
 	elog(DEBUG2, "sending replication keepalive");
 
+	/*
+	 * If the walsender currently deems the standby to be available for causal
+	 * reads, then it grants a causal reads lease.  The lease authorizes the
+	 * standby to consider itself available for causal reads until a short
+	 * time in the future.  The primary promises to uphold the causal reads
+	 * guarantee until that time, by stalling commits until the the lease has
+	 * expired if necessary.
+	 */
+	now = GetCurrentTimestamp();
+	if (MyWalSnd->causal_reads_state < WALSNDCRSTATE_AVAILABLE)
+		causal_reads_lease = 0; /* Not available, no lease granted. */
+	else
+	{
+		/*
+		 * Since this timestamp is being sent to the standby where it will be
+		 * compared against a time generated by the standby's system clock, we
+		 * must consider clock skew.  First, we decide on a maximum tolerable
+		 * difference between system clocks.  If the primary's clock is ahead
+		 * of the standby's by more than this, then all bets are off (the
+		 * standby could falsely believe it has a valid lease).  If the
+		 * primary's clock is behind the standby's by more than this, then the
+		 * standby will err the other way and generate spurious errors in
+		 * causal_reads mode.  Rather than having a separate GUC for this, we
+		 * derive it from causal_reads_timeout.
+		 */
+		int max_clock_skew = causal_reads_timeout / CAUSAL_READS_CLOCK_SKEW_RATIO;
+
+		/* Compute and remember the expiry time of the lease we're granting. */
+		causal_reads_last_lease = TimestampTzPlusMilliseconds(now, causal_reads_timeout);
+		/* The version we'll send to the standby is adjusted to tolerate clock skew. */
+		causal_reads_lease =
+			TimestampTzPlusMilliseconds(causal_reads_last_lease, -max_clock_skew);
+	}
+
 	/* construct the message... */
 	resetStringInfo(&output_message);
 	pq_sendbyte(&output_message, 'k');
 	pq_sendint64(&output_message, sentPtr);
-	pq_sendint64(&output_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(now));
 	pq_sendbyte(&output_message, requestReply ? 1 : 0);
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(causal_reads_lease));
 
 	/* ... and send it wrapped in CopyData */
 	pq_putmessage_noblock('d', output_message.data, output_message.len);
@@ -2939,23 +3206,35 @@ WalSndKeepaliveIfNecessary(TimestampTz now)
 	 * Don't send keepalive messages if timeouts are globally disabled or
 	 * we're doing something not partaking in timeouts.
 	 */
-	if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
-		return;
-
-	if (waiting_for_ping_response)
-		return;
+	if (!am_potential_causal_reads_standby)
+	{
+		if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
+			return;
+		if (waiting_for_ping_response)
+			return;
+	}
 
 	/*
 	 * If half of wal_sender_timeout has lapsed without receiving any reply
 	 * from the standby, send a keep-alive message to the standby requesting
 	 * an immediate reply.
+	 *
+	 * If causal_reads_timeout has been configured, use it to control
+	 * keepalive intervals rather than wal_sender_timeout, so that we can keep
+	 * replacing leases at the right frequency.
 	 */
-	ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-											wal_sender_timeout / 2);
+	if (am_potential_causal_reads_standby)
+		ping_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+												causal_reads_timeout /
+												CAUSAL_READS_KEEPALIVE_RATIO);
+	else
+		ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+												wal_sender_timeout / 2);
 	if (now >= ping_time)
 	{
 		WalSndKeepalive(true);
 		waiting_for_ping_response = true;
+		last_keepalive_timestamp = now;
 
 		/* Try to flush pending output to the client */
 		if (pq_flush_if_writable() != 0)
diff --git a/src/backend/utils/errcodes.txt b/src/backend/utils/errcodes.txt
index 49494f9..d81c089 100644
--- a/src/backend/utils/errcodes.txt
+++ b/src/backend/utils/errcodes.txt
@@ -306,6 +306,7 @@ Section: Class 40 - Transaction Rollback
 40001    E    ERRCODE_T_R_SERIALIZATION_FAILURE                              serialization_failure
 40003    E    ERRCODE_T_R_STATEMENT_COMPLETION_UNKNOWN                       statement_completion_unknown
 40P01    E    ERRCODE_T_R_DEADLOCK_DETECTED                                  deadlock_detected
+40P02    E    ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE                         causal_reads_not_available
 
 Section: Class 42 - Syntax Error or Access Rule Violation
 
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 06cb166..ac422e7 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -1634,6 +1634,16 @@ static struct config_bool ConfigureNamesBool[] =
 	},
 
 	{
+		{"causal_reads", PGC_USERSET, REPLICATION_STANDBY,
+		 gettext_noop("Enables causal reads."),
+		 NULL
+		},
+		&causal_reads,
+		false,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"syslog_sequence_numbers", PGC_SIGHUP, LOGGING_WHERE,
 			gettext_noop("Add sequence number to syslog messages to avoid duplicate suppression."),
 			NULL
@@ -1811,6 +1821,17 @@ static struct config_int ConfigureNamesInt[] =
 	},
 
 	{
+		{"causal_reads_timeout", PGC_SIGHUP, REPLICATION_STANDBY,
+			gettext_noop("Sets the maximum apply lag before causal reads standbys are no longer available."),
+			NULL,
+			GUC_UNIT_MS
+		},
+		&causal_reads_timeout,
+		0, 0, INT_MAX,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"max_connections", PGC_POSTMASTER, CONN_AUTH_SETTINGS,
 			gettext_noop("Sets the maximum number of concurrent connections."),
 			NULL
@@ -3454,7 +3475,18 @@ static struct config_string ConfigureNamesString[] =
 		},
 		&SyncRepStandbyNames,
 		"",
-		check_synchronous_standby_names, NULL, NULL
+		check_standby_names, NULL, NULL
+	},
+
+	{
+		{"causal_reads_standby_names", PGC_SIGHUP, REPLICATION_MASTER,
+			gettext_noop("List of names of potential causal reads standbys."),
+			NULL,
+			GUC_LIST_INPUT
+		},
+		&causal_reads_standby_names,
+		"*",
+		check_standby_names, NULL, NULL
 	},
 
 	{
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index ec4427f..fcc2c35 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -244,6 +244,15 @@
 				# from standby(s); '*' = all
 #vacuum_defer_cleanup_age = 0	# number of xacts by which cleanup is delayed
 
+#causal_reads_timeout = 0s      # maximum replication delay to tolerate from
+                                # standbys before dropping them from the set of
+				# available causal reads peers; 0 to disable
+				# causal reads
+
+#causal_reads_standy_names = '*'
+                                # standby servers that can potentially become
+				# available for causal reads; '*' = all
+
 # - Standby Servers -
 
 # These settings are ignored on a master server.
@@ -266,6 +275,14 @@
 #wal_retrieve_retry_interval = 5s	# time to wait before retrying to
 					# retrieve WAL after a failed attempt
 
+# - All Servers -
+
+#causal_reads = off                     # "on" in any pair of consecutive
+                                        # transactions guarantees that the second
+					# can see the first (even if the second
+					# is run on a standby), or will raise an
+					# error to report that the standby is
+					# unavailable for causal reads
 
 #------------------------------------------------------------------------------
 # QUERY TUNING
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index b88e012..6336240 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -46,8 +46,11 @@
 
 #include "access/transam.h"
 #include "access/xact.h"
+#include "access/xlog.h"
 #include "lib/pairingheap.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
+#include "replication/walreceiver.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
 #include "storage/procarray.h"
@@ -209,6 +212,16 @@ GetTransactionSnapshot(void)
 				 "cannot take query snapshot during a parallel operation");
 
 		/*
+		 * In causal_reads mode on a standby, check if we have definitely
+		 * applied WAL for any COMMIT that returned successfully on the
+		 * primary.
+		 */
+		if (causal_reads && RecoveryInProgress() && !WalRcvCausalReadsAvailable())
+			ereport(ERROR,
+					(errcode(ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE),
+					 errmsg("standby is not available for causal reads")));
+
+		/*
 		 * In transaction-snapshot mode, the first snapshot must live until
 		 * end of xact regardless of what the caller does with it, so we must
 		 * make a copy of it rather than returning CurrentSnapshotData
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 4054726..c0d7173 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2712,7 +2712,7 @@ DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f
 DESCR("statistics: information about currently active backends");
 DATA(insert OID = 3318 (  pg_stat_get_progress_info           PGNSP PGUID 12 1 100 0 0 f f f f t t s r 1 0 2249 "25" "{25,23,26,26,20,20,20,20,20,20,20,20,20,20}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{cmdtype,pid,datid,relid,param1,param2,param3,param4,param5,param6,param7,param8,param9,param10}" _null_ _null_ pg_stat_get_progress_info _null_ _null_ _null_ ));
 DESCR("statistics: information about progress of backends running maintenance command");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25}" "{o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25,25}" "{o,o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state,causal_reads_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index c005a42..dbfd601 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -24,14 +24,33 @@
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
 #define SYNC_REP_WAIT_APPLY		2
+#define SYNC_REP_WAIT_CAUSAL_READS 3
 
-#define NUM_SYNC_REP_WAIT_MODE	3
+#define NUM_SYNC_REP_WAIT_MODE	4
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
 #define SYNC_REP_WAITING			1
 #define SYNC_REP_WAIT_COMPLETE		2
 
+/*
+ * ratio of causal_read_timeout to max_clock_skew (4 means than the maximum
+ * tolerated clock difference between primary and standbys using causal_reads
+ * is 1/4 of causal_reads_timeout)
+ */
+#define CAUSAL_READS_CLOCK_SKEW_RATIO 4
+
+/*
+ * ratio of causal_reads_timeout to keepalive time (2 means that the effective
+ * keepalive time is 1/2 of the causal_reads_timeout GUC when it is non-zero)
+ */
+#define CAUSAL_READS_KEEPALIVE_RATIO 2
+
+/* GUC variables */
+extern int causal_reads_timeout;
+extern bool causal_reads;
+extern char *causal_reads_standby_names;
+
 /* user-settable parameters for synchronous replication */
 extern char *SyncRepStandbyNames;
 
@@ -43,16 +62,23 @@ extern void SyncRepCleanupAtProcExit(void);
 
 /* called by wal sender */
 extern void SyncRepInitConfig(void);
-extern void SyncRepReleaseWaiters(void);
+extern void SyncRepReleaseWaiters(bool walsender_cr_available_or_joining);
 
 /* called by checkpointer */
 extern void SyncRepUpdateSyncStandbysDefined(void);
 
+/* called by user backend (xact.c) */
+extern void CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN);
+
+/* called by wal sender */
+extern void CausalReadsBeginStall(TimestampTz lease_expiry_time);
+extern bool CausalReadsPotentialStandby(void);
+
 /* forward declaration to avoid pulling in walsender_private.h */
 struct WalSnd;
 extern struct WalSnd *SyncRepGetSynchronousStandby(void);
 
-extern bool check_synchronous_standby_names(char **newval, void **extra, GucSource source);
+extern bool check_standby_names(char **newval, void **extra, GucSource source);
 extern void assign_synchronous_commit(int newval, void *extra);
 
 #endif   /* _SYNCREP_H */
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 3ffa9e3..a357294 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -80,6 +80,13 @@ typedef struct
 	TimeLineID	receivedTLI;
 
 	/*
+	 * causalReadsLease is the time until which the primary has authorized
+	 * this standby to consider itself available for causal_reads mode, or 0
+	 * for not authorized.
+	 */
+	TimestampTz causalReadsLease;
+
+	/*
 	 * latestChunkStart is the starting byte position of the current "batch"
 	 * of received WAL.  It's actually the same as the previous value of
 	 * receivedUpto before the last flush to disk.  Startup process can use
@@ -169,4 +176,6 @@ extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
 extern void WalRcvWakeup(void);
 
+extern bool WalRcvCausalReadsAvailable(void);
+
 #endif   /* _WALRECEIVER_H */
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 4de43e8..f6e0e9e 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -27,6 +27,13 @@ typedef enum WalSndState
 	WALSNDSTATE_STREAMING
 } WalSndState;
 
+typedef enum WalSndCausalReadsState
+{
+	WALSNDCRSTATE_UNAVAILABLE = 0,
+	WALSNDCRSTATE_JOINING,
+	WALSNDCRSTATE_AVAILABLE
+} WalSndCausalReadsState;
+
 /*
  * Each walsender has a WalSnd struct in shared memory.
  */
@@ -34,6 +41,7 @@ typedef struct WalSnd
 {
 	pid_t		pid;			/* this walsender's process id, or 0 */
 	WalSndState state;			/* this walsender's state */
+	WalSndCausalReadsState causal_reads_state; /* the walsender's causal reads state */
 	XLogRecPtr	sentPtr;		/* WAL has been sent up to this point */
 	bool		needreload;		/* does currently-open file need to be
 								 * reloaded? */
@@ -89,6 +97,12 @@ typedef struct
 	 */
 	bool		sync_standbys_defined;
 
+	/*
+	 * Until when must commits in causal_reads stall?  This is used to wait
+	 * for causal reads leases to expire.
+	 */
+	TimestampTz	stall_causal_reads_until;
+
 	WalSnd		walsnds[FLEXIBLE_ARRAY_MEMBER];
 } WalSndCtlData;
 
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index fc4b765..44f826f 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1785,10 +1785,11 @@ pg_stat_replication| SELECT s.pid,
     w.replay_location,
     w.replay_lag,
     w.sync_priority,
-    w.sync_state
+    w.sync_state,
+    w.causal_reads_state
    FROM pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, wait_event_type, wait_event, xact_start, query_start, backend_start, state_change, client_addr, client_hostname, client_port, backend_xid, backend_xmin, ssl, sslversion, sslcipher, sslbits, sslcompression, sslclientdn),
     pg_authid u,
-    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state)
+    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state, causal_reads_state)
   WHERE ((s.usesysid = u.oid) AND (s.pid = w.pid));
 pg_stat_ssl| SELECT s.pid,
     s.ssl,
interrupting_cow.ctext/x-csrc; charset=US-ASCII; name=interrupting_cow.cDownload
#62Michael Paquier
michael.paquier@gmail.com
In reply to: Thomas Munro (#61)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Sun, Mar 27, 2016 at 7:30 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Sat, Mar 26, 2016 at 2:48 AM, Michael Paquier
<michael.paquier@gmail.com> wrote:

Should we worried about potential backward-incompatibilities with the
new return values of walrcv_receive?

There are three changes to the walrcv_receive interface:

1. It now takes a latch pointer, which may be NULL.

2. If the latch pointer is non-NULL, the existing function might
return a new sentinel value WALRCV_RECEIVE_LATCH_SET. (The
pre-existing sentinel value -1 is still in use and has the same value
and meaning as before, but it now has a name:
WALRCV_RECEIVE_COPY_ENDED.)

3. It will no longer return when the process is signalled (a latch
should be used to ask it to return instead).

Effectively, any client code would need to change at least to add NULL
or possibly a latch if it needs to ask it to return, and any
alternative implementation of the WAL receiver interface would need to
use WaitEventSet (or WaitLatchOrSocket) as its event loop instead of
whatever it might be using now so that it can detect a latch's state.
But in any case, any such code would fail to compile against 9.6 due
to the new argument, and then you'd only be able to get the new return
value if you asked for it by passing in a latch. What affected code
are we aware of -- either users of libpqwalreceiver.so or other WAL
receiver implementations?

Any negative value returned by walrcv_receive would have stopped the
replication stream, not only -1. And as you say, it's actually a good
thing that the interface of walrcv_receive is changing with this
patch, this way compilation would just fail and failures would not
happen discreetly. That's too low-level to be mentioned in the release
notes either way, so just a compilation failure is acceptable to me.

Do you have numbers to share regarding how is performing the
latch-based approach and the approach that used SIGUSR2 when
remote_apply is used?

I couldn't measure any significant change (Linux, all on localhost, 128 cores):

pgbench -c 1 -N bench2 -T 600

0001-remote-apply-v5.patch (signals), remote_apply -> 449 TPS
0001-remote-apply-v6.patch (latches), remote_apply -> 452 TPS

pgbench -c 64 -j 32 -N bench2 -T 600

0001-remote-apply-v5.patch (signals), remote_apply -> 8536 TPS
0001-remote-apply-v6.patch (latches), remote_apply -> 8534 TPS

Which concludes that both imply the same kind of performance. We are
likely seeing noise.

Incidentally, I also did some testing on what happens when you signal
a process that is busily writing and fsyncing. I tested a few
different kernels, write sizes and disk latencies and saw that things
were fine on all of them up to 10k signals/sec but after that some
systems' fsync performance started to reduce. Only Linux on Power was
still happily fsyncing at around 3/4 of full speed when signalled with
a 2MHz+ tight kill loop (!), while FreeBSD and Linux on Intel weren't
able to make much progress at all under such adversity. So I suppose
that if you could get the commit rate up into 5 figures you might be
able to measure an improvement for the latch version due to
latch-collapsing, though I noticed a large amount of signal-collapsing
going on at the OS level on all systems I tested anyway, so maybe it
wouldn't make a difference. I attach that test program for interest.

Interesting results.

Also, I updated the comment for the declaration of the latch in
walreceiver.h to say something about the new usage.

New patch series attached.

static void WalRcvQuickDieHandler(SIGNAL_ARGS);

-
static void
ProcessWalRcvInterrupts(void)
Noise here.

+   ret = WaitLatchOrSocket(latch, events, PQsocket(streamConn), timeout_ms);
+
+   if (ret & WL_POSTMASTER_DEATH)
+       exit(0);
Exiting within libpqwalreceiver.so is no good. I think that even in
the case of a postmaster cleanup we should allow things to be cleaned
up.
 /*
+ * Wake up the walreceiver if it happens to be blocked in walrcv_receive,
+ * and tell it that a commit record has been applied.
+ *
+ * This is called by the startup process whenever interesting xlog records
+ * are applied, so that walreceiver can check if it needs to send an apply
+ * notification back to the master which may be waiting in a COMMIT with
+ * synchronous_commit = remote_apply.
+ */
+void
+WalRcvWakeup(void)
+{
+   SetLatch(&WalRcv->latch);
+}
I think here that it would be good to add an assertion telling that
this can just be called by the startup process while in recovery,
WalRcv->latch is not protected by a mutex lock.
+maximum of 'timeout' ms. If a message was successfully read, returns
+its length. Otherwise returns 0 for timeout, WALRCV_RECEIVE_COPY_ENDED
+for disconnection or WALRCV_RECEIVE_LATCH_SET. On success, a pointer
Having an assigned constant name for timeout would be good for
consistency with the rest.

I have been also thinking a lot about this patch, and the fact that
the WAL receiver latch is being used within the internals of
libpqwalreceiver has been bugging me a lot, because this makes the
wait phase happening within the libpqwalreceiver depend on something
that only the WAL receiver had a only control on up to now (among the
things thought: having a second latch for libpqwalreceiver, having an
event interface for libpqwalreceiver, switch libpq_receive into being
asynchronous...). At the end, we need a way to allow the startup
process to let the WAL receiver process know that it needs to be
interrupted via shared memory, and that's the WAL receiver latch, the
removal of epoll stuff cleans up some code at the end. So it seems
that I finally made my mind on 0001 and it looks good to me except the
small things mentioned above.
--
Michael

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#63Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Michael Paquier (#62)
4 attachment(s)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Mon, Mar 28, 2016 at 8:54 PM, Michael Paquier
<michael.paquier@gmail.com> wrote:

static void WalRcvQuickDieHandler(SIGNAL_ARGS);

-
static void
ProcessWalRcvInterrupts(void)
Noise here.

Fixed.

+   ret = WaitLatchOrSocket(latch, events, PQsocket(streamConn), timeout_ms);
+
+   if (ret & WL_POSTMASTER_DEATH)
+       exit(0);
Exiting within libpqwalreceiver.so is no good. I think that even in
the case of a postmaster cleanup we should allow things to be cleaned
up.

I agree. I suppose this is really a symptom of the problem you talked
about below: see response there.

/*
+ * Wake up the walreceiver if it happens to be blocked in walrcv_receive,
+ * and tell it that a commit record has been applied.
+ *
+ * This is called by the startup process whenever interesting xlog records
+ * are applied, so that walreceiver can check if it needs to send an apply
+ * notification back to the master which may be waiting in a COMMIT with
+ * synchronous_commit = remote_apply.
+ */
+void
+WalRcvWakeup(void)
+{
+   SetLatch(&WalRcv->latch);
+}
I think here that it would be good to add an assertion telling that
this can just be called by the startup process while in recovery,
WalRcv->latch is not protected by a mutex lock.
+maximum of 'timeout' ms. If a message was successfully read, returns
+its length. Otherwise returns 0 for timeout, WALRCV_RECEIVE_COPY_ENDED
+for disconnection or WALRCV_RECEIVE_LATCH_SET. On success, a pointer
Having an assigned constant name for timeout would be good for
consistency with the rest.

Yeah, I guess it would have to mirror all the WL_XXX flags if we
continue down that path, but...

I have been also thinking a lot about this patch, and the fact that
the WAL receiver latch is being used within the internals of
libpqwalreceiver has been bugging me a lot, because this makes the
wait phase happening within the libpqwalreceiver depend on something
that only the WAL receiver had a only control on up to now (among the
things thought: having a second latch for libpqwalreceiver, having an
event interface for libpqwalreceiver, switch libpq_receive into being
asynchronous...).

Yeah, it bugs me too. Do you prefer this?

int walrcv_receive(char **buffer, int *wait_fd);

Return value -1 means end-of-copy as before, return value 0 means "no
data available now, please call me again when *wait_fd is ready to
read". Then walreceiver.c can look after the WaitLatchOrSocket call
and deal with socket readiness, postmaster death, timeout and latch,
and libpqwalreceiver.c doesn't know anything about all that stuff
anymore, but it is now part of the interface that it must expose a
file descriptor for readiness testing when it doesn't have data
available.

Please find attached a new patch series which does it that way.

At the end, we need a way to allow the startup
process to let the WAL receiver process know that it needs to be
interrupted via shared memory, and that's the WAL receiver latch, the
removal of epoll stuff cleans up some code at the end. So it seems
that I finally made my mind on 0001 and it looks good to me except the
small things mentioned above.

Thanks!

--
Thomas Munro
http://www.enterprisedb.com

Attachments:

0001-remote-apply-v8.patchapplication/octet-stream; name=0001-remote-apply-v8.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index d48a13f..6ab65a7 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -2143,8 +2143,8 @@ include_dir 'conf.d'
         Specifies whether transaction commit will wait for WAL records
         to be written to disk before the command returns a <quote>success</>
         indication to the client.  Valid values are <literal>on</>,
-        <literal>remote_write</>, <literal>local</>, and <literal>off</>.
-        The default, and safe, setting
+        <literal>remote_write</>, <literal>remote_apply</>, <literal>local</>,
+        and <literal>off</>.  The default, and safe, setting
         is <literal>on</>.  When <literal>off</>, there can be a delay between
         when success is reported to the client and when the transaction is
         really guaranteed to be safe against a server crash.  (The maximum
@@ -2177,6 +2177,10 @@ include_dir 'conf.d'
         ensure data preservation even if the standby instance of
         <productname>PostgreSQL</> were to crash, but not if the standby
         suffers an operating-system-level crash.
+        When set to <literal>remote_apply</>, commits will wait until a reply
+        from the current synchronous standby indicates it has received the
+        commit record of the transaction and applied it, so that it has become
+        visible to queries.
        </para>
        <para>
         When synchronous
diff --git a/doc/src/sgml/high-availability.sgml b/doc/src/sgml/high-availability.sgml
index 19d613e..03c6c30 100644
--- a/doc/src/sgml/high-availability.sgml
+++ b/doc/src/sgml/high-availability.sgml
@@ -1081,6 +1081,9 @@ primary_slot_name = 'node_a_slot'
     WAL record is then sent to the standby. The standby sends reply
     messages each time a new batch of WAL data is written to disk, unless
     <varname>wal_receiver_status_interval</> is set to zero on the standby.
+    In the case that <varname>synchronous_commit</> is set to
+    <literal>remote_apply</>, the standby sends reply messages when the commit
+    record is replayed, making the transaction visible.
     If the standby is the first matching standby, as specified in
     <varname>synchronous_standby_names</> on the primary, the reply
     messages from that standby will be used to wake users waiting for
@@ -1107,6 +1110,14 @@ primary_slot_name = 'node_a_slot'
    </para>
 
    <para>
+    Setting <varname>synchronous_commit</> to <literal>remote_apply</> will
+    cause each commit to wait until the current synchronous standby reports
+    that it has replayed the transaction, making it visible to user queries.
+    In simple cases, this allows for load balancing with causal consistency
+    on a single hot standby.
+   </para>
+
+   <para>
     Users will stop waiting if a fast shutdown is requested.  However, as
     when using asynchronous replication, the server will not fully
     shutdown until all outstanding WAL records are transferred to the currently
@@ -1160,9 +1171,10 @@ primary_slot_name = 'node_a_slot'
     <title>Planning for High Availability</title>
 
    <para>
-    Commits made when <varname>synchronous_commit</> is set to <literal>on</>
-    or <literal>remote_write</> will wait until the synchronous standby responds. The response
-    may never occur if the last, or only, standby should crash.
+    Commits made when <varname>synchronous_commit</> is set to <literal>on</>,
+    <literal>remote_write</> or <literal>remote_apply</> will wait until the
+    synchronous standby responds. The response may never occur if the last, or
+    only, standby should crash.
    </para>
 
    <para>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index e7234c8..893c2fa 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -1107,7 +1107,7 @@ EndPrepare(GlobalTransaction gxact)
 	 * Note that at this stage we have marked the prepare, but still show as
 	 * running in the procarray (twice!) and continue to hold locks.
 	 */
-	SyncRepWaitForLSN(gxact->prepare_end_lsn);
+	SyncRepWaitForLSN(gxact->prepare_end_lsn, false);
 
 	records.tail = records.head = NULL;
 	records.num_chunks = 0;
@@ -2103,7 +2103,7 @@ RecordTransactionCommitPrepared(TransactionId xid,
 	 * Note that at this stage we have marked clog, but still show as running
 	 * in the procarray and continue to hold locks.
 	 */
-	SyncRepWaitForLSN(recptr);
+	SyncRepWaitForLSN(recptr, true);
 }
 
 /*
@@ -2156,5 +2156,5 @@ RecordTransactionAbortPrepared(TransactionId xid,
 	 * Note that at this stage we have marked clog, but still show as running
 	 * in the procarray and continue to hold locks.
 	 */
-	SyncRepWaitForLSN(recptr);
+	SyncRepWaitForLSN(recptr, true);
 }
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index e315405..da595fd 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,7 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
-		SyncRepWaitForLSN(XactLastRecEnd);
+		SyncRepWaitForLSN(XactLastRecEnd, true);
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5123,6 +5123,13 @@ XactLogCommitRecord(TimestampTz commit_time,
 		xl_xinfo.xinfo |= XACT_COMPLETION_FORCE_SYNC_COMMIT;
 
 	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this commit is applied.
+	 */
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
+	/*
 	 * Relcache invalidations requires information about the current database
 	 * and so does logical decoding.
 	 */
@@ -5300,6 +5307,13 @@ XactLogAbortRecord(TimestampTz abort_time,
 	if (xl_xinfo.xinfo & XACT_XINFO_HAS_TWOPHASE)
 		XLogRegisterData((char *) (&xl_twophase), sizeof(xl_xact_twophase));
 
+	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this abort is applied.
+	 */
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
 	return XLogInsert(RM_XACT_ID, info);
 }
 
@@ -5459,6 +5473,13 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 	if (XactCompletionForceSyncCommit(parsed->xinfo))
 		XLogFlush(lsn);
 
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit = remote_apply), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 /*
@@ -5545,6 +5566,14 @@ xact_redo_abort(xl_xact_parsed_abort *parsed, TransactionId xid)
 		smgrdounlink(srel, true);
 		smgrclose(srel);
 	}
+
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit = remote_apply), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 void
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index b119a47..3e454f5 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -345,6 +345,9 @@ static XLogRecPtr RedoRecPtr;
  */
 static bool doPageWrites;
 
+/* Has the recovery code requested a walreceiver wakeup? */
+static bool doRequestWalReceiverReply;
+
 /*
  * RedoStartLSN points to the checkpoint's REDO location which is specified
  * in a backup label file, backup history file or control file. In standby
@@ -6879,6 +6882,19 @@ StartupXLOG(void)
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
 				SpinLockRelease(&XLogCtl->info_lck);
 
+				/*
+				 * If rm_redo reported that it applied a commit record that
+				 * the master is waiting for by calling
+				 * XLogRequestWalReceiverReply, then we wake up the receiver
+				 * so that it notices the updated lastReplayedEndRecPtr and
+				 * sends a reply to the master.
+				 */
+				if (doRequestWalReceiverReply)
+				{
+					doRequestWalReceiverReply = false;
+					WalRcvWakeup();
+				}
+
 				/* Remember this record as the last-applied one */
 				LastRec = ReadRecPtr;
 
@@ -11594,3 +11610,12 @@ SetWalWriterSleeping(bool sleeping)
 	XLogCtl->WalWriterSleeping = sleeping;
 	SpinLockRelease(&XLogCtl->info_lck);
 }
+
+/*
+ * Schedule a walreceiver wakeup in the main recovery loop.
+ */
+void
+XLogRequestWalReceiverReply(void)
+{
+	doRequestWalReceiverReply = true;
+}
diff --git a/src/backend/replication/README b/src/backend/replication/README
index 8e5bf0d..419a2d7 100644
--- a/src/backend/replication/README
+++ b/src/backend/replication/README
@@ -16,14 +16,16 @@ bool walrcv_connect(char *conninfo, XLogRecPtr startpoint)
 Establish connection to the primary, and starts streaming from 'startpoint'.
 Returns true on success.
 
-bool walrcv_receive(int timeout, unsigned char *type, char **buffer, int *len)
-
-Retrieve any message available through the connection, blocking for
-maximum of 'timeout' ms. If a message was successfully read, returns true,
-otherwise false. On success, a pointer to the message payload is stored in
-*buffer, length in *len, and the type of message received in *type. The
-returned buffer is valid until the next call to walrcv_* functions, the
-caller should not attempt freeing it.
+int walrcv_receive(char **buffer, int *wait_fd)
+
+Retrieve any message available without blocking through the
+connection.  If a message was successfully read, returns its
+length. If the connection is closed, returns -1.  Otherwise returns 0
+to indicate that no data is available, and sets *wait_fd to a file
+descriptor which can be waited on before trying again.  On success, a
+pointer to the message payload is stored in *buffer. The returned
+buffer is valid until the next call to walrcv_* functions, and the
+caller should not attempt to free it.
 
 void walrcv_send(const char *buffer, int nbytes)
 
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index 4ee4d71..a3bec49 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -52,7 +52,7 @@ static void libpqrcv_readtimelinehistoryfile(TimeLineID tli, char **filename, ch
 static bool libpqrcv_startstreaming(TimeLineID tli, XLogRecPtr startpoint,
 						char *slotname);
 static void libpqrcv_endstreaming(TimeLineID *next_tli);
-static int	libpqrcv_receive(int timeout, char **buffer);
+static int	libpqrcv_receive(char **buffer, int *wait_fd);
 static void libpqrcv_send(const char *buffer, int nbytes);
 static void libpqrcv_disconnect(void);
 
@@ -463,8 +463,7 @@ libpqrcv_disconnect(void)
 }
 
 /*
- * Receive a message available from XLOG stream, blocking for
- * maximum of 'timeout' ms.
+ * Receive a message available from XLOG stream.
  *
  * Returns:
  *
@@ -472,15 +471,15 @@ libpqrcv_disconnect(void)
  *	 point to a buffer holding the received message. The buffer is only valid
  *	 until the next libpqrcv_* call.
  *
- *	 0 if no data was available within timeout, or wait was interrupted
- *	 by signal.
+ *	 If no data was available immediately, returns 0, and *wait_fd is set to a
+ *	 file descriptor which can be waited on before trying again.
  *
  *	 -1 if the server ended the COPY.
  *
  * ereports on error.
  */
 static int
-libpqrcv_receive(int timeout, char **buffer)
+libpqrcv_receive(char **buffer, int *wait_fd)
 {
 	int			rawlen;
 
@@ -492,16 +491,7 @@ libpqrcv_receive(int timeout, char **buffer)
 	rawlen = PQgetCopyData(streamConn, &recvBuf, 1);
 	if (rawlen == 0)
 	{
-		/*
-		 * No data available yet. If the caller requested to block, wait for
-		 * more data to arrive.
-		 */
-		if (timeout > 0)
-		{
-			if (!libpq_select(timeout))
-				return 0;
-		}
-
+		/* Try consuming some data. */
 		if (PQconsumeInput(streamConn) == 0)
 			ereport(ERROR,
 					(errmsg("could not receive data from WAL stream: %s",
@@ -510,7 +500,11 @@ libpqrcv_receive(int timeout, char **buffer)
 		/* Now that we've consumed some input, try again */
 		rawlen = PQgetCopyData(streamConn, &recvBuf, 1);
 		if (rawlen == 0)
+		{
+			/* Tell caller to try again when our socket is ready. */
+			*wait_fd = PQsocket(streamConn);
 			return 0;
+		}
 	}
 	if (rawlen == -1)			/* end-of-streaming or error */
 	{
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 92faf4e..1ee1bc5 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -91,13 +91,25 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * to the wait queue. During SyncRepWakeQueue() a WALSender changes
  * the state to SYNC_REP_WAIT_COMPLETE once replication is confirmed.
  * This backend then resets its state to SYNC_REP_NOT_WAITING.
+ *
+ * 'lsn' represents the LSN to wait for.  'commit' indicates whether this LSN
+ * represents a commit/abort record.  If it's not, then we wait only for the
+ * WAL to be flushed if synchronous_commit is set to the higher level of
+ * remote_apply, because standbys only send apply feedback for commit/abort
+ * records.
  */
 void
-SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
+SyncRepWaitForLSN(XLogRecPtr lsn, bool commit)
 {
 	char	   *new_status = NULL;
 	const char *old_status;
-	int			mode = SyncRepWaitMode;
+	int			mode;
+
+	/* Cap the level for non-commit records to remote flush only. */
+	if (commit)
+		mode = SyncRepWaitMode;
+	else
+		mode = Max(SyncRepWaitMode, SYNC_REP_WAIT_FLUSH);
 
 	/*
 	 * Fast exit if user has not requested sync replication, or there are no
@@ -122,7 +134,7 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 	 * to be a low cost check.
 	 */
 	if (!WalSndCtl->sync_standbys_defined ||
-		XactCommitLSN <= WalSndCtl->lsn[mode])
+		lsn <= WalSndCtl->lsn[mode])
 	{
 		LWLockRelease(SyncRepLock);
 		return;
@@ -132,7 +144,7 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 	 * Set our waitLSN so WALSender will know when to wake us, and add
 	 * ourselves to the queue.
 	 */
-	MyProc->waitLSN = XactCommitLSN;
+	MyProc->waitLSN = lsn;
 	MyProc->syncRepState = SYNC_REP_WAITING;
 	SyncRepQueueInsert(mode);
 	Assert(SyncRepQueueIsOrderedByLSN(mode));
@@ -147,7 +159,7 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 		new_status = (char *) palloc(len + 32 + 1);
 		memcpy(new_status, old_status, len);
 		sprintf(new_status + len, " waiting for %X/%X",
-				(uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN);
+				(uint32) (lsn >> 32), (uint32) lsn);
 		set_ps_display(new_status, false);
 		new_status[len] = '\0'; /* truncate off " waiting ..." */
 	}
@@ -416,6 +428,7 @@ SyncRepReleaseWaiters(void)
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
+	int			numapply = 0;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
@@ -462,12 +475,18 @@ SyncRepReleaseWaiters(void)
 		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
 		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
 	}
+	if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+	{
+		walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+		numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY);
+	}
 
 	LWLockRelease(SyncRepLock);
 
-	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X",
+	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%x",
 		 numwrite, (uint32) (MyWalSnd->write >> 32), (uint32) MyWalSnd->write,
-	   numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush);
+		 numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush,
+		 numapply, (uint32) (MyWalSnd->apply >> 32), (uint32) MyWalSnd->apply);
 
 	/*
 	 * If we are managing the highest priority standby, though we weren't
@@ -728,6 +747,9 @@ assign_synchronous_commit(int newval, void *extra)
 		case SYNCHRONOUS_COMMIT_REMOTE_FLUSH:
 			SyncRepWaitMode = SYNC_REP_WAIT_FLUSH;
 			break;
+		case SYNCHRONOUS_COMMIT_REMOTE_APPLY:
+			SyncRepWaitMode = SYNC_REP_WAIT_APPLY;
+			break;
 		default:
 			SyncRepWaitMode = SYNC_REP_NO_WAIT;
 			break;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 7b36e02..e6543e1 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -380,6 +380,8 @@ WalReceiverMain(void)
 			{
 				char	   *buf;
 				int			len;
+				int			wait_fd = PGINVALID_SOCKET;
+				int			rc;
 
 				/*
 				 * Emergency bailout if postmaster has died.  This is to avoid
@@ -407,8 +409,8 @@ WalReceiverMain(void)
 					XLogWalRcvSendHSFeedback(true);
 				}
 
-				/* Wait a while for data to arrive */
-				len = walrcv_receive(NAPTIME_PER_CYCLE, &buf);
+				/* See if we can read data immediately */
+				len = walrcv_receive(&buf, &wait_fd);
 				if (len != 0)
 				{
 					/*
@@ -429,7 +431,7 @@ WalReceiverMain(void)
 						}
 						else if (len == 0)
 							break;
-						else if (len < 0)
+						else if (len == -1)
 						{
 							ereport(LOG,
 									(errmsg("replication terminated by primary server"),
@@ -439,7 +441,7 @@ WalReceiverMain(void)
 							endofwal = true;
 							break;
 						}
-						len = walrcv_receive(0, &buf);
+						len = walrcv_receive(&buf, &wait_fd);
 					}
 
 					/* Let the master know that we received some data. */
@@ -452,7 +454,31 @@ WalReceiverMain(void)
 					 */
 					XLogWalRcvFlush(false);
 				}
-				else
+
+				/*
+				 * Ideally we would reuse a WaitEventSet object repeatedly
+				 * here to avoid the overheads of WaitLatchOrSocket on epoll
+				 * systems, but we can't be sure that libpq (or any other
+				 * walreceiver implementation) has the same socket (even if
+				 * the fd is the same number, it may have been closed and
+				 * reopened since the last call to wait_for_wal_stream).  In
+				 * future, if there is a function for removing sockets from
+				 * WaitEventSet, then we could add and remove just the socket
+				 * each time, potentially avoiding some system calls.
+				 */
+				Assert(wait_fd != PGINVALID_SOCKET);
+				rc = WaitLatchOrSocket(&walrcv->latch,
+									   WL_POSTMASTER_DEATH | WL_SOCKET_READABLE |
+									   WL_TIMEOUT | WL_LATCH_SET,
+									   wait_fd,
+									   NAPTIME_PER_CYCLE);
+				if (rc & WL_LATCH_SET)
+				{
+					/* The recovery process has asked us to send apply feedback now. */
+					ResetLatch(&walrcv->latch);
+					XLogWalRcvSendReply(true, false);
+				}
+				if (rc & WL_TIMEOUT)
 				{
 					/*
 					 * We didn't receive anything new. If we haven't heard
@@ -1222,6 +1248,20 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 }
 
 /*
+ * Wake up the walreceiver main loop.
+ *
+ * This is called by the startup process whenever interesting xlog records
+ * are applied, so that walreceiver can check if it needs to send an apply
+ * notification back to the master which may be waiting in a COMMIT with
+ * synchronous_commit = remote_apply.
+ */
+void
+WalRcvWakeup(void)
+{
+	SetLatch(&WalRcv->latch);
+}
+
+/*
  * Return a string constant representing the state. This is used
  * in system functions and views, and should *not* be translated.
  */
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 65a6cd4..06cb166 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -345,12 +345,13 @@ static const struct config_enum_entry constraint_exclusion_options[] = {
 };
 
 /*
- * Although only "on", "off", "remote_write", and "local" are documented, we
- * accept all the likely variants of "on" and "off".
+ * Although only "on", "off", "remote_apply", "remote_write", and "local" are
+ * documented, we accept all the likely variants of "on" and "off".
  */
 static const struct config_enum_entry synchronous_commit_options[] = {
 	{"local", SYNCHRONOUS_COMMIT_LOCAL_FLUSH, false},
 	{"remote_write", SYNCHRONOUS_COMMIT_REMOTE_WRITE, false},
+	{"remote_apply", SYNCHRONOUS_COMMIT_REMOTE_APPLY, false},
 	{"on", SYNCHRONOUS_COMMIT_ON, false},
 	{"off", SYNCHRONOUS_COMMIT_OFF, false},
 	{"true", SYNCHRONOUS_COMMIT_ON, true},
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index 5536012..ec4427f 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -177,7 +177,7 @@
 					# (change requires restart)
 #fsync = on				# turns forced synchronization on or off
 #synchronous_commit = on		# synchronization level;
-					# off, local, remote_write, or on
+					# off, local, remote_write, remote_apply, or on
 #wal_sync_method = fsync		# the default is the first option
 					# supported by the operating system:
 					#   open_datasync
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 3ba23f5..21fcea1 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -60,7 +60,9 @@ typedef enum
 	SYNCHRONOUS_COMMIT_LOCAL_FLUSH,		/* wait for local flush only */
 	SYNCHRONOUS_COMMIT_REMOTE_WRITE,	/* wait for local flush and remote
 										 * write */
-	SYNCHRONOUS_COMMIT_REMOTE_FLUSH		/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_FLUSH,	/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_APPLY		/* wait for local flush and remote
+										 * apply */
 }	SyncCommitLevel;
 
 /* Define the default setting for synchonous_commit */
@@ -144,10 +146,13 @@ typedef void (*SubXactCallback) (SubXactEvent event, SubTransactionId mySubid,
  * EOXact... routines which run at the end of the original transaction
  * completion.
  */
+#define XACT_COMPLETION_SYNC_APPLY_FEEDBACK		(1U << 29)
 #define XACT_COMPLETION_UPDATE_RELCACHE_FILE	(1U << 30)
 #define XACT_COMPLETION_FORCE_SYNC_COMMIT		(1U << 31)
 
 /* Access macros for above flags */
+#define XactCompletionSyncApplyFeedback(xinfo) \
+	((xinfo & XACT_COMPLETION_SYNC_APPLY_FEEDBACK) != 0)
 #define XactCompletionRelcacheInitFileInval(xinfo) \
 	((xinfo & XACT_COMPLETION_UPDATE_RELCACHE_FILE) != 0)
 #define XactCompletionForceSyncCommit(xinfo) \
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 74a1394..a7dcdae 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -267,6 +267,8 @@ extern bool CheckPromoteSignal(void);
 extern void WakeupRecovery(void);
 extern void SetWalWriterSleeping(bool sleeping);
 
+extern void XLogRequestWalReceiverReply(void);
+
 extern void assign_max_wal_size(int newval, void *extra);
 extern void assign_checkpoint_completion_target(double newval, void *extra);
 
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index 96e059b..c005a42 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -23,8 +23,9 @@
 #define SYNC_REP_NO_WAIT		-1
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
+#define SYNC_REP_WAIT_APPLY		2
 
-#define NUM_SYNC_REP_WAIT_MODE	2
+#define NUM_SYNC_REP_WAIT_MODE	3
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
@@ -35,7 +36,7 @@
 extern char *SyncRepStandbyNames;
 
 /* called by user backend */
-extern void SyncRepWaitForLSN(XLogRecPtr XactCommitLSN);
+extern void SyncRepWaitForLSN(XLogRecPtr lsn, bool commit);
 
 /* called at backend exit */
 extern void SyncRepCleanupAtProcExit(void);
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 6eacb09..e4a1c3a 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -115,7 +115,8 @@ typedef struct
 	/*
 	 * Latch used by startup process to wake up walreceiver after telling it
 	 * where to start streaming (after setting receiveStart and
-	 * receiveStartTLI).
+	 * receiveStartTLI), and then to tell it to send apply feedback to the
+	 * primary whenever specially marked commit records are applied.
 	 */
 	Latch		latch;
 } WalRcvData;
@@ -138,7 +139,7 @@ extern PGDLLIMPORT walrcv_startstreaming_type walrcv_startstreaming;
 typedef void (*walrcv_endstreaming_type) (TimeLineID *next_tli);
 extern PGDLLIMPORT walrcv_endstreaming_type walrcv_endstreaming;
 
-typedef int (*walrcv_receive_type) (int timeout, char **buffer);
+typedef int (*walrcv_receive_type) (char **buffer, int *wait_fd);
 extern PGDLLIMPORT walrcv_receive_type walrcv_receive;
 
 typedef void (*walrcv_send_type) (const char *buffer, int nbytes);
@@ -162,5 +163,6 @@ extern void RequestXLogStreaming(TimeLineID tli, XLogRecPtr recptr,
 extern XLogRecPtr GetWalRcvWriteRecPtr(XLogRecPtr *latestChunkStart, TimeLineID *receiveTLI);
 extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
+extern void WalRcvWakeup(void);
 
 #endif   /* _WALRECEIVER_H */
0002-replay-lag-v8.patchapplication/octet-stream; name=0002-replay-lag-v8.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 105d541..7d63782 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1208,6 +1208,12 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
       standby server</entry>
     </row>
     <row>
+     <entry><structfield>replay_lag</></entry>
+     <entry><type>interval</></entry>
+     <entry>Estimated time taken for recent WAL records to be replayed on this
+      standby server</entry>
+    </row>
+    <row>
      <entry><structfield>sync_priority</></entry>
      <entry><type>integer</></entry>
      <entry>Priority of this standby server for being chosen as the
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index da595fd..b0464e7 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -5474,6 +5474,12 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 		XLogFlush(lsn);
 
 	/*
+	 * Record the primary's timestamp for the commit record, so it can be used
+	 * for tracking replay lag.
+	 */
+	SetXLogReplayTimestamp(parsed->xact_time);
+
+	/*
 	 * If asked by the primary (because someone is waiting for a synchronous
 	 * commit = remote_apply), we will need to ask walreceiver to send a
 	 * reply immediately.
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 3e454f5..504b4df 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -81,6 +81,8 @@ extern uint32 bootstrap_data_checksum_version;
 #define PROMOTE_SIGNAL_FILE		"promote"
 #define FALLBACK_PROMOTE_SIGNAL_FILE "fallback_promote"
 
+/* Size of the circular buffer of timestamped LSNs. */
+#define MAX_TIMESTAMPED_LSNS 8192
 
 /* User-settable parameters */
 int			max_wal_size = 64;	/* 1 GB */
@@ -360,6 +362,13 @@ static bool doRequestWalReceiverReply;
  */
 static XLogRecPtr RedoStartLSN = InvalidXLogRecPtr;
 
+/*
+ * LastReplayedTimestamp can be set by redo handlers when they apply a record
+ * that carries a timestamp, by calling SetXLogReplayedTimestamp.  The xlog
+ * apply loop can then update the value in shared memory.
+ */
+static TimestampTz LastReplayedTimestamp = 0;
+
 /*----------
  * Shared-memory data structures for XLOG control
  *
@@ -634,6 +643,21 @@ typedef struct XLogCtlData
 	/* current effective recovery target timeline */
 	TimeLineID	RecoveryTargetTLI;
 
+	/* timestamp from the most recently applied record associated with a timestamp. */
+	TimestampTz lastReplayedTimestamp;
+
+	/*
+	 * We maintain a circular buffer of LSNs and associated timestamps.
+	 * Walreceiver writes into it using information from timestamps, and the
+	 * startup recovery process reads from it and notifies walreceiver when
+	 * LSNs are replayed so that the timestamps can eventually be fed back to
+	 * the upstream server, to track lag.
+	 */
+	Index			timestampedLsnRead;
+	Index			timestampedLsnWrite;
+	XLogRecPtr		timestampedLsn[MAX_TIMESTAMPED_LSNS];
+	TimestampTz		timestampedLsnTime[MAX_TIMESTAMPED_LSNS];
+
 	/*
 	 * timestamp of when we started replaying the current chunk of WAL data,
 	 * only relevant for replication or archive recovery
@@ -6874,20 +6898,51 @@ StartupXLOG(void)
 				error_context_stack = errcallback.previous;
 
 				/*
-				 * Update lastReplayedEndRecPtr after this record has been
-				 * successfully replayed.
+				 * Update lastReplayedEndRecPtr and lastReplayedTimestamp
+				 * after this record has been successfully replayed.
 				 */
 				SpinLockAcquire(&XLogCtl->info_lck);
 				XLogCtl->lastReplayedEndRecPtr = EndRecPtr;
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
+				if (LastReplayedTimestamp != 0)
+				{
+					/* If replaying a record produced a timestamp, use that. */
+					XLogCtl->lastReplayedTimestamp = LastReplayedTimestamp;
+					LastReplayedTimestamp = 0;
+				}
+				else
+				{
+					/*
+					 * If we have applied LSNs associated with timestamps
+					 * received by walreceiver, then use the recorded
+					 * timestamp.  We consume from the read end of the
+					 * circular buffer.
+					 */
+					while (XLogCtl->timestampedLsnRead !=
+						   XLogCtl->timestampedLsnWrite &&
+						   XLogCtl->timestampedLsn[XLogCtl->timestampedLsnRead]
+						   <= EndRecPtr)
+					{
+						if (XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead] >
+							XLogCtl->lastReplayedTimestamp)
+						{
+							XLogCtl->lastReplayedTimestamp =
+								XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead];
+							doRequestWalReceiverReply = true;
+						}
+						XLogCtl->timestampedLsnRead =
+							(XLogCtl->timestampedLsnRead + 1) % MAX_TIMESTAMPED_LSNS;
+					}
+				}
 				SpinLockRelease(&XLogCtl->info_lck);
 
 				/*
 				 * If rm_redo reported that it applied a commit record that
 				 * the master is waiting for by calling
-				 * XLogRequestWalReceiverReply, then we wake up the receiver
-				 * so that it notices the updated lastReplayedEndRecPtr and
-				 * sends a reply to the master.
+				 * XLogRequestWalReceiverReply, or we encountered a WAL
+				 * location that was associated with a timestamp above, then
+				 * we wake up the receiver so that it notices the updated
+				 * lastReplayedEndRecPtr and sends a reply to the master.
 				 */
 				if (doRequestWalReceiverReply)
 				{
@@ -11619,3 +11674,91 @@ XLogRequestWalReceiverReply(void)
 {
 	doRequestWalReceiverReply = true;
 }
+
+/*
+ * Record the timestamp that is associated with a WAL position.
+ *
+ * This is called by walreceiver on standby servers when keepalive messages
+ * arrive, using timestamps generated on the primary server.  The timestamp
+ * will be sent back to the primary server when the standby had applied this
+ * WAL position.  The primary can use the elapsed time to estimate the replay
+ * lag.
+ */
+void
+SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn)
+{
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn == XLogCtl->lastReplayedEndRecPtr)
+	{
+		/*
+		 * That is the last replayed LSN: we are fully replayed, so we can
+		 * update the replay timestamp immediately.
+		 */
+		XLogCtl->lastReplayedTimestamp = timestamp;
+	}
+	else
+	{
+		/*
+		 * There is WAL still to be applied.  We will associate the timestamp
+		 * with this WAL position and wait for it to be replayed.  We add it
+		 * at the 'write' end of the circular buffer of LSN/timestamp
+		 * mappings, which the replay loop will eventually read.
+		 */
+		Index w = XLogCtl->timestampedLsnWrite;
+		Index r = XLogCtl->timestampedLsnRead;
+
+		XLogCtl->timestampedLsn[w] = lsn;
+		XLogCtl->timestampedLsnTime[w] = timestamp;
+
+		/* Advance the write point. */
+		w = (w + 1) % MAX_TIMESTAMPED_LSNS;
+		XLogCtl->timestampedLsnWrite = w;
+		if (w == r)
+		{
+			/*
+			 * The buffer is full.  Advance the read point (throwing away
+			 * oldest values; we will begin to overestimate replay lag, until
+			 * lag decreases to a size our buffer can manage, or the next
+			 * commit record is replayed).
+			 */
+			r = (r + 1) % MAX_TIMESTAMPED_LSNS;
+			XLogCtl->timestampedLsnRead = r;
+		}
+	}
+	SpinLockRelease(&XLogCtl->info_lck);
+}
+
+/*
+ * Set the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary.  This can be called by redo handlers that have
+ * an appropriate timestamp (currently only commit records).  Updating the
+ * shared memory value is deferred until after the redo handler returns.
+ */
+void
+SetXLogReplayTimestamp(TimestampTz timestamp)
+{
+	LastReplayedTimestamp = timestamp;
+}
+
+/*
+ * Get the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary, and also the most recently applied LSN.  (Note
+ * that the timestamp and the LSN don't necessarily relate to the same
+ * record.)
+ *
+ * This is similar to GetLatestXTime, except that it is not only advanced by
+ * commit records (see SetXLogReplayTimestampAtLsn).
+ */
+TimestampTz
+GetXLogReplayTimestamp(XLogRecPtr *lsn)
+{
+	TimestampTz result;
+
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn)
+		*lsn = XLogCtl->lastReplayedEndRecPtr;
+	result = XLogCtl->lastReplayedTimestamp;
+	SpinLockRelease(&XLogCtl->info_lck);
+
+	return result;
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 9ae1ef4..a53f07b 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -662,6 +662,7 @@ CREATE VIEW pg_stat_replication AS
             W.write_location,
             W.flush_location,
             W.replay_location,
+            W.replay_lag,
             W.sync_priority,
             W.sync_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index e6543e1..fd7aecb 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -85,6 +85,8 @@ walrcv_disconnect_type walrcv_disconnect = NULL;
 
 #define NAPTIME_PER_CYCLE 100	/* max sleep time between cycles (100ms) */
 
+#define MIN_TIME_BETWEEN_TIMESTAMPED_LSNS 1000 /* 1s */
+
 /*
  * These variables are used similarly to openLogFile/SegNo/Off,
  * but for walreceiver to write the XLOG. recvFileTLI is the TimeLineID
@@ -102,6 +104,8 @@ static uint32 recvOff = 0;
 static volatile sig_atomic_t got_SIGHUP = false;
 static volatile sig_atomic_t got_SIGTERM = false;
 
+static bool recovery_active = false;
+
 /*
  * LogstreamResult indicates the byte positions that we have already
  * written/fsynced.
@@ -143,7 +147,7 @@ static void WalRcvDie(int code, Datum arg);
 static void XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len);
 static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
-static void XLogWalRcvSendReply(bool force, bool requestReply);
+static void XLogWalRcvSendReply(bool force, bool requestReply, bool includeApplyTimestamp);
 static void XLogWalRcvSendHSFeedback(bool immed);
 static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
 
@@ -445,7 +449,7 @@ WalReceiverMain(void)
 					}
 
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(false, false);
+					XLogWalRcvSendReply(false, false, false);
 
 					/*
 					 * If we've written some records, flush them to disk and
@@ -476,7 +480,8 @@ WalReceiverMain(void)
 				{
 					/* The recovery process has asked us to send apply feedback now. */
 					ResetLatch(&walrcv->latch);
-					XLogWalRcvSendReply(true, false);
+					XLogWalRcvSendReply(true, false, true);
+					recovery_active = true;
 				}
 				if (rc & WL_TIMEOUT)
 				{
@@ -524,7 +529,7 @@ WalReceiverMain(void)
 						}
 					}
 
-					XLogWalRcvSendReply(requestReply, requestReply);
+					XLogWalRcvSendReply(requestReply, requestReply, false);
 					XLogWalRcvSendHSFeedback(false);
 				}
 			}
@@ -855,6 +860,8 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 			}
 		case 'k':				/* Keepalive */
 			{
+				bool reportApplyTimestamp = false;
+
 				/* copy message to StringInfo */
 				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
 				if (len != hdrlen)
@@ -871,9 +878,22 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 
 				ProcessWalSndrMessage(walEnd, sendTime);
 
+				/*
+				 * If no apply timestamps have been sent at the request of the
+				 * recovery process since we last received a keepalive, then
+				 * we will send one now.  This allows us to feed back
+				 * timestamps in response to pings if we are idle or if the
+				 * recovery process is somehow blocked, but we don't want to
+				 * do that if it's actively applying and periodically waking
+				 * us up with accurate apply timestamps.
+				 */
+				if (!recovery_active)
+					reportApplyTimestamp = true;
+				recovery_active = false;
+
 				/* If the primary requested a reply, send one immediately */
-				if (replyRequested)
-					XLogWalRcvSendReply(true, false);
+				if (replyRequested || reportApplyTimestamp)
+					XLogWalRcvSendReply(true, false, reportApplyTimestamp);
 				break;
 			}
 		default:
@@ -1036,7 +1056,7 @@ XLogWalRcvFlush(bool dying)
 		/* Also let the master know that we made some progress */
 		if (!dying)
 		{
-			XLogWalRcvSendReply(false, false);
+			XLogWalRcvSendReply(false, false, false);
 			XLogWalRcvSendHSFeedback(false);
 		}
 	}
@@ -1054,15 +1074,18 @@ XLogWalRcvFlush(bool dying)
  * If 'requestReply' is true, requests the server to reply immediately upon
  * receiving this message. This is used for heartbearts, when approaching
  * wal_receiver_timeout.
+ *
+ * If 'reportApplyTimestamp' is true, the latest apply timestamp is included.
  */
 static void
-XLogWalRcvSendReply(bool force, bool requestReply)
+XLogWalRcvSendReply(bool force, bool requestReply, bool reportApplyTimestamp)
 {
 	static XLogRecPtr writePtr = 0;
 	static XLogRecPtr flushPtr = 0;
 	XLogRecPtr	applyPtr;
 	static TimestampTz sendTime = 0;
 	TimestampTz now;
+	TimestampTz applyTimestamp = 0;
 
 	/*
 	 * If the user doesn't want status to be reported to the master, be sure
@@ -1078,10 +1101,8 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	 * We can compare the write and flush positions to the last message we
 	 * sent without taking any lock, but the apply position requires a spin
 	 * lock, so we don't check that unless something else has changed or 10
-	 * seconds have passed.  This means that the apply log position will
-	 * appear, from the master's point of view, to lag slightly, but since
-	 * this is only for reporting purposes and only on idle systems, that's
-	 * probably OK.
+	 * seconds have passed, or the force flag has been set (which happens when
+	 * apply feedback has been requested by the primary).
 	 */
 	if (!force
 		&& writePtr == LogstreamResult.Write
@@ -1094,7 +1115,10 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	/* Construct a new message */
 	writePtr = LogstreamResult.Write;
 	flushPtr = LogstreamResult.Flush;
-	applyPtr = GetXLogReplayRecPtr(NULL);
+	if (reportApplyTimestamp)
+		applyTimestamp = GetXLogReplayTimestamp(&applyPtr);
+	else
+		applyPtr = GetXLogReplayRecPtr(NULL);
 
 	resetStringInfo(&reply_message);
 	pq_sendbyte(&reply_message, 'r');
@@ -1102,6 +1126,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	pq_sendint64(&reply_message, flushPtr);
 	pq_sendint64(&reply_message, applyPtr);
 	pq_sendint64(&reply_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&reply_message, TimestampTzToIntegerTimestamp(applyTimestamp));
 	pq_sendbyte(&reply_message, requestReply ? 1 : 0);
 
 	/* Send it */
@@ -1206,8 +1231,8 @@ static void
 ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 {
 	WalRcvData *walrcv = WalRcv;
-
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
+	static TimestampTz lastRecordedTimestamp = 0;
 
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
@@ -1218,6 +1243,18 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
 	SpinLockRelease(&walrcv->mutex);
 
+	/*
+	 * Remember primary's timestamp at this WAL location.  We throw away
+	 * samples if they are coming too fast because we don't want to fill up
+	 * the finite circular buffer and have to throw away older samples.
+	 */
+	if (lastRecordedTimestamp < TimestampTzPlusMilliseconds(sendTime,
+															-MIN_TIME_BETWEEN_TIMESTAMPED_LSNS))
+	{
+		SetXLogReplayTimestampAtLsn(sendTime, walEnd);
+		lastRecordedTimestamp = sendTime;
+	}
+
 	if (log_min_messages <= DEBUG2)
 	{
 		char	   *sendtime;
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index f98475c..16d7abc 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1545,15 +1545,29 @@ ProcessStandbyReplyMessage(void)
 	XLogRecPtr	writePtr,
 				flushPtr,
 				applyPtr;
+	int64		applyLagUs;
 	bool		replyRequested;
+	TimestampTz now = GetCurrentTimestamp();
+	TimestampTz applyTimestamp;
 
 	/* the caller already consumed the msgtype byte */
 	writePtr = pq_getmsgint64(&reply_message);
 	flushPtr = pq_getmsgint64(&reply_message);
 	applyPtr = pq_getmsgint64(&reply_message);
 	(void) pq_getmsgint64(&reply_message);		/* sendTime; not used ATM */
+	applyTimestamp = IntegerTimestampToTimestampTz(pq_getmsgint64(&reply_message));
 	replyRequested = pq_getmsgbyte(&reply_message);
 
+	/* Compute the apply lag in milliseconds. */
+	if (applyTimestamp == 0)
+		applyLagUs = -1;
+	else
+#ifdef HAVE_INT64_TIMESTAMP
+		applyLagUs = now - applyTimestamp;
+#else
+		applyLagUs = (now - applyTimestamp) * 1000000;
+#endif
+
 	elog(DEBUG2, "write %X/%X flush %X/%X apply %X/%X%s",
 		 (uint32) (writePtr >> 32), (uint32) writePtr,
 		 (uint32) (flushPtr >> 32), (uint32) flushPtr,
@@ -1575,6 +1589,8 @@ ProcessStandbyReplyMessage(void)
 		walsnd->write = writePtr;
 		walsnd->flush = flushPtr;
 		walsnd->apply = applyPtr;
+		if (applyLagUs >= 0)
+			walsnd->applyLagUs = applyLagUs;
 		SpinLockRelease(&walsnd->mutex);
 	}
 
@@ -2745,7 +2761,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	8
+#define PG_STAT_GET_WAL_SENDERS_COLS	9
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2793,6 +2809,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		XLogRecPtr	write;
 		XLogRecPtr	flush;
 		XLogRecPtr	apply;
+		int64		applyLagUs;
 		int			priority;
 		WalSndState state;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
@@ -2807,6 +2824,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
+		applyLagUs = walsnd->applyLagUs;
 		priority = walsnd->sync_standby_priority;
 		SpinLockRelease(&walsnd->mutex);
 
@@ -2841,6 +2859,23 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				nulls[5] = true;
 			values[5] = LSNGetDatum(apply);
 
+			if (applyLagUs < 0)
+				nulls[6] = true;
+			else
+			{
+				Interval *applyLagInterval = palloc(sizeof(Interval));
+
+				applyLagInterval->month = 0;
+				applyLagInterval->day = 0;
+#ifdef HAVE_INT64_TIMESTAMP
+				applyLagInterval->time = applyLagUs;
+#else
+				applyLagInterval->time = applyLagUs / 1000000.0;
+#endif
+				nulls[6] = false;
+				values[6] = IntervalPGetDatum(applyLagInterval);
+			}
+
 			/*
 			 * Treat a standby such as a pg_basebackup background process
 			 * which always returns an invalid flush location, as an
@@ -2848,18 +2883,18 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 			 */
 			priority = XLogRecPtrIsInvalid(walsnd->flush) ? 0 : priority;
 
-			values[6] = Int32GetDatum(priority);
+			values[7] = Int32GetDatum(priority);
 
 			/*
 			 * More easily understood version of standby state. This is purely
 			 * informational, not different from priority.
 			 */
 			if (priority == 0)
-				values[7] = CStringGetTextDatum("async");
+				values[8] = CStringGetTextDatum("async");
 			else if (walsnd == sync_standby)
-				values[7] = CStringGetTextDatum("sync");
+				values[8] = CStringGetTextDatum("sync");
 			else
-				values[7] = CStringGetTextDatum("potential");
+				values[8] = CStringGetTextDatum("potential");
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
diff --git a/src/backend/utils/adt/timestamp.c b/src/backend/utils/adt/timestamp.c
index c9e5270..f382b20 100644
--- a/src/backend/utils/adt/timestamp.c
+++ b/src/backend/utils/adt/timestamp.c
@@ -1629,6 +1629,20 @@ IntegerTimestampToTimestampTz(int64 timestamp)
 #endif
 
 /*
+ * TimestampTzToIntegerTimestamp -- convert a native timestamp to int64 format
+ *
+ * When compiled with --enable-integer-datetimes, this is implemented as a
+ * no-op macro.
+ */
+#ifndef HAVE_INT64_TIMESTAMP
+int64
+TimestampTzToIntegerTimestamp(TimestampTz timestamp)
+{
+	return timestamp * 1000000;
+}
+#endif
+
+/*
  * TimestampDifference -- convert the difference between two timestamps
  *		into integer seconds and microseconds
  *
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index a7dcdae..c8be3ce 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -235,6 +235,9 @@ extern void GetXLogReceiptTime(TimestampTz *rtime, bool *fromStream);
 extern XLogRecPtr GetXLogReplayRecPtr(TimeLineID *replayTLI);
 extern XLogRecPtr GetXLogInsertRecPtr(void);
 extern XLogRecPtr GetXLogWriteRecPtr(void);
+extern void SetXLogReplayTimestamp(TimestampTz timestamp);
+extern void SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn);
+extern TimestampTz GetXLogReplayTimestamp(XLogRecPtr *lsn);
 extern bool RecoveryIsPaused(void);
 extern void SetRecoveryPause(bool recoveryPause);
 extern TimestampTz GetLatestXTime(void);
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index a595327..4054726 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2712,7 +2712,7 @@ DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f
 DESCR("statistics: information about currently active backends");
 DATA(insert OID = 3318 (  pg_stat_get_progress_info           PGNSP PGUID 12 1 100 0 0 f f f f t t s r 1 0 2249 "25" "{25,23,26,26,20,20,20,20,20,20,20,20,20,20}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{cmdtype,pid,datid,relid,param1,param2,param3,param4,param5,param6,param7,param8,param9,param10}" _null_ _null_ pg_stat_get_progress_info _null_ _null_ _null_ ));
 DESCR("statistics: information about progress of backends running maintenance command");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,23,25}" "{o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25}" "{o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 7794aa5..4de43e8 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -46,6 +46,7 @@ typedef struct WalSnd
 	XLogRecPtr	write;
 	XLogRecPtr	flush;
 	XLogRecPtr	apply;
+	int64		applyLagUs;
 
 	/* Protects shared variables shown above. */
 	slock_t		mutex;
diff --git a/src/include/utils/timestamp.h b/src/include/utils/timestamp.h
index fbead3a..297e151 100644
--- a/src/include/utils/timestamp.h
+++ b/src/include/utils/timestamp.h
@@ -227,9 +227,11 @@ extern bool TimestampDifferenceExceeds(TimestampTz start_time,
 #ifndef HAVE_INT64_TIMESTAMP
 extern int64 GetCurrentIntegerTimestamp(void);
 extern TimestampTz IntegerTimestampToTimestampTz(int64 timestamp);
+extern int64 TimestampTzToIntegerTimestamp(TimestampTz timestamp);
 #else
 #define GetCurrentIntegerTimestamp()	GetCurrentTimestamp()
 #define IntegerTimestampToTimestampTz(timestamp) (timestamp)
+#define TimestampTzToIntegerTimestamp(timestamp) (timestamp)
 #endif
 
 extern TimestampTz time_t_to_timestamptz(pg_time_t tm);
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 79f9b23..fc4b765 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1783,11 +1783,12 @@ pg_stat_replication| SELECT s.pid,
     w.write_location,
     w.flush_location,
     w.replay_location,
+    w.replay_lag,
     w.sync_priority,
     w.sync_state
    FROM pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, wait_event_type, wait_event, xact_start, query_start, backend_start, state_change, client_addr, client_hostname, client_port, backend_xid, backend_xmin, ssl, sslversion, sslcipher, sslbits, sslcompression, sslclientdn),
     pg_authid u,
-    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, sync_priority, sync_state)
+    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state)
   WHERE ((s.usesysid = u.oid) AND (s.pid = w.pid));
 pg_stat_ssl| SELECT s.pid,
     s.ssl,
0003-refactor-syncrep-exit-v8.patchapplication/octet-stream; name=0003-refactor-syncrep-exit-v8.patchDownload
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 1ee1bc5..376ddf4 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -83,6 +83,64 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * ===========================================================
  */
 
+static bool
+SyncRepCheckEarlyExit(void)
+{
+	/*
+	 * If a wait for synchronous replication is pending, we can neither
+	 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
+	 * lead the client to believe that the transaction aborted, which
+	 * is not true: it's already committed locally. The former is no good
+	 * either: the client has requested synchronous replication, and is
+	 * entitled to assume that an acknowledged commit is also replicated,
+	 * which might not be true. So in this case we issue a WARNING (which
+	 * some clients may be able to interpret) and shut off further output.
+	 * We do NOT reset ProcDiePending, so that the process will die after
+	 * the commit is cleaned up.
+	 */
+	if (ProcDiePending)
+	{
+		ereport(WARNING,
+				(errcode(ERRCODE_ADMIN_SHUTDOWN),
+				 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * It's unclear what to do if a query cancel interrupt arrives.  We
+	 * can't actually abort at this point, but ignoring the interrupt
+	 * altogether is not helpful, so we just terminate the wait with a
+	 * suitable warning.
+	 */
+	if (QueryCancelPending)
+	{
+		QueryCancelPending = false;
+		ereport(WARNING,
+				(errmsg("canceling wait for synchronous replication due to user request"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * If the postmaster dies, we'll probably never get an
+	 * acknowledgement, because all the wal sender processes will exit. So
+	 * just bail out.
+	 */
+	if (!PostmasterIsAlive())
+	{
+		ProcDiePending = true;
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	return false;
+}
+
 /*
  * Wait for synchronous replication, if requested by user.
  *
@@ -192,57 +250,9 @@ SyncRepWaitForLSN(XLogRecPtr lsn, bool commit)
 		if (syncRepState == SYNC_REP_WAIT_COMPLETE)
 			break;
 
-		/*
-		 * If a wait for synchronous replication is pending, we can neither
-		 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
-		 * lead the client to believe that the transaction aborted, which
-		 * is not true: it's already committed locally. The former is no good
-		 * either: the client has requested synchronous replication, and is
-		 * entitled to assume that an acknowledged commit is also replicated,
-		 * which might not be true. So in this case we issue a WARNING (which
-		 * some clients may be able to interpret) and shut off further output.
-		 * We do NOT reset ProcDiePending, so that the process will die after
-		 * the commit is cleaned up.
-		 */
-		if (ProcDiePending)
-		{
-			ereport(WARNING,
-					(errcode(ERRCODE_ADMIN_SHUTDOWN),
-					 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit())
 			break;
-		}
-
-		/*
-		 * It's unclear what to do if a query cancel interrupt arrives.  We
-		 * can't actually abort at this point, but ignoring the interrupt
-		 * altogether is not helpful, so we just terminate the wait with a
-		 * suitable warning.
-		 */
-		if (QueryCancelPending)
-		{
-			QueryCancelPending = false;
-			ereport(WARNING,
-					(errmsg("canceling wait for synchronous replication due to user request"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * If the postmaster dies, we'll probably never get an
-		 * acknowledgement, because all the wal sender processes will exit. So
-		 * just bail out.
-		 */
-		if (!PostmasterIsAlive())
-		{
-			ProcDiePending = true;
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
-			break;
-		}
 
 		/*
 		 * Wait on latch.  Any condition that should wake us up will set the
0004-causal-reads-v8.patchapplication/octet-stream; name=0004-causal-reads-v8.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 6ab65a7..ed6f07c 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -2749,6 +2749,35 @@ include_dir 'conf.d'
      across the cluster without problems if that is required.
     </para>
 
+    <sect2 id="runtime-config-replication-all">
+     <title>All Servers</title>
+     <para>
+      These parameters can be set on the primary or any standby.
+     </para>
+     <variablelist>
+      <varlistentry id="guc-causal-reads" xreflabel="causal_reads">
+       <term><varname>causal_reads</varname> (<type>boolean</type>)
+       <indexterm>
+        <primary><varname>causal_reads</> configuration parameter</primary>
+       </indexterm>
+       </term>
+       <listitem>
+        <para>
+         Enables causal consistency between transactions run on different
+         servers.  A transaction that is run on a standby
+         with <varname>causal_reads</> set to <literal>on</> is guaranteed
+         either to see the effects of all completed transactions run on the
+         primary with the setting on, or to receive an error "standby is not
+         available for causal reads".  Note that both transactions involved in
+         a causal dependency (a write on the primary followed by a read on any
+         server which must see the write) must be run with the setting on.
+         See <xref linkend="causal-reads"> for more details.
+        </para>
+       </listitem>
+      </varlistentry>
+     </variablelist>     
+    </sect2>
+
     <sect2 id="runtime-config-replication-sender">
      <title>Sending Server(s)</title>
 
@@ -2980,6 +3009,48 @@ include_dir 'conf.d'
       </listitem>
      </varlistentry>
 
+     <varlistentry>
+      <term><varname>causal_reads_timeout</varname> (<type>integer</type>)
+       <indexterm>
+        <primary><varname>causal_reads_timeout</> configuration parameter</primary>
+       </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies the maximum replay lag the primary will tolerate from a
+        standby before dropping it from the set of standbys available for
+        causal reads.
+       </para>
+       <para>
+        This setting is also used to control the <firstterm>leases</> used to
+        maintain the causal reads guarantee.  It must be set to a value which
+        is at least 4 times the maximum possible difference in system clocks
+        between the primary and standby servers, as described
+        in <xref linkend="causal-reads">.
+       </para>
+      </listitem>
+     </varlistentry>
+
+     <varlistentry id="guc-causal-reads-standby-names" xreflabel="causal-reads-standby-names">
+      <term><varname>causal_reads_standby_names</varname> (<type>string</type>)
+      <indexterm>
+       <primary><varname>causal_reads_standby_names</> configuration parameter</primary>
+      </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies a comma-separated list of standby names that can support
+        <firstterm>causal reads</>, as described in
+        <xref linkend="causal-reads">.  Follows the same convention
+        as <link linkend="guc-synchronous-standby-names"><literal>synchronous_standby_name</></>.
+        The default is <literal>*</>, matching all standbys.
+       </para>
+       <para>
+        This setting has no effect if <varname>causal_reads_timeout</> is not set.
+       </para>
+      </listitem>
+     </varlistentry>
+
      </variablelist>
     </sect2>
 
diff --git a/doc/src/sgml/high-availability.sgml b/doc/src/sgml/high-availability.sgml
index 03c6c30..7a0910d 100644
--- a/doc/src/sgml/high-availability.sgml
+++ b/doc/src/sgml/high-availability.sgml
@@ -1115,6 +1115,9 @@ primary_slot_name = 'node_a_slot'
     that it has replayed the transaction, making it visible to user queries.
     In simple cases, this allows for load balancing with causal consistency
     on a single hot standby.
+    (See also
+    <xref linkend="causal-reads"> which deals with multiple standbys and
+    standby failure.)
    </para>
 
    <para>
@@ -1233,6 +1236,119 @@ primary_slot_name = 'node_a_slot'
    </sect3>
   </sect2>
 
+  <sect2 id="causal-reads">
+   <title>Causal reads</title>
+   <indexterm>
+    <primary>causal reads</primary>
+    <secondary>in standby</secondary>
+   </indexterm>
+
+   <para>
+    The causal reads feature allows read-only queries to run on hot standby
+    servers without exposing stale data to the client, providing a form of
+    causal consistency.  Transactions can run on any standby with the
+    following guarantee about the visibility of preceding transactions: If you
+    set <varname>causal_reads</> to <literal>on</> in any pair of consecutive
+    transactions tx1, tx2 where tx2 begins after tx1 successfully returns,
+    then tx2 will either see tx1 or fail with a new error "standby is not
+    available for causal reads", no matter which server it runs on.  Although
+    the guarantee is expressed in terms of two individual transactions, the
+    GUC can also be set at session, role or system level to make the guarantee
+    generally, allowing for load balancing of applications that were not
+    designed with load balancing in mind.
+   </para>
+
+   <para>
+    In order to enable the feature, <varname>causal_reads_timeout</> must be
+    set to a non-zero value on the primary server.  The
+    GUC <varname>causal_reads_standby_names</> can be used to limit the set of
+    standbys that can join the dynamic set of causal reads standbys by
+    providing a comma-separated list of application names.  By default, all
+    standbys are candidates, if the feature is enabled.
+   </para>
+
+   <para>
+    The current set of servers that the primary considers to be available for
+    causal reads can be seen in
+    the <link linkend="monitoring-stats-views-table"> <literal>pg_stat_replication</></>
+    view.  Administrators, applications and load balancing middleware can use
+    this view to discover standbys that can currently handle causal reads
+    transactions without raising the error.  Since that information is only an
+    instantantaneous snapshot, clients should still be prepared for the error
+    to be raised at any time, and consider redirecting transactions to another
+    standby.
+   </para>
+
+   <para>
+    The advantages of the causal reads feature over simply
+    setting <varname>synchronous_commit</> to <literal>remote_apply</> are:
+    <orderedlist>
+      <listitem>
+       <para>
+        It allows the primary to wait for multiple standbys to replay
+        transactions.
+       </para>
+      </listitem>
+      <listitem>
+       <para>
+        It places a configurable limit on how much replay lag (and therefore
+        delay at commit time) the primary tolerates from standbys before it
+        drops them from the dynamic set of standbys it waits for.
+       </para>   
+      </listitem>
+      <listitem>
+       <para>
+        It upholds the causal reads guarantee during the transitions that
+        occur when new standbys are added or removed from the set of standbys,
+        including scenarios where contact has been lost between the primary
+        and standbys but the standby is still alive and running client
+        queries.
+       </para>
+      </listitem>
+    </orderedlist>
+   </para>
+
+   <para>
+    The protocol used to uphold the guarantee even in the case of network
+    failure depends on the system clocks of the primary and standby servers
+    being synchronized, with an allowance for a difference up to one quarter
+    of <varname>causal_reads_timeout</>.  For example,
+    if <varname>causal_reads_timeout</> is set to <literal>4s</>, then the
+    clocks must not be further than 1 second apart for the guarantee to be
+    upheld reliably during transitions.  The ubiquity of the Network Time
+    Protocol (NTP) on modern operating systems and availability of high
+    quality time servers makes it possible to choose a tolerance significantly
+    higher than the maximum expected clock difference.  An effort is
+    nevertheless made to detect and report misconfigured and faulty systems
+    with clock differences greater than the configured tolerance.
+   </para>
+
+   <note>
+    <para>
+     Current hardware clocks, NTP implementations and public time servers are
+     unlikely to allow the system clocks to differ more than tens or hundreds
+     of milliseconds, and systems synchronized with dedicated local time
+     servers may be considerably more accurate, but you should only consider
+     setting <varname>causal_reads_timeout</> below 4 seconds (allowing up to
+     1 second of clock difference) after researching your time synchronization
+     infrastructure thoroughly.
+    </para>  
+   </note>
+
+   <note>
+    <para>
+      While similar to synchronous replication in the sense that both involve
+      the primary server waiting for responses from standby servers, the
+      causal reads feature is not concerned with avoiding data loss.  A
+      primary configured for causal reads will drop all standbys that stop
+      responding or replay too slowly from the dynamic set that it waits for,
+      so you should consider configuring both synchronous replication and
+      causal reads if you need data loss avoidance guarantees and causal
+      consistency guarantees for load balancing.
+    </para>
+   </note>
+  </sect2>
+
   <sect2 id="continuous-archiving-in-standby">
    <title>Continuous archiving in standby</title>
 
@@ -1581,7 +1697,16 @@ if (!triggered)
     so there will be a measurable delay between primary and standby. Running the
     same query nearly simultaneously on both primary and standby might therefore
     return differing results. We say that data on the standby is
-    <firstterm>eventually consistent</firstterm> with the primary.  Once the
+    <firstterm>eventually consistent</firstterm> with the primary by default.
+    The data visible to a transaction running on a standby can be
+    made <firstterm>causally consistent</> with respect to a transaction that
+    has completed on the primary by setting <varname>causal_reads</>
+    to <literal>on</> in both transactions.  For more details,
+    see <xref linkend="causal-reads">.
+   </para>
+
+   <para>
+    Once the    
     commit record for a transaction is replayed on the standby, the changes
     made by that transaction will be visible to any new snapshots taken on
     the standby.  Snapshots may be taken at the start of each query or at the
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 7d63782..23d68d5 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1224,6 +1224,17 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
      <entry><type>text</></entry>
      <entry>Synchronous state of this standby server</entry>
     </row>
+    <row>
+     <entry><structfield>causal_reads_state</></entry>
+     <entry><type>text</></entry>
+     <entry>Causal reads state of this standby server.  This field will be
+     non-null only if <varname>cause_reads_timeout</> is set.  If a standby is
+     in <literal>available</> state, then it can currently serve causal reads
+     queries.  If it is not replaying fast enough or not responding to
+     keepalive messages, it will be in <literal>unavailable</> state, and if
+     it is currently transitioning to availability it will be
+     in <literal>joining</> state for a short time.</entry>
+    </row>
    </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 893c2fa..111198a 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -2098,11 +2098,12 @@ RecordTransactionCommitPrepared(TransactionId xid,
 	END_CRIT_SECTION();
 
 	/*
-	 * Wait for synchronous replication, if required.
+	 * Wait for causal reads and synchronous replication, if required.
 	 *
 	 * Note that at this stage we have marked clog, but still show as running
 	 * in the procarray and continue to hold locks.
 	 */
+	CausalReadsWaitForLSN(recptr);
 	SyncRepWaitForLSN(recptr, true);
 }
 
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index b0464e7..5ad3646 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,10 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
+	{
+		CausalReadsWaitForLSN(XactLastRecEnd);
 		SyncRepWaitForLSN(XactLastRecEnd, true);
+	}
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5126,7 +5129,7 @@ XactLogCommitRecord(TimestampTz commit_time,
 	 * Check if the caller would like to ask standbys for immediate feedback
 	 * once this commit is applied.
 	 */
-	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY || causal_reads)
 		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
 
 	/*
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index a53f07b..276ac12 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -664,7 +664,8 @@ CREATE VIEW pg_stat_replication AS
             W.replay_location,
             W.replay_lag,
             W.sync_priority,
-            W.sync_state
+            W.sync_state,
+            W.causal_reads_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
             pg_stat_get_wal_senders() AS W
     WHERE S.usesysid = U.oid AND
diff --git a/src/backend/replication/README.causal_reads b/src/backend/replication/README.causal_reads
new file mode 100644
index 0000000..1fddd62
--- /dev/null
+++ b/src/backend/replication/README.causal_reads
@@ -0,0 +1,193 @@
+The causal reads guarantee says: If you run any two consecutive
+transactions tx1, tx2 where tx1 completes before tx2 begins, with
+causal_reads set to "on" in both transactions, tx2 will see tx1 or
+raise an error to complain that it can't guarantee causal consistency,
+no matter which servers (primary or any standby) you run each
+transaction on.
+
+When both transactions run on the primary, the guarantee is trivially
+upheld.
+
+To deal with read-only physical streaming standbys, the primary keeps
+track of a set of standbys that it considers to be currently
+"available" for causal reads, and sends a stream of "leases" to those
+standbys granting them the right to handle causal reads transactions
+for a short time without any further communication with the primary.
+
+In general, the primary provides the guarantee by waiting for all of
+the "available" standbys to report that they have applied a
+transaction.  However, the set of available standbys is dynamic, and
+things get more complicated during state transitions.  There are two
+types of transitions to consider:
+
+1.  unavailable->joining->available
+
+Standbys start out as "unavailable".  If a standby is unavailable and
+is applying fast enough and matches causal_reads_standby_names, the
+primary transitions it to "available", but first it sets it to
+"joining" until it is sure that any transaction committed while it was
+unavailable has definitely been applied on the standby.  This closes a
+race that would otherwise exist if we moved directly to available
+state: tx1 might not wait for a given standby because it's
+unavailable, then a lease might be granted, and then tx2 might run a
+causal reads transaction without error but see stale data.  The
+joining state acts as an airlock: while in joining state, the primary
+waits for that standby to replay causal reads transactions in
+anticipation of the move to available, but it doesn't progress to
+available state and grant a lease to the standby until everything
+preceding joining state has also been applied.
+
+2.  available->unavailable
+
+If a standby is not applying fast enough or not responding to
+keepalive messages, then the primary kicks that standby out of the
+dynamic set of available standbys, that is, marks it as "unavailable".
+In order to make sure that the standby has started rejecting causal
+reads transactions, it needs to revoke the lease it most recently
+granted.  It does that by waiting for the lease to expire before
+allowing any causal reads commits to return.  (In future there could
+be a fast-path revocation message which waits for a serial-numbered
+acknowledgement to reduce waiting in the case where the standby is
+lagging but still reachable and responding).
+
+The rest of this document illustrates how clock skew affects the
+available->unavailable transition.
+
+The following 4 variables are derived from a single GUC, and these
+values will be used in the following illustrations:
+
+causal_reads_timeout = 4s
+lease_time           = 4s (= causal_reads_timeout)
+keepalive_time       = 2s (= lease_time / 2)
+max_clock_skew       = 1s (= lease_time / 4)
+
+Every keepalive_time, the primary transmits a lease that expires at
+local_clock_time + lease_time - max_clock_skew, shown in the following
+diagram as 't' for transmission time and '|' for expiry time.  If
+contact is lost with a standby, the primary will wait until sent_time
++ lease_time for the most recently granted lease to expire, shown on
+the following diagram 'x', to be sure that the standby's clock has
+reached the expiry time even if its clock differs by up to
+max_clock_skew.  In other words, the primary tells the standby that
+the expiry time is at one time, but it trusts that the standby will
+surely agree if it gives it some extra time.  The extra time is
+max_clock_skew.  If the clocks differ by more than max_clock_skew, all
+bets are off (but see below for attempt to detect obvious cases).
+
+0     1     2     3     4     5     6     7     8     9
+t-----------------|-----x
+            t-----------------|-----x
+                        t-----------------|-----x
+                                    t-----------------|...
+                                                t------...
+
+A standby whose clock is 2 seconds ahead of the primary's clock
+perceives gaps in the stream of leases, and will reject causal_reads
+transactions in those intervals.  The causal reads guarantee is
+upheld, but spurious errors are raised between leases, as a
+consequence of the clock skew being greater than max_clock_skew.  In
+the following diagram 'r' shows reception time, and the timeline along
+the top shows the standby's local clock time.
+
+2     3     4     5     6     7     8     9    10    11
+r-----|
+            r-----|
+                        r-----|
+                                    r-----|
+                                                r-----|
+
+If there were no network latency, a standby whose clock is exactly 1
+second ahead of the primary's clock would perceive the stream of
+leases as being replaced just in time, so there is no gap.  Since in
+reality the time of receipt is some time after the time of
+transmission due to network latency, if the standby's clock is exactly
+1 second behind, then there will be small network-latency-sized gaps
+before the next lease arrives, but still no correctness problem with
+respect to the causal reads guarantee.
+
+1     2     3     4     5     6     7     8     9    10
+r-----------|
+            r-----------|
+                        r-----------|
+                                    r-----------|
+                                                r------...
+
+A standby whose clock is perfectly in sync with the primary's
+perceives the stream of leases overlapping (this matches the primary's
+perception of the leases it sent):
+
+0     1     2     3     4     5     6     7     8     9
+r-----------------|
+            r-----------------|
+                        r-----------------|
+                                    r-----------------|
+                                                r------...
+
+A standby whose clock is exactly 1 second behind the primary's
+perceives the stream of leases as overlapping even more, but the time
+of expiry as judged by the standby is no later than the time the
+primary will wait for if required ('x').  That is, if contact is lost
+with the standby, the primary can still reliably hold up causal reads
+commits until the standby has started raising the error in
+causal_reads transactions.
+
+-1    0     1     2     3     4     5     6     7     8
+r-----------------------|
+            r-----------------------|
+                        r-----------------------|
+                                    r------------------...
+                                                r------...
+
+
+A standby whose clock is 2 seconds behind the primary's would perceive
+the stream of leases overlapping even more, and the primary would no
+longer be able to wait for a lease to expire if it wanted to revoke
+it.  But because the expiry time is after local_clock_time +
+lease_time, the standby can immediately see that its own clock must be
+more than 1 second behind the primary's, so it ignores the lease and
+logs a clock skew warning.  In the following diagram a lease expiry
+time that is obviously generated by a primary with a clock set too far
+in the future compared to the local clock is shown with a '!'.
+
+-2    -1    0     1     2     3     4     5     6     7
+r-----------------------------!
+            r-----------------------------!
+                        r-----------------------------!
+                                    r------------------...
+                                                r------...
+
+A danger window exists when the standby's clock is more than
+max_clock_skew behind the primary's clock, but not more than
+max_clock_skew + network latency time behind.  If the clock difference
+is in that range, then the algorithm presented above which is based on
+time of receipt cannot detect that the local clock is too far behind.
+The consequence of this problem could be as follows:
+
+1.  The standby loses contact with the primary due to a network fault.
+
+2.  The primary decides to drop the standby from the set of available
+    causal reads standbys due to lack of keepalive responses or
+    excessive lag, which necessitates holding up commits of causal
+    reads transactions until the most recently sent lease expires, in
+    the belief that the standby will definitely have started raising
+    the 'causal reads unavailable' error in causal reads transactions
+    by that time, if it is still alive and servicing requests.
+
+3.  The standby still has clients connected and running queries.
+
+4.  Due to clock skew in the problematic range, in the standby's
+    opinion the lease lasts slightly longer than the primary waits.
+
+5.  For a short window at most the duration of the network latency
+    time, clients running causal reads transactions are allowed to see
+    potentially stale data.
+
+For this reason we say that the causal reads guarantee only holds as
+long as the absolute difference between the system clocks of the
+machines is no more than max_clock_skew.  The theory is that NTP makes
+it possible to reason about the maximum possible clock difference
+between machines and choose a value that allows for a much larger
+difference.  However, we do make a best effort attempt to detect
+wildly divergent systems as described above, to catch the case of
+servers not running a correctly configured ntp daemon, or with a clock
+so far out of whack that ntp refuses to fix it.
\ No newline at end of file
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 376ddf4..8240d0d 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -57,6 +57,11 @@
 #include "utils/builtins.h"
 #include "utils/ps_status.h"
 
+/* GUC variables */
+int causal_reads_timeout;
+bool causal_reads;
+char *causal_reads_standby_names;
+
 /* User-settable parameters for sync rep */
 char	   *SyncRepStandbyNames;
 
@@ -69,7 +74,7 @@ static int	SyncRepWaitMode = SYNC_REP_NO_WAIT;
 
 static void SyncRepQueueInsert(int mode);
 static void SyncRepCancelWait(void);
-static int	SyncRepWakeQueue(bool all, int mode);
+static int	SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn);
 
 static int	SyncRepGetStandbyPriority(void);
 
@@ -142,6 +147,198 @@ SyncRepCheckEarlyExit(void)
 }
 
 /*
+ * Check if we can stop waiting for causal consistency.  We can stop waiting
+ * when the following conditions are met:
+ *
+ * 1.  All walsenders currently in 'joining' or 'available' state have
+ * applied the target LSN.
+ *
+ * 2.  Any stall periods caused by standbys dropping out of 'available' state
+ * have passed, so that we can be sure that their leases have expired and they
+ * have started rejecting causal reads transactions.
+ *
+ * The output parameter 'waitingFor' is set to the number of nodes we are
+ * currently waiting for.  The output parameters 'stallTimeMillis' is set to
+ * the number of milliseconds we need to wait for to observe any current
+ * commit stall.
+ *
+ * Returns true if commit can return control, because every standby has either
+ * applied the LSN or started rejecting causal_reads transactions.
+ */
+static bool
+CausalReadsCommitCanReturn(XLogRecPtr XactCommitLSN,
+						   int *waitingFor,
+						   long *stallTimeMillis)
+{
+	int i;
+	TimestampTz now;
+
+	/* Count how many joining/available nodes we are waiting for. */
+	*waitingFor = 0;
+	for (i = 0; i < max_wal_senders; ++i)
+	{
+		WalSnd *walsnd = &WalSndCtl->walsnds[i];
+
+		/*
+		 * Assuming atomic read of pid_t, we can check walsnd->pid without
+		 * acquiring the spinlock to avoid memory synchronization costs for
+		 * unused walsender slots.  We see a value that existed sometime at
+		 * least as recently as the last memory barrier.
+		 */
+		if (walsnd->pid != 0)
+		{
+			/*
+			 * We need to hold the spinlock to read LSNs, because we can't be
+			 * sure they can be read atomically.
+			 */
+			SpinLockAcquire(&walsnd->mutex);
+			if (walsnd->pid != 0 && walsnd->causal_reads_state >= WALSNDCRSTATE_JOINING)
+			{
+				if (walsnd->apply < XactCommitLSN)
+					++*waitingFor;
+			}
+			SpinLockRelease(&walsnd->mutex);
+		}
+	}
+
+	/* Check if there is a stall in progress that we need to observe. */
+	now = GetCurrentTimestamp();
+	LWLockAcquire(SyncRepLock, LW_SHARED);
+	if (WalSndCtl->stall_causal_reads_until > now)
+	{
+		long seconds;
+		int usecs;
+
+		/* Compute how long we have to wait, rounded up to nearest ms. */
+		TimestampDifference(now, WalSndCtl->stall_causal_reads_until,
+							&seconds, &usecs);
+		*stallTimeMillis = seconds * 1000 + (usecs + 999) / 1000;
+	}
+	else
+		*stallTimeMillis = 0;
+	LWLockRelease(SyncRepLock);
+
+	/* We are done if we are not waiting for any nodes or stalls. */
+	return *waitingFor == 0 && *stallTimeMillis == 0;
+}
+
+/*
+ * Wait for causal consistency in causal_reads mode, if requested by user.
+ */
+void
+CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN)
+{
+	long stallTimeMillis;
+	int waitingFor;
+	char *ps_display_buffer = NULL;
+
+	/* Leave if we aren't in causal_reads mode. */
+	if (!causal_reads)
+		return;
+
+	for (;;)
+	{
+		/* Reset latch before checking state. */
+		ResetLatch(MyLatch);
+
+		/*
+		 * Join the queue to be woken up if any causal reads joining/available
+		 * standby applies XactCommitLSN or the set of causal reads standbys
+		 * changes (if we aren't already in the queue).  We don't actually know
+		 * if we need to wait for any peers to reach the target LSN yet, but
+		 * we have to register just in case before checking the walsenders'
+		 * state to avoid a race condition that could occur if we did it after
+		 * calling CausalReadsCommitCanReturn.  (SyncRepWaitForLSN doesn't
+		 * have to do this because it can check the highest-seen LSN in
+		 * walsndctl->lsn[mode] which is protected by SyncRepLock, the same
+		 * lock as the queues.  We can't do that here, because there is no
+		 * single highest-seen LSN that is useful.  We must check
+		 * walsnd->apply for all relevant walsenders.  Therefore we must
+		 * register for notifications first, so that we can be notified via
+		 * our latch of any standby applying the LSN we're interested in after
+		 * we check but before we start waiting, or we could wait forever for
+		 * something that has already happened.)
+		 */
+		LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+		if (MyProc->syncRepState != SYNC_REP_WAITING)
+		{
+			MyProc->waitLSN = XactCommitLSN;
+			MyProc->syncRepState = SYNC_REP_WAITING;
+			SyncRepQueueInsert(SYNC_REP_WAIT_CAUSAL_READS);
+			Assert(SyncRepQueueIsOrderedByLSN(SYNC_REP_WAIT_CAUSAL_READS));
+		}
+		LWLockRelease(SyncRepLock);
+
+		/* Check if we're done. */
+		if (CausalReadsCommitCanReturn(XactCommitLSN, &waitingFor, &stallTimeMillis))
+		{
+			SyncRepCancelWait();
+			break;
+		}
+
+		Assert(waitingFor > 0 || stallTimeMillis > 0);
+
+		/* If we aren't actually waiting for any standbys, leave the queue. */
+		if (waitingFor == 0)
+			SyncRepCancelWait();
+
+		/* Update the ps title. */
+		if (update_process_title)
+		{
+			char buffer[80];
+
+			/* Remember the old value if this is our first update. */
+			if (ps_display_buffer == NULL)
+			{
+				int len;
+				const char *ps_display = get_ps_display(&len);
+
+				ps_display_buffer = palloc(len + 1);
+				memcpy(ps_display_buffer, ps_display, len);
+				ps_display_buffer[len] = '\0';
+			}
+
+			snprintf(buffer, sizeof(buffer),
+					 "waiting for %d peer(s) to apply %X/%X%s",
+					 waitingFor,
+					 (uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN,
+					 stallTimeMillis > 0 ? " (stalling)" : "");
+			set_ps_display(buffer, false);
+		}
+
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit()) /* Calls SyncRepCancelWait() if true. */
+			break;
+
+		/*
+		 * If are still waiting for peers, then we wait for any joining or
+		 * available peer to reach the LSN (or possibly stop being in one of
+		 * those states or go away).
+		 *
+		 * If not, there must be a non-zero stall time, so we wait for that to
+		 * elapse.
+		 */
+		if (waitingFor > 0)
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH, -1);
+		else
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH | WL_TIMEOUT,
+					  stallTimeMillis);
+	}
+
+	/* There is no way out of the loop that could leave us in the queue. */
+	Assert(SHMQueueIsDetached(&(MyProc->syncRepLinks)));
+	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
+	MyProc->waitLSN = 0;
+
+	/* Restore the ps display. */
+	if (ps_display_buffer != NULL)
+	{
+		set_ps_display(ps_display_buffer, false);
+		pfree(ps_display_buffer);
+	}
+}
+
+/*
  * Wait for synchronous replication, if requested by user.
  *
  * Initially backends start in state SYNC_REP_NOT_WAITING and then
@@ -425,6 +622,53 @@ SyncRepGetSynchronousStandby(void)
 }
 
 /*
+ * Check if the current WALSender process's application_name matches a name in
+ * causal_reads_standby_names (including '*' for wildcard).
+ */
+bool
+CausalReadsPotentialStandby(void)
+{
+	char *rawstring;
+	List	   *elemlist;
+	ListCell   *l;
+	bool		found = false;
+
+	/* If the feature is disable, then no. */
+	if (causal_reads_timeout == 0)
+		return false;
+
+	/* Need a modifiable copy of string */
+	rawstring = pstrdup(causal_reads_standby_names);
+
+	/* Parse string into list of identifiers */
+	if (!SplitIdentifierString(rawstring, ',', &elemlist))
+	{
+		/* syntax error in list */
+		pfree(rawstring);
+		list_free(elemlist);
+		/* GUC machinery will have already complained - no need to do again */
+		return 0;
+	}
+
+	foreach(l, elemlist)
+	{
+		char	   *standby_name = (char *) lfirst(l);
+
+		if (pg_strcasecmp(standby_name, application_name) == 0 ||
+			pg_strcasecmp(standby_name, "*") == 0)
+		{
+			found = true;
+			break;
+		}
+	}
+
+	pfree(rawstring);
+	list_free(elemlist);
+
+	return found;
+}
+
+/*
  * Update the LSNs on each queue based upon our latest state. This
  * implements a simple policy of first-valid-standby-releases-waiter.
  *
@@ -432,23 +676,27 @@ SyncRepGetSynchronousStandby(void)
  * perhaps also which information we store as well.
  */
 void
-SyncRepReleaseWaiters(void)
+SyncRepReleaseWaiters(bool walsender_cr_available_or_joining)
 {
 	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
 	int			numapply = 0;
+	int			numcausalreadsapply = 0;
+	bool		is_highest_priority_sync_standby;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
-	 * potential sync standbys then we have nothing to do. If we are still
-	 * starting up, still running base backup or the current flush position
-	 * is still invalid, then leave quickly also.
+	 * potential sync standbys and not in a state that causal_reads waits for,
+	 * then we have nothing to do. If we are still starting up, still running
+	 * base backup or the current flush position is still invalid, then leave
+	 * quickly also.
 	 */
-	if (MyWalSnd->sync_standby_priority == 0 ||
-		MyWalSnd->state < WALSNDSTATE_STREAMING ||
-		XLogRecPtrIsInvalid(MyWalSnd->flush))
+	if (!walsender_cr_available_or_joining &&
+		(MyWalSnd->sync_standby_priority == 0 ||
+		 MyWalSnd->state < WALSNDSTATE_STREAMING ||
+		 XLogRecPtrIsInvalid(MyWalSnd->flush)))
 		return;
 
 	/*
@@ -458,13 +706,19 @@ SyncRepReleaseWaiters(void)
 	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
 	syncWalSnd = SyncRepGetSynchronousStandby();
 
-	/* We should have found ourselves at least */
-	Assert(syncWalSnd != NULL);
+	/*
+	 * If we aren't managing the highest priority standby then make a note of
+	 * that so we can announce a takeover in the log if we ever get that job.
+	 */
+	is_highest_priority_sync_standby = syncWalSnd == MyWalSnd;
+	if (!is_highest_priority_sync_standby)
+		announce_next_takeover = true;
 
 	/*
-	 * If we aren't managing the highest priority standby then just leave.
+	 * If we aren't managing the highest priority standby or a standby in
+	 * causal reads 'joining' or 'available' state, then just leave.
 	 */
-	if (syncWalSnd != MyWalSnd)
+	if (!is_highest_priority_sync_standby && !walsender_cr_available_or_joining)
 	{
 		LWLockRelease(SyncRepLock);
 		announce_next_takeover = true;
@@ -473,24 +727,45 @@ SyncRepReleaseWaiters(void)
 
 	/*
 	 * Set the lsn first so that when we wake backends they will release up to
-	 * this location.
+	 * this location.  For the single-standby synchronous commit levels, we
+	 * only do this if we are the current synchronous standby and we are
+	 * advancing the LSN further than it has been advanced before, so that
+	 * SyncRepWaitForLSN can skip waiting in some cases.
 	 */
-	if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
-		numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
-		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+	if (is_highest_priority_sync_standby)
 	{
-		walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
-		numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY);
+		if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
+			numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE,
+										MyWalSnd->write);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
+			numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH,
+										MyWalSnd->flush);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+			numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY,
+										MyWalSnd->apply);
+		}
 	}
 
+	/*
+	 * For causal_reads, all walsenders currently in available or joining
+	 * state must reach the LSN on their own, and standbys will reach LSNs in
+	 * any order.  It doesn't make sense to keep the highest seen LSN in a
+	 * single walsndctl->lsn element.  (CausalReadsWaitForLSN has handling for
+	 * LSNs that have already been reached).
+	 */
+	if (walsender_cr_available_or_joining)
+		numcausalreadsapply =
+			SyncRepWakeQueue(false, SYNC_REP_WAIT_CAUSAL_READS,
+							 MyWalSnd->apply);
+
 	LWLockRelease(SyncRepLock);
 
 	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%x",
@@ -502,7 +777,7 @@ SyncRepReleaseWaiters(void)
 	 * If we are managing the highest priority standby, though we weren't
 	 * prior to this, then announce we are now the sync standby.
 	 */
-	if (announce_next_takeover)
+	if (is_highest_priority_sync_standby && announce_next_takeover)
 	{
 		announce_next_takeover = false;
 		ereport(LOG,
@@ -577,9 +852,8 @@ SyncRepGetStandbyPriority(void)
  * Must hold SyncRepLock.
  */
 static int
-SyncRepWakeQueue(bool all, int mode)
+SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn)
 {
-	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	PGPROC	   *proc = NULL;
 	PGPROC	   *thisproc = NULL;
 	int			numprocs = 0;
@@ -596,7 +870,7 @@ SyncRepWakeQueue(bool all, int mode)
 		/*
 		 * Assume the queue is ordered by LSN
 		 */
-		if (!all && walsndctl->lsn[mode] < proc->waitLSN)
+		if (!all && lsn < proc->waitLSN)
 			return numprocs;
 
 		/*
@@ -656,7 +930,7 @@ SyncRepUpdateSyncStandbysDefined(void)
 			int			i;
 
 			for (i = 0; i < NUM_SYNC_REP_WAIT_MODE; i++)
-				SyncRepWakeQueue(true, i);
+				SyncRepWakeQueue(true, i, InvalidXLogRecPtr);
 		}
 
 		/*
@@ -708,13 +982,31 @@ SyncRepQueueIsOrderedByLSN(int mode)
 #endif
 
 /*
+ * Make sure that CausalReadsWaitForLSN can't return until after the given
+ * lease expiry time has been reached.  In other words, revoke the lease.
+ *
+ * Wake up all backends waiting in CausalReadsWaitForLSN, because the set of
+ * available/joining peers has changed, and there is a new stall time they
+ * need to observe.
+ */
+void
+CausalReadsBeginStall(TimestampTz lease_expiry_time)
+{
+	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+	WalSndCtl->stall_causal_reads_until =
+		Max(WalSndCtl->stall_causal_reads_until, lease_expiry_time);
+	SyncRepWakeQueue(true, SYNC_REP_WAIT_CAUSAL_READS, InvalidXLogRecPtr);
+	LWLockRelease(SyncRepLock);
+}
+
+/*
  * ===========================================================
  * Synchronous Replication functions executed by any process
  * ===========================================================
  */
 
 bool
-check_synchronous_standby_names(char **newval, void **extra, GucSource source)
+check_standby_names(char **newval, void **extra, GucSource source)
 {
 	char	   *rawstring;
 	List	   *elemlist;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index fd7aecb..22587a2 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -55,6 +55,7 @@
 #include "libpq/pqformat.h"
 #include "libpq/pqsignal.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "replication/walsender.h"
 #include "storage/ipc.h"
@@ -149,7 +150,8 @@ static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
 static void XLogWalRcvSendReply(bool force, bool requestReply, bool includeApplyTimestamp);
 static void XLogWalRcvSendHSFeedback(bool immed);
-static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
+static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+								  TimestampTz *causalReadsUntil);
 
 /* Signal handlers */
 static void WalRcvSigHupHandler(SIGNAL_ARGS);
@@ -831,6 +833,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 	XLogRecPtr	walEnd;
 	TimestampTz sendTime;
 	bool		replyRequested;
+	TimestampTz causalReadsLease;
 
 	resetStringInfo(&incoming_message);
 
@@ -851,7 +854,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				walEnd = pq_getmsgint64(&incoming_message);
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, NULL);
 
 				buf += hdrlen;
 				len -= hdrlen;
@@ -863,7 +866,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				bool reportApplyTimestamp = false;
 
 				/* copy message to StringInfo */
-				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
+				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char) + sizeof(int64);
 				if (len != hdrlen)
 					ereport(ERROR,
 							(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -875,8 +878,10 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
 				replyRequested = pq_getmsgbyte(&incoming_message);
+				causalReadsLease = IntegerTimestampToTimestampTz(
+					pq_getmsgint64(&incoming_message));
 
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, &causalReadsLease);
 
 				/*
 				 * If no apply timestamps have been sent at the request of the
@@ -1225,15 +1230,52 @@ XLogWalRcvSendHSFeedback(bool immed)
  * Update shared memory status upon receiving a message from primary.
  *
  * 'walEnd' and 'sendTime' are the end-of-WAL and timestamp of the latest
- * message, reported by primary.
+ * message, reported by primary.  'causalReadsLease' is a pointer to
+ * the time the primary promises that this standby can safely claim to be
+ * causally consistent, to 0 if it cannot, or a NULL pointer for no change.
  */
 static void
-ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
+ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+					  TimestampTz *causalReadsLease)
 {
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
 	static TimestampTz lastRecordedTimestamp = 0;
 
+	/* Sanity check for the causalReadsLease time. */
+	if (causalReadsLease != NULL && *causalReadsLease != 0)
+	{
+		/* Deduce max_clock_skew from the causalReadsLease and sendTime. */
+#ifdef HAVE_INT64_TIMESTAMP
+		int64 diffMillis = (*causalReadsLease - sendTime) / 1000;
+#else
+		int64 diffMillis = (*causalReadsLease - sendTime) * 1000;
+#endif
+		int64 max_clock_skew = diffMillis / (CAUSAL_READS_CLOCK_SKEW_RATIO - 1);
+
+		if (sendTime > TimestampTzPlusMilliseconds(lastMsgReceiptTime, max_clock_skew))
+		{
+			/*
+			 * The primary's clock is more than max_clock_skew + network
+			 * latency ahead of the standby's clock.  (If the primary's clock
+			 * is more than max_clock_skew ahead of the standby's clock, but
+			 * by less than the network latency, then there isn't much we can
+			 * do to detect that; but it still seems useful to have this basic
+			 * sanity check for wildly misconfigured servers.)
+			 */
+			elog(LOG, "the primary server's clock time is too far ahead");
+			causalReadsLease = NULL;
+		}
+		/*
+		 * We could also try to detect cases where sendTime is more than
+		 * max_clock_skew in the past according to the standby's clock, but
+		 * that is indistinguishable from network latency/buffering, so we
+		 * could produce misleading error messages; if we do nothing, the
+		 * consequence is 'standby is not available for causal reads' errors
+		 * which should cause the user to investigate.
+		 */
+	}
+
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
 	if (walrcv->latestWalEnd < walEnd)
@@ -1241,6 +1283,8 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->latestWalEnd = walEnd;
 	walrcv->lastMsgSendTime = sendTime;
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
+	if (causalReadsLease != NULL)
+		walrcv->causalReadsLease = *causalReadsLease;
 	SpinLockRelease(&walrcv->mutex);
 
 	/*
diff --git a/src/backend/replication/walreceiverfuncs.c b/src/backend/replication/walreceiverfuncs.c
index 5f6e423..e502f74 100644
--- a/src/backend/replication/walreceiverfuncs.c
+++ b/src/backend/replication/walreceiverfuncs.c
@@ -28,6 +28,7 @@
 #include "replication/walreceiver.h"
 #include "storage/pmsignal.h"
 #include "storage/shmem.h"
+#include "utils/guc.h"
 #include "utils/timestamp.h"
 
 WalRcvData *WalRcv = NULL;
@@ -374,3 +375,21 @@ GetReplicationTransferLatency(void)
 
 	return ms;
 }
+
+/*
+ * Used by snapmgr to check if this standby has a valid lease, granting it the
+ * right to consider itself available for causal reads.
+ */
+bool
+WalRcvCausalReadsAvailable(void)
+{
+	WalRcvData *walrcv = WalRcv;
+	TimestampTz now = GetCurrentTimestamp();
+	bool result;
+
+	SpinLockAcquire(&walrcv->mutex);
+	result = walrcv->causalReadsLease != 0 && now <= walrcv->causalReadsLease;
+	SpinLockRelease(&walrcv->mutex);
+
+	return result;
+}
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 16d7abc..b4dad72 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -154,9 +154,20 @@ static StringInfoData tmpbuf;
  */
 static TimestampTz last_reply_timestamp = 0;
 
+static TimestampTz last_keepalive_timestamp = 0;
+
 /* Have we sent a heartbeat message asking for reply, since last reply? */
 static bool waiting_for_ping_response = false;
 
+/* How long do need to stay in JOINING state? */
+static XLogRecPtr causal_reads_joining_until = 0;
+
+/* The last causal reads lease sent to the standby. */
+static TimestampTz causal_reads_last_lease = 0;
+
+/* Is this WALSender listed in causal_reads_standby_names? */
+static bool am_potential_causal_reads_standby = false;
+
 /*
  * While streaming WAL in Copy mode, streamingDoneSending is set to true
  * after we have sent CopyDone. We should not send any more CopyData messages
@@ -242,6 +253,57 @@ InitWalSender(void)
 	SendPostmasterSignal(PMSIGNAL_ADVANCE_STATE_MACHINE);
 }
 
+ /*
+ * If we are exiting unexpectedly, we may need to communicate with concurrent
+ * causal_reads commits to maintain the causal consistency guarantee.
+ */
+static void
+PrepareUncleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * We've lost contact with the standby, but it may still be alive.  We
+		 * can't let any causal_reads transactions return until we've stalled
+		 * for long enough for a zombie standby to start raising errors
+		 * because its lease has expired.
+		 */
+		elog(LOG, "standby \"%s\" is lost (no longer available for causal reads)", application_name);
+		CausalReadsBeginStall(causal_reads_last_lease);
+
+		/*
+		 * We set the state to a lower level _after_ beginning the stall,
+		 * otherwise there would be a tiny window where commits could return
+		 * without observing the stall.
+		 */
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
+ * We are shutting down because we received a goodbye message from the
+ * walreceiver.
+ */
+static void
+PrepareCleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * The standby is shutting down, so it won't be running any more
+		 * transactions.  It is therefore safe to stop waiting for it, and no
+		 * stall is necessary.
+		 */
+		elog(LOG, "standby \"%s\" is leaving (no longer available for causal reads)", application_name);
+
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
 /*
  * Clean up after an error.
  *
@@ -266,7 +328,10 @@ WalSndErrorCleanup(void)
 
 	replication_active = false;
 	if (walsender_ready_to_stop)
+	{
+		PrepareUncleanExit();
 		proc_exit(0);
+	}
 
 	/* Revert back to startup state */
 	WalSndSetState(WALSNDSTATE_STARTUP);
@@ -278,6 +343,8 @@ WalSndErrorCleanup(void)
 static void
 WalSndShutdown(void)
 {
+	PrepareUncleanExit();
+
 	/*
 	 * Reset whereToSendOutput to prevent ereport from attempting to send any
 	 * more messages to the standby.
@@ -1388,6 +1455,7 @@ ProcessRepliesIfAny(void)
 		if (r < 0)
 		{
 			/* unexpected error or EOF */
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1404,6 +1472,7 @@ ProcessRepliesIfAny(void)
 		resetStringInfo(&reply_message);
 		if (pq_getmessage(&reply_message, 0))
 		{
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1453,6 +1522,7 @@ ProcessRepliesIfAny(void)
 				 * 'X' means that the standby is closing down the socket.
 				 */
 			case 'X':
+				PrepareCleanExit();
 				proc_exit(0);
 
 			default:
@@ -1584,6 +1654,83 @@ ProcessStandbyReplyMessage(void)
 	 */
 	{
 		WalSnd *walsnd = MyWalSnd;
+		WalSndCausalReadsState causal_reads_state = walsnd->causal_reads_state;
+		bool causal_reads_state_changed = false;
+		bool causal_reads_set_joining_until = false;
+
+		/*
+		 * Handle causal reads state transitions, if a causal_reads_timeout is
+		 * configured, this standby is listed in causal_reads_standby_names,
+		 * and we are a primary database (not a cascading standby).
+		 */
+		if (am_potential_causal_reads_standby &&
+			!am_cascading_walsender &&
+			applyLagUs >= 0)
+		{
+			if (applyLagUs / 1000 < causal_reads_timeout)
+			{
+				if (causal_reads_state == WALSNDCRSTATE_UNAVAILABLE)
+				{
+					/*
+					 * The standby is applying fast enough.  We can't grant a
+					 * lease yet though, we need to wait for everything that
+					 * was committed while this standby was unavailable to be
+					 * applied first.  We move to joining state while we wait
+					 * for the standby to catch up.
+					 */
+					causal_reads_state = WALSNDCRSTATE_JOINING;
+					causal_reads_set_joining_until = true;
+					causal_reads_state_changed = true;
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING &&
+						 applyPtr >= causal_reads_joining_until)
+				{
+					/*
+					 * The standby has applied everything committed before we
+					 * reached joining state, and has been waiting for remote
+					 * apply on this standby while it's been in joining state,
+					 * so it is safe to move to available state and send a
+					 * lease.
+					 */
+					causal_reads_state = WALSNDCRSTATE_AVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+			else
+			{
+				if (causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+					/*
+					 * We are dropping a causal reads available standby, so we
+					 * mustn't let any commit command that is waiting in
+					 * CausalReadsWaitForLSN return until we are sure that the
+					 * standby definitely knows that it's not available and
+					 * starts raising errors for causal_reads transactions.
+					 * TODO: We could just wait until the standby acks that
+					 * its lease has been cancelled, and start numbering
+					 * keepalives and sending the number back in replies, so
+					 * we know it's acking the right message; then lagging
+					 * standbys would be less disruptive, but for now we just
+					 * wait for the lease to expire, as we do when we lose
+					 * contact with a standby, for the sake of simplicity.
+					 */
+					CausalReadsBeginStall(causal_reads_last_lease);
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING)
+				{
+					/*
+					 * Dropping a joining standby doesn't require a stall,
+					 * because the standby doesn't think it's available, so
+					 * it's already raising the error for causal_reads
+					 * transactions.
+					 */
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+		}
 
 		SpinLockAcquire(&walsnd->mutex);
 		walsnd->write = writePtr;
@@ -1591,11 +1738,33 @@ ProcessStandbyReplyMessage(void)
 		walsnd->apply = applyPtr;
 		if (applyLagUs >= 0)
 			walsnd->applyLagUs = applyLagUs;
+		walsnd->causal_reads_state = causal_reads_state;
 		SpinLockRelease(&walsnd->mutex);
+
+		if (causal_reads_set_joining_until)
+		{
+			/*
+			 * Record the end of the primary's WAL at some arbitrary point
+			 * observed _after_ we moved to joining state (so that causal
+			 * reads commits start waiting, closing a race).  The standby
+			 * won't become available until it has replayed up to here.
+			 */
+			causal_reads_joining_until = GetFlushRecPtr();
+		}
+
+		if (causal_reads_state_changed)
+		{
+			WalSndKeepalive(true);
+			elog(LOG, "standby \"%s\" is %s", application_name,
+				 causal_reads_state == WALSNDCRSTATE_UNAVAILABLE ? "unavailable for causal reads" :
+				 causal_reads_state == WALSNDCRSTATE_JOINING ? "joining as a causal reads standby..." :
+				 causal_reads_state == WALSNDCRSTATE_AVAILABLE ? "available for causal reads" :
+				 "UNKNOWN");
+		}
 	}
 
 	if (!am_cascading_walsender)
-		SyncRepReleaseWaiters();
+		SyncRepReleaseWaiters(MyWalSnd->causal_reads_state >= WALSNDCRSTATE_JOINING);
 
 	/*
 	 * Advance our local xmin horizon when the client confirmed a flush.
@@ -1736,33 +1905,53 @@ ProcessStandbyHSFeedbackMessage(void)
  * If wal_sender_timeout is enabled we want to wake up in time to send
  * keepalives and to abort the connection if wal_sender_timeout has been
  * reached.
+ *
+ * But if causal_reads_timeout is enabled, we override that and send
+ * keepalives at a constant rate to replace expiring leases.
  */
 static long
 WalSndComputeSleeptime(TimestampTz now)
 {
 	long		sleeptime = 10000;		/* 10 s */
 
-	if (wal_sender_timeout > 0 && last_reply_timestamp > 0)
+	if ((wal_sender_timeout > 0 && last_reply_timestamp > 0) ||
+		am_potential_causal_reads_standby)
 	{
 		TimestampTz wakeup_time;
 		long		sec_to_timeout;
 		int			microsec_to_timeout;
 
-		/*
-		 * At the latest stop sleeping once wal_sender_timeout has been
-		 * reached.
-		 */
-		wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-												  wal_sender_timeout);
-
-		/*
-		 * If no ping has been sent yet, wakeup when it's time to do so.
-		 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
-		 * the timeout passed without a response.
-		 */
-		if (!waiting_for_ping_response)
+		if (am_potential_causal_reads_standby)
+		{
+			/*
+			 * Leases last for a period of between 50% and 100% of
+			 * causal_reads_timeout, depending on clock skew, assuming clock
+			 * skew is under the 25% of causal_reads_timeout.  We send new
+			 * leases every half a lease, so that there are no gaps between
+			 * leases.
+			 */
+			wakeup_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+													  causal_reads_timeout /
+													  CAUSAL_READS_KEEPALIVE_RATIO);
+		}
+		else
+		{
+			/*
+			 * At the latest stop sleeping once wal_sender_timeout has been
+			 * reached.
+			 */
 			wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-													  wal_sender_timeout / 2);
+													  wal_sender_timeout);
+
+			/*
+			 * If no ping has been sent yet, wakeup when it's time to do so.
+			 * WalSndKeepaliveIfNecessary() wants to send a keepalive once
+			 * half of the timeout passed without a response.
+			 */
+			if (!waiting_for_ping_response)
+				wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+														  wal_sender_timeout / 2);
+		}
 
 		/* Compute relative time until wakeup. */
 		TimestampDifference(now, wakeup_time,
@@ -1778,20 +1967,33 @@ WalSndComputeSleeptime(TimestampTz now)
 /*
  * Check whether there have been responses by the client within
  * wal_sender_timeout and shutdown if not.
+ *
+ * If causal_reads_timeout is configured we override that, so that
+ * unresponsive standbys are detected sooner.
  */
 static void
 WalSndCheckTimeOut(TimestampTz now)
 {
 	TimestampTz timeout;
+	int allowed_time;
 
 	/* don't bail out if we're doing something that doesn't require timeouts */
 	if (last_reply_timestamp <= 0)
 		return;
 
-	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
-										  wal_sender_timeout);
+	/*
+	 * If a causal_reads_timeout is configured, it is used instead of
+	 * wal_sender_timeout, to limit the time before an unresponsive causal
+	 * reads standby is dropped.
+	 */
+	if (am_potential_causal_reads_standby)
+		allowed_time = causal_reads_timeout;
+	else
+		allowed_time = wal_sender_timeout;
 
-	if (wal_sender_timeout > 0 && now >= timeout)
+	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
+										  allowed_time);
+	if (allowed_time > 0 && now >= timeout)
 	{
 		/*
 		 * Since typically expiration of replication timeout means
@@ -1824,6 +2026,9 @@ WalSndLoop(WalSndSendDataCallback send_data)
 	last_reply_timestamp = GetCurrentTimestamp();
 	waiting_for_ping_response = false;
 
+	/* Check if we are managing potential causal_reads standby. */
+	am_potential_causal_reads_standby = CausalReadsPotentialStandby();
+
 	/*
 	 * Loop until we reach the end of this timeline or the client requests to
 	 * stop streaming.
@@ -1984,6 +2189,7 @@ InitWalSenderSlot(void)
 			walsnd->flush = InvalidXLogRecPtr;
 			walsnd->apply = InvalidXLogRecPtr;
 			walsnd->state = WALSNDSTATE_STARTUP;
+			walsnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
 			walsnd->latch = &MyProc->procLatch;
 			SpinLockRelease(&walsnd->mutex);
 			/* don't need the lock anymore */
@@ -2753,6 +2959,24 @@ WalSndGetStateString(WalSndState state)
 	return "UNKNOWN";
 }
 
+/*
+ * Return a string constant representing the causal reads state. This is used
+ * in system views, and should *not* be translated.
+ */
+static const char *
+WalSndGetCausalReadsStateString(WalSndCausalReadsState causal_reads_state)
+{
+	switch (causal_reads_state)
+	{
+		case WALSNDCRSTATE_UNAVAILABLE:
+			return "unavailable";
+		case WALSNDCRSTATE_JOINING:
+			return "joining";
+		case WALSNDCRSTATE_AVAILABLE:
+			return "available";
+	}
+	return "UNKNOWN";
+}
 
 /*
  * Returns activity of walsenders, including pids and xlog locations sent to
@@ -2761,7 +2985,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	9
+#define PG_STAT_GET_WAL_SENDERS_COLS	10
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2812,6 +3036,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		int64		applyLagUs;
 		int			priority;
 		WalSndState state;
+		WalSndCausalReadsState causalReadsState;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
 		bool		nulls[PG_STAT_GET_WAL_SENDERS_COLS];
 
@@ -2821,6 +3046,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		SpinLockAcquire(&walsnd->mutex);
 		sentPtr = walsnd->sentPtr;
 		state = walsnd->state;
+		causalReadsState = walsnd->causal_reads_state;
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
@@ -2895,6 +3121,9 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				values[8] = CStringGetTextDatum("sync");
 			else
 				values[8] = CStringGetTextDatum("potential");
+
+			values[9] =
+				CStringGetTextDatum(WalSndGetCausalReadsStateString(causalReadsState));
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
@@ -2914,14 +3143,52 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 static void
 WalSndKeepalive(bool requestReply)
 {
+	TimestampTz now;
+	TimestampTz causal_reads_lease;
+
 	elog(DEBUG2, "sending replication keepalive");
 
+	/*
+	 * If the walsender currently deems the standby to be available for causal
+	 * reads, then it grants a causal reads lease.  The lease authorizes the
+	 * standby to consider itself available for causal reads until a short
+	 * time in the future.  The primary promises to uphold the causal reads
+	 * guarantee until that time, by stalling commits until the the lease has
+	 * expired if necessary.
+	 */
+	now = GetCurrentTimestamp();
+	if (MyWalSnd->causal_reads_state < WALSNDCRSTATE_AVAILABLE)
+		causal_reads_lease = 0; /* Not available, no lease granted. */
+	else
+	{
+		/*
+		 * Since this timestamp is being sent to the standby where it will be
+		 * compared against a time generated by the standby's system clock, we
+		 * must consider clock skew.  First, we decide on a maximum tolerable
+		 * difference between system clocks.  If the primary's clock is ahead
+		 * of the standby's by more than this, then all bets are off (the
+		 * standby could falsely believe it has a valid lease).  If the
+		 * primary's clock is behind the standby's by more than this, then the
+		 * standby will err the other way and generate spurious errors in
+		 * causal_reads mode.  Rather than having a separate GUC for this, we
+		 * derive it from causal_reads_timeout.
+		 */
+		int max_clock_skew = causal_reads_timeout / CAUSAL_READS_CLOCK_SKEW_RATIO;
+
+		/* Compute and remember the expiry time of the lease we're granting. */
+		causal_reads_last_lease = TimestampTzPlusMilliseconds(now, causal_reads_timeout);
+		/* The version we'll send to the standby is adjusted to tolerate clock skew. */
+		causal_reads_lease =
+			TimestampTzPlusMilliseconds(causal_reads_last_lease, -max_clock_skew);
+	}
+
 	/* construct the message... */
 	resetStringInfo(&output_message);
 	pq_sendbyte(&output_message, 'k');
 	pq_sendint64(&output_message, sentPtr);
-	pq_sendint64(&output_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(now));
 	pq_sendbyte(&output_message, requestReply ? 1 : 0);
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(causal_reads_lease));
 
 	/* ... and send it wrapped in CopyData */
 	pq_putmessage_noblock('d', output_message.data, output_message.len);
@@ -2939,23 +3206,35 @@ WalSndKeepaliveIfNecessary(TimestampTz now)
 	 * Don't send keepalive messages if timeouts are globally disabled or
 	 * we're doing something not partaking in timeouts.
 	 */
-	if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
-		return;
-
-	if (waiting_for_ping_response)
-		return;
+	if (!am_potential_causal_reads_standby)
+	{
+		if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
+			return;
+		if (waiting_for_ping_response)
+			return;
+	}
 
 	/*
 	 * If half of wal_sender_timeout has lapsed without receiving any reply
 	 * from the standby, send a keep-alive message to the standby requesting
 	 * an immediate reply.
+	 *
+	 * If causal_reads_timeout has been configured, use it to control
+	 * keepalive intervals rather than wal_sender_timeout, so that we can keep
+	 * replacing leases at the right frequency.
 	 */
-	ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-											wal_sender_timeout / 2);
+	if (am_potential_causal_reads_standby)
+		ping_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+												causal_reads_timeout /
+												CAUSAL_READS_KEEPALIVE_RATIO);
+	else
+		ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+												wal_sender_timeout / 2);
 	if (now >= ping_time)
 	{
 		WalSndKeepalive(true);
 		waiting_for_ping_response = true;
+		last_keepalive_timestamp = now;
 
 		/* Try to flush pending output to the client */
 		if (pq_flush_if_writable() != 0)
diff --git a/src/backend/utils/errcodes.txt b/src/backend/utils/errcodes.txt
index 49494f9..d81c089 100644
--- a/src/backend/utils/errcodes.txt
+++ b/src/backend/utils/errcodes.txt
@@ -306,6 +306,7 @@ Section: Class 40 - Transaction Rollback
 40001    E    ERRCODE_T_R_SERIALIZATION_FAILURE                              serialization_failure
 40003    E    ERRCODE_T_R_STATEMENT_COMPLETION_UNKNOWN                       statement_completion_unknown
 40P01    E    ERRCODE_T_R_DEADLOCK_DETECTED                                  deadlock_detected
+40P02    E    ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE                         causal_reads_not_available
 
 Section: Class 42 - Syntax Error or Access Rule Violation
 
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 06cb166..ac422e7 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -1634,6 +1634,16 @@ static struct config_bool ConfigureNamesBool[] =
 	},
 
 	{
+		{"causal_reads", PGC_USERSET, REPLICATION_STANDBY,
+		 gettext_noop("Enables causal reads."),
+		 NULL
+		},
+		&causal_reads,
+		false,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"syslog_sequence_numbers", PGC_SIGHUP, LOGGING_WHERE,
 			gettext_noop("Add sequence number to syslog messages to avoid duplicate suppression."),
 			NULL
@@ -1811,6 +1821,17 @@ static struct config_int ConfigureNamesInt[] =
 	},
 
 	{
+		{"causal_reads_timeout", PGC_SIGHUP, REPLICATION_STANDBY,
+			gettext_noop("Sets the maximum apply lag before causal reads standbys are no longer available."),
+			NULL,
+			GUC_UNIT_MS
+		},
+		&causal_reads_timeout,
+		0, 0, INT_MAX,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"max_connections", PGC_POSTMASTER, CONN_AUTH_SETTINGS,
 			gettext_noop("Sets the maximum number of concurrent connections."),
 			NULL
@@ -3454,7 +3475,18 @@ static struct config_string ConfigureNamesString[] =
 		},
 		&SyncRepStandbyNames,
 		"",
-		check_synchronous_standby_names, NULL, NULL
+		check_standby_names, NULL, NULL
+	},
+
+	{
+		{"causal_reads_standby_names", PGC_SIGHUP, REPLICATION_MASTER,
+			gettext_noop("List of names of potential causal reads standbys."),
+			NULL,
+			GUC_LIST_INPUT
+		},
+		&causal_reads_standby_names,
+		"*",
+		check_standby_names, NULL, NULL
 	},
 
 	{
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index ec4427f..fcc2c35 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -244,6 +244,15 @@
 				# from standby(s); '*' = all
 #vacuum_defer_cleanup_age = 0	# number of xacts by which cleanup is delayed
 
+#causal_reads_timeout = 0s      # maximum replication delay to tolerate from
+                                # standbys before dropping them from the set of
+				# available causal reads peers; 0 to disable
+				# causal reads
+
+#causal_reads_standy_names = '*'
+                                # standby servers that can potentially become
+				# available for causal reads; '*' = all
+
 # - Standby Servers -
 
 # These settings are ignored on a master server.
@@ -266,6 +275,14 @@
 #wal_retrieve_retry_interval = 5s	# time to wait before retrying to
 					# retrieve WAL after a failed attempt
 
+# - All Servers -
+
+#causal_reads = off                     # "on" in any pair of consecutive
+                                        # transactions guarantees that the second
+					# can see the first (even if the second
+					# is run on a standby), or will raise an
+					# error to report that the standby is
+					# unavailable for causal reads
 
 #------------------------------------------------------------------------------
 # QUERY TUNING
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index b88e012..6336240 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -46,8 +46,11 @@
 
 #include "access/transam.h"
 #include "access/xact.h"
+#include "access/xlog.h"
 #include "lib/pairingheap.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
+#include "replication/walreceiver.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
 #include "storage/procarray.h"
@@ -209,6 +212,16 @@ GetTransactionSnapshot(void)
 				 "cannot take query snapshot during a parallel operation");
 
 		/*
+		 * In causal_reads mode on a standby, check if we have definitely
+		 * applied WAL for any COMMIT that returned successfully on the
+		 * primary.
+		 */
+		if (causal_reads && RecoveryInProgress() && !WalRcvCausalReadsAvailable())
+			ereport(ERROR,
+					(errcode(ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE),
+					 errmsg("standby is not available for causal reads")));
+
+		/*
 		 * In transaction-snapshot mode, the first snapshot must live until
 		 * end of xact regardless of what the caller does with it, so we must
 		 * make a copy of it rather than returning CurrentSnapshotData
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 4054726..c0d7173 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2712,7 +2712,7 @@ DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f
 DESCR("statistics: information about currently active backends");
 DATA(insert OID = 3318 (  pg_stat_get_progress_info           PGNSP PGUID 12 1 100 0 0 f f f f t t s r 1 0 2249 "25" "{25,23,26,26,20,20,20,20,20,20,20,20,20,20}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{cmdtype,pid,datid,relid,param1,param2,param3,param4,param5,param6,param7,param8,param9,param10}" _null_ _null_ pg_stat_get_progress_info _null_ _null_ _null_ ));
 DESCR("statistics: information about progress of backends running maintenance command");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25}" "{o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25,25}" "{o,o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state,causal_reads_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index c005a42..dbfd601 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -24,14 +24,33 @@
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
 #define SYNC_REP_WAIT_APPLY		2
+#define SYNC_REP_WAIT_CAUSAL_READS 3
 
-#define NUM_SYNC_REP_WAIT_MODE	3
+#define NUM_SYNC_REP_WAIT_MODE	4
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
 #define SYNC_REP_WAITING			1
 #define SYNC_REP_WAIT_COMPLETE		2
 
+/*
+ * ratio of causal_read_timeout to max_clock_skew (4 means than the maximum
+ * tolerated clock difference between primary and standbys using causal_reads
+ * is 1/4 of causal_reads_timeout)
+ */
+#define CAUSAL_READS_CLOCK_SKEW_RATIO 4
+
+/*
+ * ratio of causal_reads_timeout to keepalive time (2 means that the effective
+ * keepalive time is 1/2 of the causal_reads_timeout GUC when it is non-zero)
+ */
+#define CAUSAL_READS_KEEPALIVE_RATIO 2
+
+/* GUC variables */
+extern int causal_reads_timeout;
+extern bool causal_reads;
+extern char *causal_reads_standby_names;
+
 /* user-settable parameters for synchronous replication */
 extern char *SyncRepStandbyNames;
 
@@ -43,16 +62,23 @@ extern void SyncRepCleanupAtProcExit(void);
 
 /* called by wal sender */
 extern void SyncRepInitConfig(void);
-extern void SyncRepReleaseWaiters(void);
+extern void SyncRepReleaseWaiters(bool walsender_cr_available_or_joining);
 
 /* called by checkpointer */
 extern void SyncRepUpdateSyncStandbysDefined(void);
 
+/* called by user backend (xact.c) */
+extern void CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN);
+
+/* called by wal sender */
+extern void CausalReadsBeginStall(TimestampTz lease_expiry_time);
+extern bool CausalReadsPotentialStandby(void);
+
 /* forward declaration to avoid pulling in walsender_private.h */
 struct WalSnd;
 extern struct WalSnd *SyncRepGetSynchronousStandby(void);
 
-extern bool check_synchronous_standby_names(char **newval, void **extra, GucSource source);
+extern bool check_standby_names(char **newval, void **extra, GucSource source);
 extern void assign_synchronous_commit(int newval, void *extra);
 
 #endif   /* _SYNCREP_H */
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index e4a1c3a..056c448 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -80,6 +80,13 @@ typedef struct
 	TimeLineID	receivedTLI;
 
 	/*
+	 * causalReadsLease is the time until which the primary has authorized
+	 * this standby to consider itself available for causal_reads mode, or 0
+	 * for not authorized.
+	 */
+	TimestampTz causalReadsLease;
+
+	/*
 	 * latestChunkStart is the starting byte position of the current "batch"
 	 * of received WAL.  It's actually the same as the previous value of
 	 * receivedUpto before the last flush to disk.  Startup process can use
@@ -165,4 +172,6 @@ extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
 extern void WalRcvWakeup(void);
 
+extern bool WalRcvCausalReadsAvailable(void);
+
 #endif   /* _WALRECEIVER_H */
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 4de43e8..f6e0e9e 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -27,6 +27,13 @@ typedef enum WalSndState
 	WALSNDSTATE_STREAMING
 } WalSndState;
 
+typedef enum WalSndCausalReadsState
+{
+	WALSNDCRSTATE_UNAVAILABLE = 0,
+	WALSNDCRSTATE_JOINING,
+	WALSNDCRSTATE_AVAILABLE
+} WalSndCausalReadsState;
+
 /*
  * Each walsender has a WalSnd struct in shared memory.
  */
@@ -34,6 +41,7 @@ typedef struct WalSnd
 {
 	pid_t		pid;			/* this walsender's process id, or 0 */
 	WalSndState state;			/* this walsender's state */
+	WalSndCausalReadsState causal_reads_state; /* the walsender's causal reads state */
 	XLogRecPtr	sentPtr;		/* WAL has been sent up to this point */
 	bool		needreload;		/* does currently-open file need to be
 								 * reloaded? */
@@ -89,6 +97,12 @@ typedef struct
 	 */
 	bool		sync_standbys_defined;
 
+	/*
+	 * Until when must commits in causal_reads stall?  This is used to wait
+	 * for causal reads leases to expire.
+	 */
+	TimestampTz	stall_causal_reads_until;
+
 	WalSnd		walsnds[FLEXIBLE_ARRAY_MEMBER];
 } WalSndCtlData;
 
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index fc4b765..44f826f 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1785,10 +1785,11 @@ pg_stat_replication| SELECT s.pid,
     w.replay_location,
     w.replay_lag,
     w.sync_priority,
-    w.sync_state
+    w.sync_state,
+    w.causal_reads_state
    FROM pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, wait_event_type, wait_event, xact_start, query_start, backend_start, state_change, client_addr, client_hostname, client_port, backend_xid, backend_xmin, ssl, sslversion, sslcipher, sslbits, sslcompression, sslclientdn),
     pg_authid u,
-    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state)
+    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state, causal_reads_state)
   WHERE ((s.usesysid = u.oid) AND (s.pid = w.pid));
 pg_stat_ssl| SELECT s.pid,
     s.ssl,
#64Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Thomas Munro (#63)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Tue, Mar 29, 2016 at 1:56 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Mon, Mar 28, 2016 at 8:54 PM, Michael Paquier
<michael.paquier@gmail.com> wrote:

I have been also thinking a lot about this patch, and the fact that
the WAL receiver latch is being used within the internals of
libpqwalreceiver has been bugging me a lot, because this makes the
wait phase happening within the libpqwalreceiver depend on something
that only the WAL receiver had a only control on up to now (among the
things thought: having a second latch for libpqwalreceiver, having an
event interface for libpqwalreceiver, switch libpq_receive into being
asynchronous...).

Yeah, it bugs me too. Do you prefer this?

int walrcv_receive(char **buffer, int *wait_fd);

Return value -1 means end-of-copy as before, return value 0 means "no
data available now, please call me again when *wait_fd is ready to
read". Then walreceiver.c can look after the WaitLatchOrSocket call
and deal with socket readiness, postmaster death, timeout and latch,
and libpqwalreceiver.c doesn't know anything about all that stuff
anymore, but it is now part of the interface that it must expose a
file descriptor for readiness testing when it doesn't have data
available.

Please find attached a new patch series which does it that way.

Oops, there is a bug in the primary disconnection case when len == 1
and it breaks out of the loop and wait_fd is invalid. I'll follow up
on that tomorrow, but I'm interested to hear your thoughts (and anyone
else's!) on that interface change and general approach.

--
Thomas Munro
http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#65Michael Paquier
michael.paquier@gmail.com
In reply to: Thomas Munro (#64)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Mon, Mar 28, 2016 at 10:08 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Tue, Mar 29, 2016 at 1:56 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Mon, Mar 28, 2016 at 8:54 PM, Michael Paquier
<michael.paquier@gmail.com> wrote:

I have been also thinking a lot about this patch, and the fact that
the WAL receiver latch is being used within the internals of
libpqwalreceiver has been bugging me a lot, because this makes the
wait phase happening within the libpqwalreceiver depend on something
that only the WAL receiver had a only control on up to now (among the
things thought: having a second latch for libpqwalreceiver, having an
event interface for libpqwalreceiver, switch libpq_receive into being
asynchronous...).

Yeah, it bugs me too. Do you prefer this?

int walrcv_receive(char **buffer, int *wait_fd);

Return value -1 means end-of-copy as before, return value 0 means "no
data available now, please call me again when *wait_fd is ready to
read". Then walreceiver.c can look after the WaitLatchOrSocket call
and deal with socket readiness, postmaster death, timeout and latch,
and libpqwalreceiver.c doesn't know anything about all that stuff
anymore, but it is now part of the interface that it must expose a
file descriptor for readiness testing when it doesn't have data
available.

Please find attached a new patch series which does it that way.

Oops, there is a bug in the primary disconnection case when len == 1
and it breaks out of the loop and wait_fd is invalid. I'll follow up
on that tomorrow, but I'm interested to hear your thoughts (and anyone
else's!) on that interface change and general approach.

I definitely prefer that, that's neater! libpq_select could be
simplified because a timeout does not matter much.
--
Michael

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#66Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Michael Paquier (#65)
4 attachment(s)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Tue, Mar 29, 2016 at 2:28 AM, Michael Paquier
<michael.paquier@gmail.com> wrote:

On Mon, Mar 28, 2016 at 10:08 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Tue, Mar 29, 2016 at 1:56 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Mon, Mar 28, 2016 at 8:54 PM, Michael Paquier
<michael.paquier@gmail.com> wrote:

I have been also thinking a lot about this patch, and the fact that
the WAL receiver latch is being used within the internals of
libpqwalreceiver has been bugging me a lot, because this makes the
wait phase happening within the libpqwalreceiver depend on something
that only the WAL receiver had a only control on up to now (among the
things thought: having a second latch for libpqwalreceiver, having an
event interface for libpqwalreceiver, switch libpq_receive into being
asynchronous...).

Yeah, it bugs me too. Do you prefer this?

int walrcv_receive(char **buffer, int *wait_fd);

Return value -1 means end-of-copy as before, return value 0 means "no
data available now, please call me again when *wait_fd is ready to
read". Then walreceiver.c can look after the WaitLatchOrSocket call
and deal with socket readiness, postmaster death, timeout and latch,
and libpqwalreceiver.c doesn't know anything about all that stuff
anymore, but it is now part of the interface that it must expose a
file descriptor for readiness testing when it doesn't have data
available.

Please find attached a new patch series which does it that way.

Oops, there is a bug in the primary disconnection case when len == 1
and it breaks out of the loop and wait_fd is invalid. I'll follow up
on that tomorrow, but I'm interested to hear your thoughts (and anyone
else's!) on that interface change and general approach.

I definitely prefer that, that's neater! libpq_select could be
simplified because a timeout does not matter much.

Ok, here is a new version that exits the streaming loop correctly when
endofwal becomes true. To hit that codepath you have to set up a
cascading standby with recovery_target_timeline = 'latest', and then
promote the standby it's talking to. I also got rid of the
PostmasterIsAlive() check which became superfluous.

You're right that libpq_select is now only ever called with timeout =
-1 so could theoretically lose the parameter, but I decided against
cluttering this patch up by touching that for now. It seems like the
only reason it's used by libpqrcv_PQexec is something to do with
interrupts on Windows, which I'm not able to test so that was another
reason not to touch it. (BTW, isn't the select call in libpq_select
lacking an exceptfds set, and can't it therefore block forever when
there is an error condition on the socket and no timeout?)

--
Thomas Munro
http://www.enterprisedb.com

Attachments:

0001-remote-apply-v9.patchapplication/octet-stream; name=0001-remote-apply-v9.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index d48a13f..6ab65a7 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -2143,8 +2143,8 @@ include_dir 'conf.d'
         Specifies whether transaction commit will wait for WAL records
         to be written to disk before the command returns a <quote>success</>
         indication to the client.  Valid values are <literal>on</>,
-        <literal>remote_write</>, <literal>local</>, and <literal>off</>.
-        The default, and safe, setting
+        <literal>remote_write</>, <literal>remote_apply</>, <literal>local</>,
+        and <literal>off</>.  The default, and safe, setting
         is <literal>on</>.  When <literal>off</>, there can be a delay between
         when success is reported to the client and when the transaction is
         really guaranteed to be safe against a server crash.  (The maximum
@@ -2177,6 +2177,10 @@ include_dir 'conf.d'
         ensure data preservation even if the standby instance of
         <productname>PostgreSQL</> were to crash, but not if the standby
         suffers an operating-system-level crash.
+        When set to <literal>remote_apply</>, commits will wait until a reply
+        from the current synchronous standby indicates it has received the
+        commit record of the transaction and applied it, so that it has become
+        visible to queries.
        </para>
        <para>
         When synchronous
diff --git a/doc/src/sgml/high-availability.sgml b/doc/src/sgml/high-availability.sgml
index 19d613e..03c6c30 100644
--- a/doc/src/sgml/high-availability.sgml
+++ b/doc/src/sgml/high-availability.sgml
@@ -1081,6 +1081,9 @@ primary_slot_name = 'node_a_slot'
     WAL record is then sent to the standby. The standby sends reply
     messages each time a new batch of WAL data is written to disk, unless
     <varname>wal_receiver_status_interval</> is set to zero on the standby.
+    In the case that <varname>synchronous_commit</> is set to
+    <literal>remote_apply</>, the standby sends reply messages when the commit
+    record is replayed, making the transaction visible.
     If the standby is the first matching standby, as specified in
     <varname>synchronous_standby_names</> on the primary, the reply
     messages from that standby will be used to wake users waiting for
@@ -1107,6 +1110,14 @@ primary_slot_name = 'node_a_slot'
    </para>
 
    <para>
+    Setting <varname>synchronous_commit</> to <literal>remote_apply</> will
+    cause each commit to wait until the current synchronous standby reports
+    that it has replayed the transaction, making it visible to user queries.
+    In simple cases, this allows for load balancing with causal consistency
+    on a single hot standby.
+   </para>
+
+   <para>
     Users will stop waiting if a fast shutdown is requested.  However, as
     when using asynchronous replication, the server will not fully
     shutdown until all outstanding WAL records are transferred to the currently
@@ -1160,9 +1171,10 @@ primary_slot_name = 'node_a_slot'
     <title>Planning for High Availability</title>
 
    <para>
-    Commits made when <varname>synchronous_commit</> is set to <literal>on</>
-    or <literal>remote_write</> will wait until the synchronous standby responds. The response
-    may never occur if the last, or only, standby should crash.
+    Commits made when <varname>synchronous_commit</> is set to <literal>on</>,
+    <literal>remote_write</> or <literal>remote_apply</> will wait until the
+    synchronous standby responds. The response may never occur if the last, or
+    only, standby should crash.
    </para>
 
    <para>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index e7234c8..893c2fa 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -1107,7 +1107,7 @@ EndPrepare(GlobalTransaction gxact)
 	 * Note that at this stage we have marked the prepare, but still show as
 	 * running in the procarray (twice!) and continue to hold locks.
 	 */
-	SyncRepWaitForLSN(gxact->prepare_end_lsn);
+	SyncRepWaitForLSN(gxact->prepare_end_lsn, false);
 
 	records.tail = records.head = NULL;
 	records.num_chunks = 0;
@@ -2103,7 +2103,7 @@ RecordTransactionCommitPrepared(TransactionId xid,
 	 * Note that at this stage we have marked clog, but still show as running
 	 * in the procarray and continue to hold locks.
 	 */
-	SyncRepWaitForLSN(recptr);
+	SyncRepWaitForLSN(recptr, true);
 }
 
 /*
@@ -2156,5 +2156,5 @@ RecordTransactionAbortPrepared(TransactionId xid,
 	 * Note that at this stage we have marked clog, but still show as running
 	 * in the procarray and continue to hold locks.
 	 */
-	SyncRepWaitForLSN(recptr);
+	SyncRepWaitForLSN(recptr, true);
 }
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index e315405..da595fd 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,7 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
-		SyncRepWaitForLSN(XactLastRecEnd);
+		SyncRepWaitForLSN(XactLastRecEnd, true);
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5123,6 +5123,13 @@ XactLogCommitRecord(TimestampTz commit_time,
 		xl_xinfo.xinfo |= XACT_COMPLETION_FORCE_SYNC_COMMIT;
 
 	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this commit is applied.
+	 */
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
+	/*
 	 * Relcache invalidations requires information about the current database
 	 * and so does logical decoding.
 	 */
@@ -5300,6 +5307,13 @@ XactLogAbortRecord(TimestampTz abort_time,
 	if (xl_xinfo.xinfo & XACT_XINFO_HAS_TWOPHASE)
 		XLogRegisterData((char *) (&xl_twophase), sizeof(xl_xact_twophase));
 
+	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this abort is applied.
+	 */
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
+
 	return XLogInsert(RM_XACT_ID, info);
 }
 
@@ -5459,6 +5473,13 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 	if (XactCompletionForceSyncCommit(parsed->xinfo))
 		XLogFlush(lsn);
 
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit = remote_apply), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 /*
@@ -5545,6 +5566,14 @@ xact_redo_abort(xl_xact_parsed_abort *parsed, TransactionId xid)
 		smgrdounlink(srel, true);
 		smgrclose(srel);
 	}
+
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit = remote_apply), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionSyncApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 void
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index b119a47..3e454f5 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -345,6 +345,9 @@ static XLogRecPtr RedoRecPtr;
  */
 static bool doPageWrites;
 
+/* Has the recovery code requested a walreceiver wakeup? */
+static bool doRequestWalReceiverReply;
+
 /*
  * RedoStartLSN points to the checkpoint's REDO location which is specified
  * in a backup label file, backup history file or control file. In standby
@@ -6879,6 +6882,19 @@ StartupXLOG(void)
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
 				SpinLockRelease(&XLogCtl->info_lck);
 
+				/*
+				 * If rm_redo reported that it applied a commit record that
+				 * the master is waiting for by calling
+				 * XLogRequestWalReceiverReply, then we wake up the receiver
+				 * so that it notices the updated lastReplayedEndRecPtr and
+				 * sends a reply to the master.
+				 */
+				if (doRequestWalReceiverReply)
+				{
+					doRequestWalReceiverReply = false;
+					WalRcvWakeup();
+				}
+
 				/* Remember this record as the last-applied one */
 				LastRec = ReadRecPtr;
 
@@ -11594,3 +11610,12 @@ SetWalWriterSleeping(bool sleeping)
 	XLogCtl->WalWriterSleeping = sleeping;
 	SpinLockRelease(&XLogCtl->info_lck);
 }
+
+/*
+ * Schedule a walreceiver wakeup in the main recovery loop.
+ */
+void
+XLogRequestWalReceiverReply(void)
+{
+	doRequestWalReceiverReply = true;
+}
diff --git a/src/backend/replication/README b/src/backend/replication/README
index 8e5bf0d..419a2d7 100644
--- a/src/backend/replication/README
+++ b/src/backend/replication/README
@@ -16,14 +16,16 @@ bool walrcv_connect(char *conninfo, XLogRecPtr startpoint)
 Establish connection to the primary, and starts streaming from 'startpoint'.
 Returns true on success.
 
-bool walrcv_receive(int timeout, unsigned char *type, char **buffer, int *len)
-
-Retrieve any message available through the connection, blocking for
-maximum of 'timeout' ms. If a message was successfully read, returns true,
-otherwise false. On success, a pointer to the message payload is stored in
-*buffer, length in *len, and the type of message received in *type. The
-returned buffer is valid until the next call to walrcv_* functions, the
-caller should not attempt freeing it.
+int walrcv_receive(char **buffer, int *wait_fd)
+
+Retrieve any message available without blocking through the
+connection.  If a message was successfully read, returns its
+length. If the connection is closed, returns -1.  Otherwise returns 0
+to indicate that no data is available, and sets *wait_fd to a file
+descriptor which can be waited on before trying again.  On success, a
+pointer to the message payload is stored in *buffer. The returned
+buffer is valid until the next call to walrcv_* functions, and the
+caller should not attempt to free it.
 
 void walrcv_send(const char *buffer, int nbytes)
 
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index 4ee4d71..a3bec49 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -52,7 +52,7 @@ static void libpqrcv_readtimelinehistoryfile(TimeLineID tli, char **filename, ch
 static bool libpqrcv_startstreaming(TimeLineID tli, XLogRecPtr startpoint,
 						char *slotname);
 static void libpqrcv_endstreaming(TimeLineID *next_tli);
-static int	libpqrcv_receive(int timeout, char **buffer);
+static int	libpqrcv_receive(char **buffer, int *wait_fd);
 static void libpqrcv_send(const char *buffer, int nbytes);
 static void libpqrcv_disconnect(void);
 
@@ -463,8 +463,7 @@ libpqrcv_disconnect(void)
 }
 
 /*
- * Receive a message available from XLOG stream, blocking for
- * maximum of 'timeout' ms.
+ * Receive a message available from XLOG stream.
  *
  * Returns:
  *
@@ -472,15 +471,15 @@ libpqrcv_disconnect(void)
  *	 point to a buffer holding the received message. The buffer is only valid
  *	 until the next libpqrcv_* call.
  *
- *	 0 if no data was available within timeout, or wait was interrupted
- *	 by signal.
+ *	 If no data was available immediately, returns 0, and *wait_fd is set to a
+ *	 file descriptor which can be waited on before trying again.
  *
  *	 -1 if the server ended the COPY.
  *
  * ereports on error.
  */
 static int
-libpqrcv_receive(int timeout, char **buffer)
+libpqrcv_receive(char **buffer, int *wait_fd)
 {
 	int			rawlen;
 
@@ -492,16 +491,7 @@ libpqrcv_receive(int timeout, char **buffer)
 	rawlen = PQgetCopyData(streamConn, &recvBuf, 1);
 	if (rawlen == 0)
 	{
-		/*
-		 * No data available yet. If the caller requested to block, wait for
-		 * more data to arrive.
-		 */
-		if (timeout > 0)
-		{
-			if (!libpq_select(timeout))
-				return 0;
-		}
-
+		/* Try consuming some data. */
 		if (PQconsumeInput(streamConn) == 0)
 			ereport(ERROR,
 					(errmsg("could not receive data from WAL stream: %s",
@@ -510,7 +500,11 @@ libpqrcv_receive(int timeout, char **buffer)
 		/* Now that we've consumed some input, try again */
 		rawlen = PQgetCopyData(streamConn, &recvBuf, 1);
 		if (rawlen == 0)
+		{
+			/* Tell caller to try again when our socket is ready. */
+			*wait_fd = PQsocket(streamConn);
 			return 0;
+		}
 	}
 	if (rawlen == -1)			/* end-of-streaming or error */
 	{
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 92faf4e..1ee1bc5 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -91,13 +91,25 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * to the wait queue. During SyncRepWakeQueue() a WALSender changes
  * the state to SYNC_REP_WAIT_COMPLETE once replication is confirmed.
  * This backend then resets its state to SYNC_REP_NOT_WAITING.
+ *
+ * 'lsn' represents the LSN to wait for.  'commit' indicates whether this LSN
+ * represents a commit/abort record.  If it's not, then we wait only for the
+ * WAL to be flushed if synchronous_commit is set to the higher level of
+ * remote_apply, because standbys only send apply feedback for commit/abort
+ * records.
  */
 void
-SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
+SyncRepWaitForLSN(XLogRecPtr lsn, bool commit)
 {
 	char	   *new_status = NULL;
 	const char *old_status;
-	int			mode = SyncRepWaitMode;
+	int			mode;
+
+	/* Cap the level for non-commit records to remote flush only. */
+	if (commit)
+		mode = SyncRepWaitMode;
+	else
+		mode = Max(SyncRepWaitMode, SYNC_REP_WAIT_FLUSH);
 
 	/*
 	 * Fast exit if user has not requested sync replication, or there are no
@@ -122,7 +134,7 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 	 * to be a low cost check.
 	 */
 	if (!WalSndCtl->sync_standbys_defined ||
-		XactCommitLSN <= WalSndCtl->lsn[mode])
+		lsn <= WalSndCtl->lsn[mode])
 	{
 		LWLockRelease(SyncRepLock);
 		return;
@@ -132,7 +144,7 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 	 * Set our waitLSN so WALSender will know when to wake us, and add
 	 * ourselves to the queue.
 	 */
-	MyProc->waitLSN = XactCommitLSN;
+	MyProc->waitLSN = lsn;
 	MyProc->syncRepState = SYNC_REP_WAITING;
 	SyncRepQueueInsert(mode);
 	Assert(SyncRepQueueIsOrderedByLSN(mode));
@@ -147,7 +159,7 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 		new_status = (char *) palloc(len + 32 + 1);
 		memcpy(new_status, old_status, len);
 		sprintf(new_status + len, " waiting for %X/%X",
-				(uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN);
+				(uint32) (lsn >> 32), (uint32) lsn);
 		set_ps_display(new_status, false);
 		new_status[len] = '\0'; /* truncate off " waiting ..." */
 	}
@@ -416,6 +428,7 @@ SyncRepReleaseWaiters(void)
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
+	int			numapply = 0;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
@@ -462,12 +475,18 @@ SyncRepReleaseWaiters(void)
 		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
 		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
 	}
+	if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+	{
+		walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+		numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY);
+	}
 
 	LWLockRelease(SyncRepLock);
 
-	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X",
+	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%x",
 		 numwrite, (uint32) (MyWalSnd->write >> 32), (uint32) MyWalSnd->write,
-	   numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush);
+		 numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush,
+		 numapply, (uint32) (MyWalSnd->apply >> 32), (uint32) MyWalSnd->apply);
 
 	/*
 	 * If we are managing the highest priority standby, though we weren't
@@ -728,6 +747,9 @@ assign_synchronous_commit(int newval, void *extra)
 		case SYNCHRONOUS_COMMIT_REMOTE_FLUSH:
 			SyncRepWaitMode = SYNC_REP_WAIT_FLUSH;
 			break;
+		case SYNCHRONOUS_COMMIT_REMOTE_APPLY:
+			SyncRepWaitMode = SYNC_REP_WAIT_APPLY;
+			break;
 		default:
 			SyncRepWaitMode = SYNC_REP_NO_WAIT;
 			break;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 7b36e02..d1c703c 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -352,8 +352,6 @@ WalReceiverMain(void)
 		if (walrcv_startstreaming(startpointTLI, startpoint,
 								  slotname[0] != '\0' ? slotname : NULL))
 		{
-			bool		endofwal = false;
-
 			if (first_stream)
 				ereport(LOG,
 						(errmsg("started streaming WAL from primary at %X/%X on timeline %u",
@@ -376,18 +374,13 @@ WalReceiverMain(void)
 			ping_sent = false;
 
 			/* Loop until end-of-streaming or error */
-			while (!endofwal)
+			for (;;)
 			{
 				char	   *buf;
 				int			len;
-
-				/*
-				 * Emergency bailout if postmaster has died.  This is to avoid
-				 * the necessity for manual cleanup of all postmaster
-				 * children.
-				 */
-				if (!PostmasterIsAlive())
-					exit(1);
+				bool		endofwal = false;
+				int			wait_fd = PGINVALID_SOCKET;
+				int			rc;
 
 				/*
 				 * Exit walreceiver if we're not in recovery. This should not
@@ -407,8 +400,8 @@ WalReceiverMain(void)
 					XLogWalRcvSendHSFeedback(true);
 				}
 
-				/* Wait a while for data to arrive */
-				len = walrcv_receive(NAPTIME_PER_CYCLE, &buf);
+				/* See if we can read data immediately */
+				len = walrcv_receive(&buf, &wait_fd);
 				if (len != 0)
 				{
 					/*
@@ -439,7 +432,7 @@ WalReceiverMain(void)
 							endofwal = true;
 							break;
 						}
-						len = walrcv_receive(0, &buf);
+						len = walrcv_receive(&buf, &wait_fd);
 					}
 
 					/* Let the master know that we received some data. */
@@ -452,7 +445,44 @@ WalReceiverMain(void)
 					 */
 					XLogWalRcvFlush(false);
 				}
-				else
+
+				/* Check if we need to exit the streaming loop. */
+				if (endofwal)
+					break;
+
+				/*
+				 * Ideally we would reuse a WaitEventSet object repeatedly
+				 * here to avoid the overheads of WaitLatchOrSocket on epoll
+				 * systems, but we can't be sure that libpq (or any other
+				 * walreceiver implementation) has the same socket (even if
+				 * the fd is the same number, it may have been closed and
+				 * reopened since the last time).  In future, if there is a
+				 * function for removing sockets from WaitEventSet, then we
+				 * could add and remove just the socket each time, potentially
+				 * avoiding some system calls.
+				 */
+				Assert(wait_fd != PGINVALID_SOCKET);
+				rc = WaitLatchOrSocket(&walrcv->latch,
+									   WL_POSTMASTER_DEATH | WL_SOCKET_READABLE |
+									   WL_TIMEOUT | WL_LATCH_SET,
+									   wait_fd,
+									   NAPTIME_PER_CYCLE);
+				if (rc & WL_LATCH_SET)
+				{
+					/* The recovery process has asked us to send apply feedback now. */
+					ResetLatch(&walrcv->latch);
+					XLogWalRcvSendReply(true, false);
+				}
+				if (rc & WL_POSTMASTER_DEATH)
+				{
+					/*
+					 * Emergency bailout if postmaster has died.  This is to
+					 * avoid the necessity for manual cleanup of all
+					 * postmaster children.
+					 */
+					exit(1);
+				}
+				if (rc & WL_TIMEOUT)
 				{
 					/*
 					 * We didn't receive anything new. If we haven't heard
@@ -1222,6 +1252,20 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 }
 
 /*
+ * Wake up the walreceiver main loop.
+ *
+ * This is called by the startup process whenever interesting xlog records
+ * are applied, so that walreceiver can check if it needs to send an apply
+ * notification back to the master which may be waiting in a COMMIT with
+ * synchronous_commit = remote_apply.
+ */
+void
+WalRcvWakeup(void)
+{
+	SetLatch(&WalRcv->latch);
+}
+
+/*
  * Return a string constant representing the state. This is used
  * in system functions and views, and should *not* be translated.
  */
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 65a6cd4..06cb166 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -345,12 +345,13 @@ static const struct config_enum_entry constraint_exclusion_options[] = {
 };
 
 /*
- * Although only "on", "off", "remote_write", and "local" are documented, we
- * accept all the likely variants of "on" and "off".
+ * Although only "on", "off", "remote_apply", "remote_write", and "local" are
+ * documented, we accept all the likely variants of "on" and "off".
  */
 static const struct config_enum_entry synchronous_commit_options[] = {
 	{"local", SYNCHRONOUS_COMMIT_LOCAL_FLUSH, false},
 	{"remote_write", SYNCHRONOUS_COMMIT_REMOTE_WRITE, false},
+	{"remote_apply", SYNCHRONOUS_COMMIT_REMOTE_APPLY, false},
 	{"on", SYNCHRONOUS_COMMIT_ON, false},
 	{"off", SYNCHRONOUS_COMMIT_OFF, false},
 	{"true", SYNCHRONOUS_COMMIT_ON, true},
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index 5536012..ec4427f 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -177,7 +177,7 @@
 					# (change requires restart)
 #fsync = on				# turns forced synchronization on or off
 #synchronous_commit = on		# synchronization level;
-					# off, local, remote_write, or on
+					# off, local, remote_write, remote_apply, or on
 #wal_sync_method = fsync		# the default is the first option
 					# supported by the operating system:
 					#   open_datasync
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 3ba23f5..21fcea1 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -60,7 +60,9 @@ typedef enum
 	SYNCHRONOUS_COMMIT_LOCAL_FLUSH,		/* wait for local flush only */
 	SYNCHRONOUS_COMMIT_REMOTE_WRITE,	/* wait for local flush and remote
 										 * write */
-	SYNCHRONOUS_COMMIT_REMOTE_FLUSH		/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_FLUSH,	/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_APPLY		/* wait for local flush and remote
+										 * apply */
 }	SyncCommitLevel;
 
 /* Define the default setting for synchonous_commit */
@@ -144,10 +146,13 @@ typedef void (*SubXactCallback) (SubXactEvent event, SubTransactionId mySubid,
  * EOXact... routines which run at the end of the original transaction
  * completion.
  */
+#define XACT_COMPLETION_SYNC_APPLY_FEEDBACK		(1U << 29)
 #define XACT_COMPLETION_UPDATE_RELCACHE_FILE	(1U << 30)
 #define XACT_COMPLETION_FORCE_SYNC_COMMIT		(1U << 31)
 
 /* Access macros for above flags */
+#define XactCompletionSyncApplyFeedback(xinfo) \
+	((xinfo & XACT_COMPLETION_SYNC_APPLY_FEEDBACK) != 0)
 #define XactCompletionRelcacheInitFileInval(xinfo) \
 	((xinfo & XACT_COMPLETION_UPDATE_RELCACHE_FILE) != 0)
 #define XactCompletionForceSyncCommit(xinfo) \
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 74a1394..a7dcdae 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -267,6 +267,8 @@ extern bool CheckPromoteSignal(void);
 extern void WakeupRecovery(void);
 extern void SetWalWriterSleeping(bool sleeping);
 
+extern void XLogRequestWalReceiverReply(void);
+
 extern void assign_max_wal_size(int newval, void *extra);
 extern void assign_checkpoint_completion_target(double newval, void *extra);
 
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index 96e059b..c005a42 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -23,8 +23,9 @@
 #define SYNC_REP_NO_WAIT		-1
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
+#define SYNC_REP_WAIT_APPLY		2
 
-#define NUM_SYNC_REP_WAIT_MODE	2
+#define NUM_SYNC_REP_WAIT_MODE	3
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
@@ -35,7 +36,7 @@
 extern char *SyncRepStandbyNames;
 
 /* called by user backend */
-extern void SyncRepWaitForLSN(XLogRecPtr XactCommitLSN);
+extern void SyncRepWaitForLSN(XLogRecPtr lsn, bool commit);
 
 /* called at backend exit */
 extern void SyncRepCleanupAtProcExit(void);
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 6eacb09..1c24563 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -115,7 +115,8 @@ typedef struct
 	/*
 	 * Latch used by startup process to wake up walreceiver after telling it
 	 * where to start streaming (after setting receiveStart and
-	 * receiveStartTLI).
+	 * receiveStartTLI), and also to tell it to send apply feedback to the
+	 * primary whenever specially marked commit records are applied.
 	 */
 	Latch		latch;
 } WalRcvData;
@@ -138,7 +139,7 @@ extern PGDLLIMPORT walrcv_startstreaming_type walrcv_startstreaming;
 typedef void (*walrcv_endstreaming_type) (TimeLineID *next_tli);
 extern PGDLLIMPORT walrcv_endstreaming_type walrcv_endstreaming;
 
-typedef int (*walrcv_receive_type) (int timeout, char **buffer);
+typedef int (*walrcv_receive_type) (char **buffer, int *wait_fd);
 extern PGDLLIMPORT walrcv_receive_type walrcv_receive;
 
 typedef void (*walrcv_send_type) (const char *buffer, int nbytes);
@@ -162,5 +163,6 @@ extern void RequestXLogStreaming(TimeLineID tli, XLogRecPtr recptr,
 extern XLogRecPtr GetWalRcvWriteRecPtr(XLogRecPtr *latestChunkStart, TimeLineID *receiveTLI);
 extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
+extern void WalRcvWakeup(void);
 
 #endif   /* _WALRECEIVER_H */
0002-replay-lag-v9.patchapplication/octet-stream; name=0002-replay-lag-v9.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 105d541..7d63782 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1208,6 +1208,12 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
       standby server</entry>
     </row>
     <row>
+     <entry><structfield>replay_lag</></entry>
+     <entry><type>interval</></entry>
+     <entry>Estimated time taken for recent WAL records to be replayed on this
+      standby server</entry>
+    </row>
+    <row>
      <entry><structfield>sync_priority</></entry>
      <entry><type>integer</></entry>
      <entry>Priority of this standby server for being chosen as the
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index da595fd..b0464e7 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -5474,6 +5474,12 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 		XLogFlush(lsn);
 
 	/*
+	 * Record the primary's timestamp for the commit record, so it can be used
+	 * for tracking replay lag.
+	 */
+	SetXLogReplayTimestamp(parsed->xact_time);
+
+	/*
 	 * If asked by the primary (because someone is waiting for a synchronous
 	 * commit = remote_apply), we will need to ask walreceiver to send a
 	 * reply immediately.
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 3e454f5..504b4df 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -81,6 +81,8 @@ extern uint32 bootstrap_data_checksum_version;
 #define PROMOTE_SIGNAL_FILE		"promote"
 #define FALLBACK_PROMOTE_SIGNAL_FILE "fallback_promote"
 
+/* Size of the circular buffer of timestamped LSNs. */
+#define MAX_TIMESTAMPED_LSNS 8192
 
 /* User-settable parameters */
 int			max_wal_size = 64;	/* 1 GB */
@@ -360,6 +362,13 @@ static bool doRequestWalReceiverReply;
  */
 static XLogRecPtr RedoStartLSN = InvalidXLogRecPtr;
 
+/*
+ * LastReplayedTimestamp can be set by redo handlers when they apply a record
+ * that carries a timestamp, by calling SetXLogReplayedTimestamp.  The xlog
+ * apply loop can then update the value in shared memory.
+ */
+static TimestampTz LastReplayedTimestamp = 0;
+
 /*----------
  * Shared-memory data structures for XLOG control
  *
@@ -634,6 +643,21 @@ typedef struct XLogCtlData
 	/* current effective recovery target timeline */
 	TimeLineID	RecoveryTargetTLI;
 
+	/* timestamp from the most recently applied record associated with a timestamp. */
+	TimestampTz lastReplayedTimestamp;
+
+	/*
+	 * We maintain a circular buffer of LSNs and associated timestamps.
+	 * Walreceiver writes into it using information from timestamps, and the
+	 * startup recovery process reads from it and notifies walreceiver when
+	 * LSNs are replayed so that the timestamps can eventually be fed back to
+	 * the upstream server, to track lag.
+	 */
+	Index			timestampedLsnRead;
+	Index			timestampedLsnWrite;
+	XLogRecPtr		timestampedLsn[MAX_TIMESTAMPED_LSNS];
+	TimestampTz		timestampedLsnTime[MAX_TIMESTAMPED_LSNS];
+
 	/*
 	 * timestamp of when we started replaying the current chunk of WAL data,
 	 * only relevant for replication or archive recovery
@@ -6874,20 +6898,51 @@ StartupXLOG(void)
 				error_context_stack = errcallback.previous;
 
 				/*
-				 * Update lastReplayedEndRecPtr after this record has been
-				 * successfully replayed.
+				 * Update lastReplayedEndRecPtr and lastReplayedTimestamp
+				 * after this record has been successfully replayed.
 				 */
 				SpinLockAcquire(&XLogCtl->info_lck);
 				XLogCtl->lastReplayedEndRecPtr = EndRecPtr;
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
+				if (LastReplayedTimestamp != 0)
+				{
+					/* If replaying a record produced a timestamp, use that. */
+					XLogCtl->lastReplayedTimestamp = LastReplayedTimestamp;
+					LastReplayedTimestamp = 0;
+				}
+				else
+				{
+					/*
+					 * If we have applied LSNs associated with timestamps
+					 * received by walreceiver, then use the recorded
+					 * timestamp.  We consume from the read end of the
+					 * circular buffer.
+					 */
+					while (XLogCtl->timestampedLsnRead !=
+						   XLogCtl->timestampedLsnWrite &&
+						   XLogCtl->timestampedLsn[XLogCtl->timestampedLsnRead]
+						   <= EndRecPtr)
+					{
+						if (XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead] >
+							XLogCtl->lastReplayedTimestamp)
+						{
+							XLogCtl->lastReplayedTimestamp =
+								XLogCtl->timestampedLsnTime[XLogCtl->timestampedLsnRead];
+							doRequestWalReceiverReply = true;
+						}
+						XLogCtl->timestampedLsnRead =
+							(XLogCtl->timestampedLsnRead + 1) % MAX_TIMESTAMPED_LSNS;
+					}
+				}
 				SpinLockRelease(&XLogCtl->info_lck);
 
 				/*
 				 * If rm_redo reported that it applied a commit record that
 				 * the master is waiting for by calling
-				 * XLogRequestWalReceiverReply, then we wake up the receiver
-				 * so that it notices the updated lastReplayedEndRecPtr and
-				 * sends a reply to the master.
+				 * XLogRequestWalReceiverReply, or we encountered a WAL
+				 * location that was associated with a timestamp above, then
+				 * we wake up the receiver so that it notices the updated
+				 * lastReplayedEndRecPtr and sends a reply to the master.
 				 */
 				if (doRequestWalReceiverReply)
 				{
@@ -11619,3 +11674,91 @@ XLogRequestWalReceiverReply(void)
 {
 	doRequestWalReceiverReply = true;
 }
+
+/*
+ * Record the timestamp that is associated with a WAL position.
+ *
+ * This is called by walreceiver on standby servers when keepalive messages
+ * arrive, using timestamps generated on the primary server.  The timestamp
+ * will be sent back to the primary server when the standby had applied this
+ * WAL position.  The primary can use the elapsed time to estimate the replay
+ * lag.
+ */
+void
+SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn)
+{
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn == XLogCtl->lastReplayedEndRecPtr)
+	{
+		/*
+		 * That is the last replayed LSN: we are fully replayed, so we can
+		 * update the replay timestamp immediately.
+		 */
+		XLogCtl->lastReplayedTimestamp = timestamp;
+	}
+	else
+	{
+		/*
+		 * There is WAL still to be applied.  We will associate the timestamp
+		 * with this WAL position and wait for it to be replayed.  We add it
+		 * at the 'write' end of the circular buffer of LSN/timestamp
+		 * mappings, which the replay loop will eventually read.
+		 */
+		Index w = XLogCtl->timestampedLsnWrite;
+		Index r = XLogCtl->timestampedLsnRead;
+
+		XLogCtl->timestampedLsn[w] = lsn;
+		XLogCtl->timestampedLsnTime[w] = timestamp;
+
+		/* Advance the write point. */
+		w = (w + 1) % MAX_TIMESTAMPED_LSNS;
+		XLogCtl->timestampedLsnWrite = w;
+		if (w == r)
+		{
+			/*
+			 * The buffer is full.  Advance the read point (throwing away
+			 * oldest values; we will begin to overestimate replay lag, until
+			 * lag decreases to a size our buffer can manage, or the next
+			 * commit record is replayed).
+			 */
+			r = (r + 1) % MAX_TIMESTAMPED_LSNS;
+			XLogCtl->timestampedLsnRead = r;
+		}
+	}
+	SpinLockRelease(&XLogCtl->info_lck);
+}
+
+/*
+ * Set the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary.  This can be called by redo handlers that have
+ * an appropriate timestamp (currently only commit records).  Updating the
+ * shared memory value is deferred until after the redo handler returns.
+ */
+void
+SetXLogReplayTimestamp(TimestampTz timestamp)
+{
+	LastReplayedTimestamp = timestamp;
+}
+
+/*
+ * Get the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary, and also the most recently applied LSN.  (Note
+ * that the timestamp and the LSN don't necessarily relate to the same
+ * record.)
+ *
+ * This is similar to GetLatestXTime, except that it is not only advanced by
+ * commit records (see SetXLogReplayTimestampAtLsn).
+ */
+TimestampTz
+GetXLogReplayTimestamp(XLogRecPtr *lsn)
+{
+	TimestampTz result;
+
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn)
+		*lsn = XLogCtl->lastReplayedEndRecPtr;
+	result = XLogCtl->lastReplayedTimestamp;
+	SpinLockRelease(&XLogCtl->info_lck);
+
+	return result;
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 9ae1ef4..a53f07b 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -662,6 +662,7 @@ CREATE VIEW pg_stat_replication AS
             W.write_location,
             W.flush_location,
             W.replay_location,
+            W.replay_lag,
             W.sync_priority,
             W.sync_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index d1c703c..aff3fdb 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -85,6 +85,8 @@ walrcv_disconnect_type walrcv_disconnect = NULL;
 
 #define NAPTIME_PER_CYCLE 100	/* max sleep time between cycles (100ms) */
 
+#define MIN_TIME_BETWEEN_TIMESTAMPED_LSNS 1000 /* 1s */
+
 /*
  * These variables are used similarly to openLogFile/SegNo/Off,
  * but for walreceiver to write the XLOG. recvFileTLI is the TimeLineID
@@ -102,6 +104,8 @@ static uint32 recvOff = 0;
 static volatile sig_atomic_t got_SIGHUP = false;
 static volatile sig_atomic_t got_SIGTERM = false;
 
+static bool recovery_active = false;
+
 /*
  * LogstreamResult indicates the byte positions that we have already
  * written/fsynced.
@@ -143,7 +147,7 @@ static void WalRcvDie(int code, Datum arg);
 static void XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len);
 static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
-static void XLogWalRcvSendReply(bool force, bool requestReply);
+static void XLogWalRcvSendReply(bool force, bool requestReply, bool includeApplyTimestamp);
 static void XLogWalRcvSendHSFeedback(bool immed);
 static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
 
@@ -436,7 +440,7 @@ WalReceiverMain(void)
 					}
 
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(false, false);
+					XLogWalRcvSendReply(false, false, false);
 
 					/*
 					 * If we've written some records, flush them to disk and
@@ -471,7 +475,8 @@ WalReceiverMain(void)
 				{
 					/* The recovery process has asked us to send apply feedback now. */
 					ResetLatch(&walrcv->latch);
-					XLogWalRcvSendReply(true, false);
+					XLogWalRcvSendReply(true, false, true);
+					recovery_active = true;
 				}
 				if (rc & WL_POSTMASTER_DEATH)
 				{
@@ -528,7 +533,7 @@ WalReceiverMain(void)
 						}
 					}
 
-					XLogWalRcvSendReply(requestReply, requestReply);
+					XLogWalRcvSendReply(requestReply, requestReply, false);
 					XLogWalRcvSendHSFeedback(false);
 				}
 			}
@@ -859,6 +864,8 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 			}
 		case 'k':				/* Keepalive */
 			{
+				bool reportApplyTimestamp = false;
+
 				/* copy message to StringInfo */
 				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
 				if (len != hdrlen)
@@ -875,9 +882,22 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 
 				ProcessWalSndrMessage(walEnd, sendTime);
 
+				/*
+				 * If no apply timestamps have been sent at the request of the
+				 * recovery process since we last received a keepalive, then
+				 * we will send one now.  This allows us to feed back
+				 * timestamps in response to pings if we are idle or if the
+				 * recovery process is somehow blocked, but we don't want to
+				 * do that if it's actively applying and periodically waking
+				 * us up with accurate apply timestamps.
+				 */
+				if (!recovery_active)
+					reportApplyTimestamp = true;
+				recovery_active = false;
+
 				/* If the primary requested a reply, send one immediately */
-				if (replyRequested)
-					XLogWalRcvSendReply(true, false);
+				if (replyRequested || reportApplyTimestamp)
+					XLogWalRcvSendReply(true, false, reportApplyTimestamp);
 				break;
 			}
 		default:
@@ -1040,7 +1060,7 @@ XLogWalRcvFlush(bool dying)
 		/* Also let the master know that we made some progress */
 		if (!dying)
 		{
-			XLogWalRcvSendReply(false, false);
+			XLogWalRcvSendReply(false, false, false);
 			XLogWalRcvSendHSFeedback(false);
 		}
 	}
@@ -1058,15 +1078,18 @@ XLogWalRcvFlush(bool dying)
  * If 'requestReply' is true, requests the server to reply immediately upon
  * receiving this message. This is used for heartbearts, when approaching
  * wal_receiver_timeout.
+ *
+ * If 'reportApplyTimestamp' is true, the latest apply timestamp is included.
  */
 static void
-XLogWalRcvSendReply(bool force, bool requestReply)
+XLogWalRcvSendReply(bool force, bool requestReply, bool reportApplyTimestamp)
 {
 	static XLogRecPtr writePtr = 0;
 	static XLogRecPtr flushPtr = 0;
 	XLogRecPtr	applyPtr;
 	static TimestampTz sendTime = 0;
 	TimestampTz now;
+	TimestampTz applyTimestamp = 0;
 
 	/*
 	 * If the user doesn't want status to be reported to the master, be sure
@@ -1082,10 +1105,8 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	 * We can compare the write and flush positions to the last message we
 	 * sent without taking any lock, but the apply position requires a spin
 	 * lock, so we don't check that unless something else has changed or 10
-	 * seconds have passed.  This means that the apply log position will
-	 * appear, from the master's point of view, to lag slightly, but since
-	 * this is only for reporting purposes and only on idle systems, that's
-	 * probably OK.
+	 * seconds have passed, or the force flag has been set (which happens when
+	 * apply feedback has been requested by the primary).
 	 */
 	if (!force
 		&& writePtr == LogstreamResult.Write
@@ -1098,7 +1119,10 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	/* Construct a new message */
 	writePtr = LogstreamResult.Write;
 	flushPtr = LogstreamResult.Flush;
-	applyPtr = GetXLogReplayRecPtr(NULL);
+	if (reportApplyTimestamp)
+		applyTimestamp = GetXLogReplayTimestamp(&applyPtr);
+	else
+		applyPtr = GetXLogReplayRecPtr(NULL);
 
 	resetStringInfo(&reply_message);
 	pq_sendbyte(&reply_message, 'r');
@@ -1106,6 +1130,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	pq_sendint64(&reply_message, flushPtr);
 	pq_sendint64(&reply_message, applyPtr);
 	pq_sendint64(&reply_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&reply_message, TimestampTzToIntegerTimestamp(applyTimestamp));
 	pq_sendbyte(&reply_message, requestReply ? 1 : 0);
 
 	/* Send it */
@@ -1210,8 +1235,8 @@ static void
 ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 {
 	WalRcvData *walrcv = WalRcv;
-
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
+	static TimestampTz lastRecordedTimestamp = 0;
 
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
@@ -1222,6 +1247,18 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
 	SpinLockRelease(&walrcv->mutex);
 
+	/*
+	 * Remember primary's timestamp at this WAL location.  We throw away
+	 * samples if they are coming too fast because we don't want to fill up
+	 * the finite circular buffer and have to throw away older samples.
+	 */
+	if (lastRecordedTimestamp < TimestampTzPlusMilliseconds(sendTime,
+															-MIN_TIME_BETWEEN_TIMESTAMPED_LSNS))
+	{
+		SetXLogReplayTimestampAtLsn(sendTime, walEnd);
+		lastRecordedTimestamp = sendTime;
+	}
+
 	if (log_min_messages <= DEBUG2)
 	{
 		char	   *sendtime;
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index f98475c..16d7abc 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1545,15 +1545,29 @@ ProcessStandbyReplyMessage(void)
 	XLogRecPtr	writePtr,
 				flushPtr,
 				applyPtr;
+	int64		applyLagUs;
 	bool		replyRequested;
+	TimestampTz now = GetCurrentTimestamp();
+	TimestampTz applyTimestamp;
 
 	/* the caller already consumed the msgtype byte */
 	writePtr = pq_getmsgint64(&reply_message);
 	flushPtr = pq_getmsgint64(&reply_message);
 	applyPtr = pq_getmsgint64(&reply_message);
 	(void) pq_getmsgint64(&reply_message);		/* sendTime; not used ATM */
+	applyTimestamp = IntegerTimestampToTimestampTz(pq_getmsgint64(&reply_message));
 	replyRequested = pq_getmsgbyte(&reply_message);
 
+	/* Compute the apply lag in milliseconds. */
+	if (applyTimestamp == 0)
+		applyLagUs = -1;
+	else
+#ifdef HAVE_INT64_TIMESTAMP
+		applyLagUs = now - applyTimestamp;
+#else
+		applyLagUs = (now - applyTimestamp) * 1000000;
+#endif
+
 	elog(DEBUG2, "write %X/%X flush %X/%X apply %X/%X%s",
 		 (uint32) (writePtr >> 32), (uint32) writePtr,
 		 (uint32) (flushPtr >> 32), (uint32) flushPtr,
@@ -1575,6 +1589,8 @@ ProcessStandbyReplyMessage(void)
 		walsnd->write = writePtr;
 		walsnd->flush = flushPtr;
 		walsnd->apply = applyPtr;
+		if (applyLagUs >= 0)
+			walsnd->applyLagUs = applyLagUs;
 		SpinLockRelease(&walsnd->mutex);
 	}
 
@@ -2745,7 +2761,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	8
+#define PG_STAT_GET_WAL_SENDERS_COLS	9
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2793,6 +2809,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		XLogRecPtr	write;
 		XLogRecPtr	flush;
 		XLogRecPtr	apply;
+		int64		applyLagUs;
 		int			priority;
 		WalSndState state;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
@@ -2807,6 +2824,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
+		applyLagUs = walsnd->applyLagUs;
 		priority = walsnd->sync_standby_priority;
 		SpinLockRelease(&walsnd->mutex);
 
@@ -2841,6 +2859,23 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				nulls[5] = true;
 			values[5] = LSNGetDatum(apply);
 
+			if (applyLagUs < 0)
+				nulls[6] = true;
+			else
+			{
+				Interval *applyLagInterval = palloc(sizeof(Interval));
+
+				applyLagInterval->month = 0;
+				applyLagInterval->day = 0;
+#ifdef HAVE_INT64_TIMESTAMP
+				applyLagInterval->time = applyLagUs;
+#else
+				applyLagInterval->time = applyLagUs / 1000000.0;
+#endif
+				nulls[6] = false;
+				values[6] = IntervalPGetDatum(applyLagInterval);
+			}
+
 			/*
 			 * Treat a standby such as a pg_basebackup background process
 			 * which always returns an invalid flush location, as an
@@ -2848,18 +2883,18 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 			 */
 			priority = XLogRecPtrIsInvalid(walsnd->flush) ? 0 : priority;
 
-			values[6] = Int32GetDatum(priority);
+			values[7] = Int32GetDatum(priority);
 
 			/*
 			 * More easily understood version of standby state. This is purely
 			 * informational, not different from priority.
 			 */
 			if (priority == 0)
-				values[7] = CStringGetTextDatum("async");
+				values[8] = CStringGetTextDatum("async");
 			else if (walsnd == sync_standby)
-				values[7] = CStringGetTextDatum("sync");
+				values[8] = CStringGetTextDatum("sync");
 			else
-				values[7] = CStringGetTextDatum("potential");
+				values[8] = CStringGetTextDatum("potential");
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
diff --git a/src/backend/utils/adt/timestamp.c b/src/backend/utils/adt/timestamp.c
index c9e5270..f382b20 100644
--- a/src/backend/utils/adt/timestamp.c
+++ b/src/backend/utils/adt/timestamp.c
@@ -1629,6 +1629,20 @@ IntegerTimestampToTimestampTz(int64 timestamp)
 #endif
 
 /*
+ * TimestampTzToIntegerTimestamp -- convert a native timestamp to int64 format
+ *
+ * When compiled with --enable-integer-datetimes, this is implemented as a
+ * no-op macro.
+ */
+#ifndef HAVE_INT64_TIMESTAMP
+int64
+TimestampTzToIntegerTimestamp(TimestampTz timestamp)
+{
+	return timestamp * 1000000;
+}
+#endif
+
+/*
  * TimestampDifference -- convert the difference between two timestamps
  *		into integer seconds and microseconds
  *
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index a7dcdae..c8be3ce 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -235,6 +235,9 @@ extern void GetXLogReceiptTime(TimestampTz *rtime, bool *fromStream);
 extern XLogRecPtr GetXLogReplayRecPtr(TimeLineID *replayTLI);
 extern XLogRecPtr GetXLogInsertRecPtr(void);
 extern XLogRecPtr GetXLogWriteRecPtr(void);
+extern void SetXLogReplayTimestamp(TimestampTz timestamp);
+extern void SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn);
+extern TimestampTz GetXLogReplayTimestamp(XLogRecPtr *lsn);
 extern bool RecoveryIsPaused(void);
 extern void SetRecoveryPause(bool recoveryPause);
 extern TimestampTz GetLatestXTime(void);
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index a595327..4054726 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2712,7 +2712,7 @@ DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f
 DESCR("statistics: information about currently active backends");
 DATA(insert OID = 3318 (  pg_stat_get_progress_info           PGNSP PGUID 12 1 100 0 0 f f f f t t s r 1 0 2249 "25" "{25,23,26,26,20,20,20,20,20,20,20,20,20,20}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{cmdtype,pid,datid,relid,param1,param2,param3,param4,param5,param6,param7,param8,param9,param10}" _null_ _null_ pg_stat_get_progress_info _null_ _null_ _null_ ));
 DESCR("statistics: information about progress of backends running maintenance command");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,23,25}" "{o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25}" "{o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 1c24563..e4a1c3a 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -115,7 +115,7 @@ typedef struct
 	/*
 	 * Latch used by startup process to wake up walreceiver after telling it
 	 * where to start streaming (after setting receiveStart and
-	 * receiveStartTLI), and also to tell it to send apply feedback to the
+	 * receiveStartTLI), and then to tell it to send apply feedback to the
 	 * primary whenever specially marked commit records are applied.
 	 */
 	Latch		latch;
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 7794aa5..4de43e8 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -46,6 +46,7 @@ typedef struct WalSnd
 	XLogRecPtr	write;
 	XLogRecPtr	flush;
 	XLogRecPtr	apply;
+	int64		applyLagUs;
 
 	/* Protects shared variables shown above. */
 	slock_t		mutex;
diff --git a/src/include/utils/timestamp.h b/src/include/utils/timestamp.h
index fbead3a..297e151 100644
--- a/src/include/utils/timestamp.h
+++ b/src/include/utils/timestamp.h
@@ -227,9 +227,11 @@ extern bool TimestampDifferenceExceeds(TimestampTz start_time,
 #ifndef HAVE_INT64_TIMESTAMP
 extern int64 GetCurrentIntegerTimestamp(void);
 extern TimestampTz IntegerTimestampToTimestampTz(int64 timestamp);
+extern int64 TimestampTzToIntegerTimestamp(TimestampTz timestamp);
 #else
 #define GetCurrentIntegerTimestamp()	GetCurrentTimestamp()
 #define IntegerTimestampToTimestampTz(timestamp) (timestamp)
+#define TimestampTzToIntegerTimestamp(timestamp) (timestamp)
 #endif
 
 extern TimestampTz time_t_to_timestamptz(pg_time_t tm);
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 79f9b23..fc4b765 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1783,11 +1783,12 @@ pg_stat_replication| SELECT s.pid,
     w.write_location,
     w.flush_location,
     w.replay_location,
+    w.replay_lag,
     w.sync_priority,
     w.sync_state
    FROM pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, wait_event_type, wait_event, xact_start, query_start, backend_start, state_change, client_addr, client_hostname, client_port, backend_xid, backend_xmin, ssl, sslversion, sslcipher, sslbits, sslcompression, sslclientdn),
     pg_authid u,
-    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, sync_priority, sync_state)
+    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state)
   WHERE ((s.usesysid = u.oid) AND (s.pid = w.pid));
 pg_stat_ssl| SELECT s.pid,
     s.ssl,
0003-refactor-syncrep-exit-v9.patchapplication/octet-stream; name=0003-refactor-syncrep-exit-v9.patchDownload
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 1ee1bc5..376ddf4 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -83,6 +83,64 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * ===========================================================
  */
 
+static bool
+SyncRepCheckEarlyExit(void)
+{
+	/*
+	 * If a wait for synchronous replication is pending, we can neither
+	 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
+	 * lead the client to believe that the transaction aborted, which
+	 * is not true: it's already committed locally. The former is no good
+	 * either: the client has requested synchronous replication, and is
+	 * entitled to assume that an acknowledged commit is also replicated,
+	 * which might not be true. So in this case we issue a WARNING (which
+	 * some clients may be able to interpret) and shut off further output.
+	 * We do NOT reset ProcDiePending, so that the process will die after
+	 * the commit is cleaned up.
+	 */
+	if (ProcDiePending)
+	{
+		ereport(WARNING,
+				(errcode(ERRCODE_ADMIN_SHUTDOWN),
+				 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * It's unclear what to do if a query cancel interrupt arrives.  We
+	 * can't actually abort at this point, but ignoring the interrupt
+	 * altogether is not helpful, so we just terminate the wait with a
+	 * suitable warning.
+	 */
+	if (QueryCancelPending)
+	{
+		QueryCancelPending = false;
+		ereport(WARNING,
+				(errmsg("canceling wait for synchronous replication due to user request"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * If the postmaster dies, we'll probably never get an
+	 * acknowledgement, because all the wal sender processes will exit. So
+	 * just bail out.
+	 */
+	if (!PostmasterIsAlive())
+	{
+		ProcDiePending = true;
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	return false;
+}
+
 /*
  * Wait for synchronous replication, if requested by user.
  *
@@ -192,57 +250,9 @@ SyncRepWaitForLSN(XLogRecPtr lsn, bool commit)
 		if (syncRepState == SYNC_REP_WAIT_COMPLETE)
 			break;
 
-		/*
-		 * If a wait for synchronous replication is pending, we can neither
-		 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
-		 * lead the client to believe that the transaction aborted, which
-		 * is not true: it's already committed locally. The former is no good
-		 * either: the client has requested synchronous replication, and is
-		 * entitled to assume that an acknowledged commit is also replicated,
-		 * which might not be true. So in this case we issue a WARNING (which
-		 * some clients may be able to interpret) and shut off further output.
-		 * We do NOT reset ProcDiePending, so that the process will die after
-		 * the commit is cleaned up.
-		 */
-		if (ProcDiePending)
-		{
-			ereport(WARNING,
-					(errcode(ERRCODE_ADMIN_SHUTDOWN),
-					 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit())
 			break;
-		}
-
-		/*
-		 * It's unclear what to do if a query cancel interrupt arrives.  We
-		 * can't actually abort at this point, but ignoring the interrupt
-		 * altogether is not helpful, so we just terminate the wait with a
-		 * suitable warning.
-		 */
-		if (QueryCancelPending)
-		{
-			QueryCancelPending = false;
-			ereport(WARNING,
-					(errmsg("canceling wait for synchronous replication due to user request"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * If the postmaster dies, we'll probably never get an
-		 * acknowledgement, because all the wal sender processes will exit. So
-		 * just bail out.
-		 */
-		if (!PostmasterIsAlive())
-		{
-			ProcDiePending = true;
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
-			break;
-		}
 
 		/*
 		 * Wait on latch.  Any condition that should wake us up will set the
0004-causal-reads-v9.patchapplication/octet-stream; name=0004-causal-reads-v9.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 6ab65a7..ed6f07c 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -2749,6 +2749,35 @@ include_dir 'conf.d'
      across the cluster without problems if that is required.
     </para>
 
+    <sect2 id="runtime-config-replication-all">
+     <title>All Servers</title>
+     <para>
+      These parameters can be set on the primary or any standby.
+     </para>
+     <variablelist>
+      <varlistentry id="guc-causal-reads" xreflabel="causal_reads">
+       <term><varname>causal_reads</varname> (<type>boolean</type>)
+       <indexterm>
+        <primary><varname>causal_reads</> configuration parameter</primary>
+       </indexterm>
+       </term>
+       <listitem>
+        <para>
+         Enables causal consistency between transactions run on different
+         servers.  A transaction that is run on a standby
+         with <varname>causal_reads</> set to <literal>on</> is guaranteed
+         either to see the effects of all completed transactions run on the
+         primary with the setting on, or to receive an error "standby is not
+         available for causal reads".  Note that both transactions involved in
+         a causal dependency (a write on the primary followed by a read on any
+         server which must see the write) must be run with the setting on.
+         See <xref linkend="causal-reads"> for more details.
+        </para>
+       </listitem>
+      </varlistentry>
+     </variablelist>     
+    </sect2>
+
     <sect2 id="runtime-config-replication-sender">
      <title>Sending Server(s)</title>
 
@@ -2980,6 +3009,48 @@ include_dir 'conf.d'
       </listitem>
      </varlistentry>
 
+     <varlistentry>
+      <term><varname>causal_reads_timeout</varname> (<type>integer</type>)
+       <indexterm>
+        <primary><varname>causal_reads_timeout</> configuration parameter</primary>
+       </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies the maximum replay lag the primary will tolerate from a
+        standby before dropping it from the set of standbys available for
+        causal reads.
+       </para>
+       <para>
+        This setting is also used to control the <firstterm>leases</> used to
+        maintain the causal reads guarantee.  It must be set to a value which
+        is at least 4 times the maximum possible difference in system clocks
+        between the primary and standby servers, as described
+        in <xref linkend="causal-reads">.
+       </para>
+      </listitem>
+     </varlistentry>
+
+     <varlistentry id="guc-causal-reads-standby-names" xreflabel="causal-reads-standby-names">
+      <term><varname>causal_reads_standby_names</varname> (<type>string</type>)
+      <indexterm>
+       <primary><varname>causal_reads_standby_names</> configuration parameter</primary>
+      </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies a comma-separated list of standby names that can support
+        <firstterm>causal reads</>, as described in
+        <xref linkend="causal-reads">.  Follows the same convention
+        as <link linkend="guc-synchronous-standby-names"><literal>synchronous_standby_name</></>.
+        The default is <literal>*</>, matching all standbys.
+       </para>
+       <para>
+        This setting has no effect if <varname>causal_reads_timeout</> is not set.
+       </para>
+      </listitem>
+     </varlistentry>
+
      </variablelist>
     </sect2>
 
diff --git a/doc/src/sgml/high-availability.sgml b/doc/src/sgml/high-availability.sgml
index 03c6c30..7a0910d 100644
--- a/doc/src/sgml/high-availability.sgml
+++ b/doc/src/sgml/high-availability.sgml
@@ -1115,6 +1115,9 @@ primary_slot_name = 'node_a_slot'
     that it has replayed the transaction, making it visible to user queries.
     In simple cases, this allows for load balancing with causal consistency
     on a single hot standby.
+    (See also
+    <xref linkend="causal-reads"> which deals with multiple standbys and
+    standby failure.)
    </para>
 
    <para>
@@ -1233,6 +1236,119 @@ primary_slot_name = 'node_a_slot'
    </sect3>
   </sect2>
 
+  <sect2 id="causal-reads">
+   <title>Causal reads</title>
+   <indexterm>
+    <primary>causal reads</primary>
+    <secondary>in standby</secondary>
+   </indexterm>
+
+   <para>
+    The causal reads feature allows read-only queries to run on hot standby
+    servers without exposing stale data to the client, providing a form of
+    causal consistency.  Transactions can run on any standby with the
+    following guarantee about the visibility of preceding transactions: If you
+    set <varname>causal_reads</> to <literal>on</> in any pair of consecutive
+    transactions tx1, tx2 where tx2 begins after tx1 successfully returns,
+    then tx2 will either see tx1 or fail with a new error "standby is not
+    available for causal reads", no matter which server it runs on.  Although
+    the guarantee is expressed in terms of two individual transactions, the
+    GUC can also be set at session, role or system level to make the guarantee
+    generally, allowing for load balancing of applications that were not
+    designed with load balancing in mind.
+   </para>
+
+   <para>
+    In order to enable the feature, <varname>causal_reads_timeout</> must be
+    set to a non-zero value on the primary server.  The
+    GUC <varname>causal_reads_standby_names</> can be used to limit the set of
+    standbys that can join the dynamic set of causal reads standbys by
+    providing a comma-separated list of application names.  By default, all
+    standbys are candidates, if the feature is enabled.
+   </para>
+
+   <para>
+    The current set of servers that the primary considers to be available for
+    causal reads can be seen in
+    the <link linkend="monitoring-stats-views-table"> <literal>pg_stat_replication</></>
+    view.  Administrators, applications and load balancing middleware can use
+    this view to discover standbys that can currently handle causal reads
+    transactions without raising the error.  Since that information is only an
+    instantantaneous snapshot, clients should still be prepared for the error
+    to be raised at any time, and consider redirecting transactions to another
+    standby.
+   </para>
+
+   <para>
+    The advantages of the causal reads feature over simply
+    setting <varname>synchronous_commit</> to <literal>remote_apply</> are:
+    <orderedlist>
+      <listitem>
+       <para>
+        It allows the primary to wait for multiple standbys to replay
+        transactions.
+       </para>
+      </listitem>
+      <listitem>
+       <para>
+        It places a configurable limit on how much replay lag (and therefore
+        delay at commit time) the primary tolerates from standbys before it
+        drops them from the dynamic set of standbys it waits for.
+       </para>   
+      </listitem>
+      <listitem>
+       <para>
+        It upholds the causal reads guarantee during the transitions that
+        occur when new standbys are added or removed from the set of standbys,
+        including scenarios where contact has been lost between the primary
+        and standbys but the standby is still alive and running client
+        queries.
+       </para>
+      </listitem>
+    </orderedlist>
+   </para>
+
+   <para>
+    The protocol used to uphold the guarantee even in the case of network
+    failure depends on the system clocks of the primary and standby servers
+    being synchronized, with an allowance for a difference up to one quarter
+    of <varname>causal_reads_timeout</>.  For example,
+    if <varname>causal_reads_timeout</> is set to <literal>4s</>, then the
+    clocks must not be further than 1 second apart for the guarantee to be
+    upheld reliably during transitions.  The ubiquity of the Network Time
+    Protocol (NTP) on modern operating systems and availability of high
+    quality time servers makes it possible to choose a tolerance significantly
+    higher than the maximum expected clock difference.  An effort is
+    nevertheless made to detect and report misconfigured and faulty systems
+    with clock differences greater than the configured tolerance.
+   </para>
+
+   <note>
+    <para>
+     Current hardware clocks, NTP implementations and public time servers are
+     unlikely to allow the system clocks to differ more than tens or hundreds
+     of milliseconds, and systems synchronized with dedicated local time
+     servers may be considerably more accurate, but you should only consider
+     setting <varname>causal_reads_timeout</> below 4 seconds (allowing up to
+     1 second of clock difference) after researching your time synchronization
+     infrastructure thoroughly.
+    </para>  
+   </note>
+
+   <note>
+    <para>
+      While similar to synchronous replication in the sense that both involve
+      the primary server waiting for responses from standby servers, the
+      causal reads feature is not concerned with avoiding data loss.  A
+      primary configured for causal reads will drop all standbys that stop
+      responding or replay too slowly from the dynamic set that it waits for,
+      so you should consider configuring both synchronous replication and
+      causal reads if you need data loss avoidance guarantees and causal
+      consistency guarantees for load balancing.
+    </para>
+   </note>
+  </sect2>
+
   <sect2 id="continuous-archiving-in-standby">
    <title>Continuous archiving in standby</title>
 
@@ -1581,7 +1697,16 @@ if (!triggered)
     so there will be a measurable delay between primary and standby. Running the
     same query nearly simultaneously on both primary and standby might therefore
     return differing results. We say that data on the standby is
-    <firstterm>eventually consistent</firstterm> with the primary.  Once the
+    <firstterm>eventually consistent</firstterm> with the primary by default.
+    The data visible to a transaction running on a standby can be
+    made <firstterm>causally consistent</> with respect to a transaction that
+    has completed on the primary by setting <varname>causal_reads</>
+    to <literal>on</> in both transactions.  For more details,
+    see <xref linkend="causal-reads">.
+   </para>
+
+   <para>
+    Once the    
     commit record for a transaction is replayed on the standby, the changes
     made by that transaction will be visible to any new snapshots taken on
     the standby.  Snapshots may be taken at the start of each query or at the
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 7d63782..23d68d5 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1224,6 +1224,17 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
      <entry><type>text</></entry>
      <entry>Synchronous state of this standby server</entry>
     </row>
+    <row>
+     <entry><structfield>causal_reads_state</></entry>
+     <entry><type>text</></entry>
+     <entry>Causal reads state of this standby server.  This field will be
+     non-null only if <varname>cause_reads_timeout</> is set.  If a standby is
+     in <literal>available</> state, then it can currently serve causal reads
+     queries.  If it is not replaying fast enough or not responding to
+     keepalive messages, it will be in <literal>unavailable</> state, and if
+     it is currently transitioning to availability it will be
+     in <literal>joining</> state for a short time.</entry>
+    </row>
    </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 893c2fa..111198a 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -2098,11 +2098,12 @@ RecordTransactionCommitPrepared(TransactionId xid,
 	END_CRIT_SECTION();
 
 	/*
-	 * Wait for synchronous replication, if required.
+	 * Wait for causal reads and synchronous replication, if required.
 	 *
 	 * Note that at this stage we have marked clog, but still show as running
 	 * in the procarray and continue to hold locks.
 	 */
+	CausalReadsWaitForLSN(recptr);
 	SyncRepWaitForLSN(recptr, true);
 }
 
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index b0464e7..5ad3646 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,10 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
+	{
+		CausalReadsWaitForLSN(XactLastRecEnd);
 		SyncRepWaitForLSN(XactLastRecEnd, true);
+	}
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5126,7 +5129,7 @@ XactLogCommitRecord(TimestampTz commit_time,
 	 * Check if the caller would like to ask standbys for immediate feedback
 	 * once this commit is applied.
 	 */
-	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY || causal_reads)
 		xl_xinfo.xinfo |= XACT_COMPLETION_SYNC_APPLY_FEEDBACK;
 
 	/*
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index a53f07b..276ac12 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -664,7 +664,8 @@ CREATE VIEW pg_stat_replication AS
             W.replay_location,
             W.replay_lag,
             W.sync_priority,
-            W.sync_state
+            W.sync_state,
+            W.causal_reads_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
             pg_stat_get_wal_senders() AS W
     WHERE S.usesysid = U.oid AND
diff --git a/src/backend/replication/README.causal_reads b/src/backend/replication/README.causal_reads
new file mode 100644
index 0000000..1fddd62
--- /dev/null
+++ b/src/backend/replication/README.causal_reads
@@ -0,0 +1,193 @@
+The causal reads guarantee says: If you run any two consecutive
+transactions tx1, tx2 where tx1 completes before tx2 begins, with
+causal_reads set to "on" in both transactions, tx2 will see tx1 or
+raise an error to complain that it can't guarantee causal consistency,
+no matter which servers (primary or any standby) you run each
+transaction on.
+
+When both transactions run on the primary, the guarantee is trivially
+upheld.
+
+To deal with read-only physical streaming standbys, the primary keeps
+track of a set of standbys that it considers to be currently
+"available" for causal reads, and sends a stream of "leases" to those
+standbys granting them the right to handle causal reads transactions
+for a short time without any further communication with the primary.
+
+In general, the primary provides the guarantee by waiting for all of
+the "available" standbys to report that they have applied a
+transaction.  However, the set of available standbys is dynamic, and
+things get more complicated during state transitions.  There are two
+types of transitions to consider:
+
+1.  unavailable->joining->available
+
+Standbys start out as "unavailable".  If a standby is unavailable and
+is applying fast enough and matches causal_reads_standby_names, the
+primary transitions it to "available", but first it sets it to
+"joining" until it is sure that any transaction committed while it was
+unavailable has definitely been applied on the standby.  This closes a
+race that would otherwise exist if we moved directly to available
+state: tx1 might not wait for a given standby because it's
+unavailable, then a lease might be granted, and then tx2 might run a
+causal reads transaction without error but see stale data.  The
+joining state acts as an airlock: while in joining state, the primary
+waits for that standby to replay causal reads transactions in
+anticipation of the move to available, but it doesn't progress to
+available state and grant a lease to the standby until everything
+preceding joining state has also been applied.
+
+2.  available->unavailable
+
+If a standby is not applying fast enough or not responding to
+keepalive messages, then the primary kicks that standby out of the
+dynamic set of available standbys, that is, marks it as "unavailable".
+In order to make sure that the standby has started rejecting causal
+reads transactions, it needs to revoke the lease it most recently
+granted.  It does that by waiting for the lease to expire before
+allowing any causal reads commits to return.  (In future there could
+be a fast-path revocation message which waits for a serial-numbered
+acknowledgement to reduce waiting in the case where the standby is
+lagging but still reachable and responding).
+
+The rest of this document illustrates how clock skew affects the
+available->unavailable transition.
+
+The following 4 variables are derived from a single GUC, and these
+values will be used in the following illustrations:
+
+causal_reads_timeout = 4s
+lease_time           = 4s (= causal_reads_timeout)
+keepalive_time       = 2s (= lease_time / 2)
+max_clock_skew       = 1s (= lease_time / 4)
+
+Every keepalive_time, the primary transmits a lease that expires at
+local_clock_time + lease_time - max_clock_skew, shown in the following
+diagram as 't' for transmission time and '|' for expiry time.  If
+contact is lost with a standby, the primary will wait until sent_time
++ lease_time for the most recently granted lease to expire, shown on
+the following diagram 'x', to be sure that the standby's clock has
+reached the expiry time even if its clock differs by up to
+max_clock_skew.  In other words, the primary tells the standby that
+the expiry time is at one time, but it trusts that the standby will
+surely agree if it gives it some extra time.  The extra time is
+max_clock_skew.  If the clocks differ by more than max_clock_skew, all
+bets are off (but see below for attempt to detect obvious cases).
+
+0     1     2     3     4     5     6     7     8     9
+t-----------------|-----x
+            t-----------------|-----x
+                        t-----------------|-----x
+                                    t-----------------|...
+                                                t------...
+
+A standby whose clock is 2 seconds ahead of the primary's clock
+perceives gaps in the stream of leases, and will reject causal_reads
+transactions in those intervals.  The causal reads guarantee is
+upheld, but spurious errors are raised between leases, as a
+consequence of the clock skew being greater than max_clock_skew.  In
+the following diagram 'r' shows reception time, and the timeline along
+the top shows the standby's local clock time.
+
+2     3     4     5     6     7     8     9    10    11
+r-----|
+            r-----|
+                        r-----|
+                                    r-----|
+                                                r-----|
+
+If there were no network latency, a standby whose clock is exactly 1
+second ahead of the primary's clock would perceive the stream of
+leases as being replaced just in time, so there is no gap.  Since in
+reality the time of receipt is some time after the time of
+transmission due to network latency, if the standby's clock is exactly
+1 second behind, then there will be small network-latency-sized gaps
+before the next lease arrives, but still no correctness problem with
+respect to the causal reads guarantee.
+
+1     2     3     4     5     6     7     8     9    10
+r-----------|
+            r-----------|
+                        r-----------|
+                                    r-----------|
+                                                r------...
+
+A standby whose clock is perfectly in sync with the primary's
+perceives the stream of leases overlapping (this matches the primary's
+perception of the leases it sent):
+
+0     1     2     3     4     5     6     7     8     9
+r-----------------|
+            r-----------------|
+                        r-----------------|
+                                    r-----------------|
+                                                r------...
+
+A standby whose clock is exactly 1 second behind the primary's
+perceives the stream of leases as overlapping even more, but the time
+of expiry as judged by the standby is no later than the time the
+primary will wait for if required ('x').  That is, if contact is lost
+with the standby, the primary can still reliably hold up causal reads
+commits until the standby has started raising the error in
+causal_reads transactions.
+
+-1    0     1     2     3     4     5     6     7     8
+r-----------------------|
+            r-----------------------|
+                        r-----------------------|
+                                    r------------------...
+                                                r------...
+
+
+A standby whose clock is 2 seconds behind the primary's would perceive
+the stream of leases overlapping even more, and the primary would no
+longer be able to wait for a lease to expire if it wanted to revoke
+it.  But because the expiry time is after local_clock_time +
+lease_time, the standby can immediately see that its own clock must be
+more than 1 second behind the primary's, so it ignores the lease and
+logs a clock skew warning.  In the following diagram a lease expiry
+time that is obviously generated by a primary with a clock set too far
+in the future compared to the local clock is shown with a '!'.
+
+-2    -1    0     1     2     3     4     5     6     7
+r-----------------------------!
+            r-----------------------------!
+                        r-----------------------------!
+                                    r------------------...
+                                                r------...
+
+A danger window exists when the standby's clock is more than
+max_clock_skew behind the primary's clock, but not more than
+max_clock_skew + network latency time behind.  If the clock difference
+is in that range, then the algorithm presented above which is based on
+time of receipt cannot detect that the local clock is too far behind.
+The consequence of this problem could be as follows:
+
+1.  The standby loses contact with the primary due to a network fault.
+
+2.  The primary decides to drop the standby from the set of available
+    causal reads standbys due to lack of keepalive responses or
+    excessive lag, which necessitates holding up commits of causal
+    reads transactions until the most recently sent lease expires, in
+    the belief that the standby will definitely have started raising
+    the 'causal reads unavailable' error in causal reads transactions
+    by that time, if it is still alive and servicing requests.
+
+3.  The standby still has clients connected and running queries.
+
+4.  Due to clock skew in the problematic range, in the standby's
+    opinion the lease lasts slightly longer than the primary waits.
+
+5.  For a short window at most the duration of the network latency
+    time, clients running causal reads transactions are allowed to see
+    potentially stale data.
+
+For this reason we say that the causal reads guarantee only holds as
+long as the absolute difference between the system clocks of the
+machines is no more than max_clock_skew.  The theory is that NTP makes
+it possible to reason about the maximum possible clock difference
+between machines and choose a value that allows for a much larger
+difference.  However, we do make a best effort attempt to detect
+wildly divergent systems as described above, to catch the case of
+servers not running a correctly configured ntp daemon, or with a clock
+so far out of whack that ntp refuses to fix it.
\ No newline at end of file
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 376ddf4..8240d0d 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -57,6 +57,11 @@
 #include "utils/builtins.h"
 #include "utils/ps_status.h"
 
+/* GUC variables */
+int causal_reads_timeout;
+bool causal_reads;
+char *causal_reads_standby_names;
+
 /* User-settable parameters for sync rep */
 char	   *SyncRepStandbyNames;
 
@@ -69,7 +74,7 @@ static int	SyncRepWaitMode = SYNC_REP_NO_WAIT;
 
 static void SyncRepQueueInsert(int mode);
 static void SyncRepCancelWait(void);
-static int	SyncRepWakeQueue(bool all, int mode);
+static int	SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn);
 
 static int	SyncRepGetStandbyPriority(void);
 
@@ -142,6 +147,198 @@ SyncRepCheckEarlyExit(void)
 }
 
 /*
+ * Check if we can stop waiting for causal consistency.  We can stop waiting
+ * when the following conditions are met:
+ *
+ * 1.  All walsenders currently in 'joining' or 'available' state have
+ * applied the target LSN.
+ *
+ * 2.  Any stall periods caused by standbys dropping out of 'available' state
+ * have passed, so that we can be sure that their leases have expired and they
+ * have started rejecting causal reads transactions.
+ *
+ * The output parameter 'waitingFor' is set to the number of nodes we are
+ * currently waiting for.  The output parameters 'stallTimeMillis' is set to
+ * the number of milliseconds we need to wait for to observe any current
+ * commit stall.
+ *
+ * Returns true if commit can return control, because every standby has either
+ * applied the LSN or started rejecting causal_reads transactions.
+ */
+static bool
+CausalReadsCommitCanReturn(XLogRecPtr XactCommitLSN,
+						   int *waitingFor,
+						   long *stallTimeMillis)
+{
+	int i;
+	TimestampTz now;
+
+	/* Count how many joining/available nodes we are waiting for. */
+	*waitingFor = 0;
+	for (i = 0; i < max_wal_senders; ++i)
+	{
+		WalSnd *walsnd = &WalSndCtl->walsnds[i];
+
+		/*
+		 * Assuming atomic read of pid_t, we can check walsnd->pid without
+		 * acquiring the spinlock to avoid memory synchronization costs for
+		 * unused walsender slots.  We see a value that existed sometime at
+		 * least as recently as the last memory barrier.
+		 */
+		if (walsnd->pid != 0)
+		{
+			/*
+			 * We need to hold the spinlock to read LSNs, because we can't be
+			 * sure they can be read atomically.
+			 */
+			SpinLockAcquire(&walsnd->mutex);
+			if (walsnd->pid != 0 && walsnd->causal_reads_state >= WALSNDCRSTATE_JOINING)
+			{
+				if (walsnd->apply < XactCommitLSN)
+					++*waitingFor;
+			}
+			SpinLockRelease(&walsnd->mutex);
+		}
+	}
+
+	/* Check if there is a stall in progress that we need to observe. */
+	now = GetCurrentTimestamp();
+	LWLockAcquire(SyncRepLock, LW_SHARED);
+	if (WalSndCtl->stall_causal_reads_until > now)
+	{
+		long seconds;
+		int usecs;
+
+		/* Compute how long we have to wait, rounded up to nearest ms. */
+		TimestampDifference(now, WalSndCtl->stall_causal_reads_until,
+							&seconds, &usecs);
+		*stallTimeMillis = seconds * 1000 + (usecs + 999) / 1000;
+	}
+	else
+		*stallTimeMillis = 0;
+	LWLockRelease(SyncRepLock);
+
+	/* We are done if we are not waiting for any nodes or stalls. */
+	return *waitingFor == 0 && *stallTimeMillis == 0;
+}
+
+/*
+ * Wait for causal consistency in causal_reads mode, if requested by user.
+ */
+void
+CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN)
+{
+	long stallTimeMillis;
+	int waitingFor;
+	char *ps_display_buffer = NULL;
+
+	/* Leave if we aren't in causal_reads mode. */
+	if (!causal_reads)
+		return;
+
+	for (;;)
+	{
+		/* Reset latch before checking state. */
+		ResetLatch(MyLatch);
+
+		/*
+		 * Join the queue to be woken up if any causal reads joining/available
+		 * standby applies XactCommitLSN or the set of causal reads standbys
+		 * changes (if we aren't already in the queue).  We don't actually know
+		 * if we need to wait for any peers to reach the target LSN yet, but
+		 * we have to register just in case before checking the walsenders'
+		 * state to avoid a race condition that could occur if we did it after
+		 * calling CausalReadsCommitCanReturn.  (SyncRepWaitForLSN doesn't
+		 * have to do this because it can check the highest-seen LSN in
+		 * walsndctl->lsn[mode] which is protected by SyncRepLock, the same
+		 * lock as the queues.  We can't do that here, because there is no
+		 * single highest-seen LSN that is useful.  We must check
+		 * walsnd->apply for all relevant walsenders.  Therefore we must
+		 * register for notifications first, so that we can be notified via
+		 * our latch of any standby applying the LSN we're interested in after
+		 * we check but before we start waiting, or we could wait forever for
+		 * something that has already happened.)
+		 */
+		LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+		if (MyProc->syncRepState != SYNC_REP_WAITING)
+		{
+			MyProc->waitLSN = XactCommitLSN;
+			MyProc->syncRepState = SYNC_REP_WAITING;
+			SyncRepQueueInsert(SYNC_REP_WAIT_CAUSAL_READS);
+			Assert(SyncRepQueueIsOrderedByLSN(SYNC_REP_WAIT_CAUSAL_READS));
+		}
+		LWLockRelease(SyncRepLock);
+
+		/* Check if we're done. */
+		if (CausalReadsCommitCanReturn(XactCommitLSN, &waitingFor, &stallTimeMillis))
+		{
+			SyncRepCancelWait();
+			break;
+		}
+
+		Assert(waitingFor > 0 || stallTimeMillis > 0);
+
+		/* If we aren't actually waiting for any standbys, leave the queue. */
+		if (waitingFor == 0)
+			SyncRepCancelWait();
+
+		/* Update the ps title. */
+		if (update_process_title)
+		{
+			char buffer[80];
+
+			/* Remember the old value if this is our first update. */
+			if (ps_display_buffer == NULL)
+			{
+				int len;
+				const char *ps_display = get_ps_display(&len);
+
+				ps_display_buffer = palloc(len + 1);
+				memcpy(ps_display_buffer, ps_display, len);
+				ps_display_buffer[len] = '\0';
+			}
+
+			snprintf(buffer, sizeof(buffer),
+					 "waiting for %d peer(s) to apply %X/%X%s",
+					 waitingFor,
+					 (uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN,
+					 stallTimeMillis > 0 ? " (stalling)" : "");
+			set_ps_display(buffer, false);
+		}
+
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit()) /* Calls SyncRepCancelWait() if true. */
+			break;
+
+		/*
+		 * If are still waiting for peers, then we wait for any joining or
+		 * available peer to reach the LSN (or possibly stop being in one of
+		 * those states or go away).
+		 *
+		 * If not, there must be a non-zero stall time, so we wait for that to
+		 * elapse.
+		 */
+		if (waitingFor > 0)
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH, -1);
+		else
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH | WL_TIMEOUT,
+					  stallTimeMillis);
+	}
+
+	/* There is no way out of the loop that could leave us in the queue. */
+	Assert(SHMQueueIsDetached(&(MyProc->syncRepLinks)));
+	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
+	MyProc->waitLSN = 0;
+
+	/* Restore the ps display. */
+	if (ps_display_buffer != NULL)
+	{
+		set_ps_display(ps_display_buffer, false);
+		pfree(ps_display_buffer);
+	}
+}
+
+/*
  * Wait for synchronous replication, if requested by user.
  *
  * Initially backends start in state SYNC_REP_NOT_WAITING and then
@@ -425,6 +622,53 @@ SyncRepGetSynchronousStandby(void)
 }
 
 /*
+ * Check if the current WALSender process's application_name matches a name in
+ * causal_reads_standby_names (including '*' for wildcard).
+ */
+bool
+CausalReadsPotentialStandby(void)
+{
+	char *rawstring;
+	List	   *elemlist;
+	ListCell   *l;
+	bool		found = false;
+
+	/* If the feature is disable, then no. */
+	if (causal_reads_timeout == 0)
+		return false;
+
+	/* Need a modifiable copy of string */
+	rawstring = pstrdup(causal_reads_standby_names);
+
+	/* Parse string into list of identifiers */
+	if (!SplitIdentifierString(rawstring, ',', &elemlist))
+	{
+		/* syntax error in list */
+		pfree(rawstring);
+		list_free(elemlist);
+		/* GUC machinery will have already complained - no need to do again */
+		return 0;
+	}
+
+	foreach(l, elemlist)
+	{
+		char	   *standby_name = (char *) lfirst(l);
+
+		if (pg_strcasecmp(standby_name, application_name) == 0 ||
+			pg_strcasecmp(standby_name, "*") == 0)
+		{
+			found = true;
+			break;
+		}
+	}
+
+	pfree(rawstring);
+	list_free(elemlist);
+
+	return found;
+}
+
+/*
  * Update the LSNs on each queue based upon our latest state. This
  * implements a simple policy of first-valid-standby-releases-waiter.
  *
@@ -432,23 +676,27 @@ SyncRepGetSynchronousStandby(void)
  * perhaps also which information we store as well.
  */
 void
-SyncRepReleaseWaiters(void)
+SyncRepReleaseWaiters(bool walsender_cr_available_or_joining)
 {
 	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
 	int			numapply = 0;
+	int			numcausalreadsapply = 0;
+	bool		is_highest_priority_sync_standby;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
-	 * potential sync standbys then we have nothing to do. If we are still
-	 * starting up, still running base backup or the current flush position
-	 * is still invalid, then leave quickly also.
+	 * potential sync standbys and not in a state that causal_reads waits for,
+	 * then we have nothing to do. If we are still starting up, still running
+	 * base backup or the current flush position is still invalid, then leave
+	 * quickly also.
 	 */
-	if (MyWalSnd->sync_standby_priority == 0 ||
-		MyWalSnd->state < WALSNDSTATE_STREAMING ||
-		XLogRecPtrIsInvalid(MyWalSnd->flush))
+	if (!walsender_cr_available_or_joining &&
+		(MyWalSnd->sync_standby_priority == 0 ||
+		 MyWalSnd->state < WALSNDSTATE_STREAMING ||
+		 XLogRecPtrIsInvalid(MyWalSnd->flush)))
 		return;
 
 	/*
@@ -458,13 +706,19 @@ SyncRepReleaseWaiters(void)
 	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
 	syncWalSnd = SyncRepGetSynchronousStandby();
 
-	/* We should have found ourselves at least */
-	Assert(syncWalSnd != NULL);
+	/*
+	 * If we aren't managing the highest priority standby then make a note of
+	 * that so we can announce a takeover in the log if we ever get that job.
+	 */
+	is_highest_priority_sync_standby = syncWalSnd == MyWalSnd;
+	if (!is_highest_priority_sync_standby)
+		announce_next_takeover = true;
 
 	/*
-	 * If we aren't managing the highest priority standby then just leave.
+	 * If we aren't managing the highest priority standby or a standby in
+	 * causal reads 'joining' or 'available' state, then just leave.
 	 */
-	if (syncWalSnd != MyWalSnd)
+	if (!is_highest_priority_sync_standby && !walsender_cr_available_or_joining)
 	{
 		LWLockRelease(SyncRepLock);
 		announce_next_takeover = true;
@@ -473,24 +727,45 @@ SyncRepReleaseWaiters(void)
 
 	/*
 	 * Set the lsn first so that when we wake backends they will release up to
-	 * this location.
+	 * this location.  For the single-standby synchronous commit levels, we
+	 * only do this if we are the current synchronous standby and we are
+	 * advancing the LSN further than it has been advanced before, so that
+	 * SyncRepWaitForLSN can skip waiting in some cases.
 	 */
-	if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
-		numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
-		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+	if (is_highest_priority_sync_standby)
 	{
-		walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
-		numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY);
+		if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
+			numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE,
+										MyWalSnd->write);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
+			numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH,
+										MyWalSnd->flush);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+			numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY,
+										MyWalSnd->apply);
+		}
 	}
 
+	/*
+	 * For causal_reads, all walsenders currently in available or joining
+	 * state must reach the LSN on their own, and standbys will reach LSNs in
+	 * any order.  It doesn't make sense to keep the highest seen LSN in a
+	 * single walsndctl->lsn element.  (CausalReadsWaitForLSN has handling for
+	 * LSNs that have already been reached).
+	 */
+	if (walsender_cr_available_or_joining)
+		numcausalreadsapply =
+			SyncRepWakeQueue(false, SYNC_REP_WAIT_CAUSAL_READS,
+							 MyWalSnd->apply);
+
 	LWLockRelease(SyncRepLock);
 
 	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%x",
@@ -502,7 +777,7 @@ SyncRepReleaseWaiters(void)
 	 * If we are managing the highest priority standby, though we weren't
 	 * prior to this, then announce we are now the sync standby.
 	 */
-	if (announce_next_takeover)
+	if (is_highest_priority_sync_standby && announce_next_takeover)
 	{
 		announce_next_takeover = false;
 		ereport(LOG,
@@ -577,9 +852,8 @@ SyncRepGetStandbyPriority(void)
  * Must hold SyncRepLock.
  */
 static int
-SyncRepWakeQueue(bool all, int mode)
+SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn)
 {
-	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	PGPROC	   *proc = NULL;
 	PGPROC	   *thisproc = NULL;
 	int			numprocs = 0;
@@ -596,7 +870,7 @@ SyncRepWakeQueue(bool all, int mode)
 		/*
 		 * Assume the queue is ordered by LSN
 		 */
-		if (!all && walsndctl->lsn[mode] < proc->waitLSN)
+		if (!all && lsn < proc->waitLSN)
 			return numprocs;
 
 		/*
@@ -656,7 +930,7 @@ SyncRepUpdateSyncStandbysDefined(void)
 			int			i;
 
 			for (i = 0; i < NUM_SYNC_REP_WAIT_MODE; i++)
-				SyncRepWakeQueue(true, i);
+				SyncRepWakeQueue(true, i, InvalidXLogRecPtr);
 		}
 
 		/*
@@ -708,13 +982,31 @@ SyncRepQueueIsOrderedByLSN(int mode)
 #endif
 
 /*
+ * Make sure that CausalReadsWaitForLSN can't return until after the given
+ * lease expiry time has been reached.  In other words, revoke the lease.
+ *
+ * Wake up all backends waiting in CausalReadsWaitForLSN, because the set of
+ * available/joining peers has changed, and there is a new stall time they
+ * need to observe.
+ */
+void
+CausalReadsBeginStall(TimestampTz lease_expiry_time)
+{
+	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+	WalSndCtl->stall_causal_reads_until =
+		Max(WalSndCtl->stall_causal_reads_until, lease_expiry_time);
+	SyncRepWakeQueue(true, SYNC_REP_WAIT_CAUSAL_READS, InvalidXLogRecPtr);
+	LWLockRelease(SyncRepLock);
+}
+
+/*
  * ===========================================================
  * Synchronous Replication functions executed by any process
  * ===========================================================
  */
 
 bool
-check_synchronous_standby_names(char **newval, void **extra, GucSource source)
+check_standby_names(char **newval, void **extra, GucSource source)
 {
 	char	   *rawstring;
 	List	   *elemlist;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index aff3fdb..ba3fd61 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -55,6 +55,7 @@
 #include "libpq/pqformat.h"
 #include "libpq/pqsignal.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "replication/walsender.h"
 #include "storage/ipc.h"
@@ -149,7 +150,8 @@ static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
 static void XLogWalRcvSendReply(bool force, bool requestReply, bool includeApplyTimestamp);
 static void XLogWalRcvSendHSFeedback(bool immed);
-static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
+static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+								  TimestampTz *causalReadsUntil);
 
 /* Signal handlers */
 static void WalRcvSigHupHandler(SIGNAL_ARGS);
@@ -835,6 +837,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 	XLogRecPtr	walEnd;
 	TimestampTz sendTime;
 	bool		replyRequested;
+	TimestampTz causalReadsLease;
 
 	resetStringInfo(&incoming_message);
 
@@ -855,7 +858,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				walEnd = pq_getmsgint64(&incoming_message);
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, NULL);
 
 				buf += hdrlen;
 				len -= hdrlen;
@@ -867,7 +870,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				bool reportApplyTimestamp = false;
 
 				/* copy message to StringInfo */
-				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
+				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char) + sizeof(int64);
 				if (len != hdrlen)
 					ereport(ERROR,
 							(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -879,8 +882,10 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
 				replyRequested = pq_getmsgbyte(&incoming_message);
+				causalReadsLease = IntegerTimestampToTimestampTz(
+					pq_getmsgint64(&incoming_message));
 
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, &causalReadsLease);
 
 				/*
 				 * If no apply timestamps have been sent at the request of the
@@ -1229,15 +1234,52 @@ XLogWalRcvSendHSFeedback(bool immed)
  * Update shared memory status upon receiving a message from primary.
  *
  * 'walEnd' and 'sendTime' are the end-of-WAL and timestamp of the latest
- * message, reported by primary.
+ * message, reported by primary.  'causalReadsLease' is a pointer to
+ * the time the primary promises that this standby can safely claim to be
+ * causally consistent, to 0 if it cannot, or a NULL pointer for no change.
  */
 static void
-ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
+ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+					  TimestampTz *causalReadsLease)
 {
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
 	static TimestampTz lastRecordedTimestamp = 0;
 
+	/* Sanity check for the causalReadsLease time. */
+	if (causalReadsLease != NULL && *causalReadsLease != 0)
+	{
+		/* Deduce max_clock_skew from the causalReadsLease and sendTime. */
+#ifdef HAVE_INT64_TIMESTAMP
+		int64 diffMillis = (*causalReadsLease - sendTime) / 1000;
+#else
+		int64 diffMillis = (*causalReadsLease - sendTime) * 1000;
+#endif
+		int64 max_clock_skew = diffMillis / (CAUSAL_READS_CLOCK_SKEW_RATIO - 1);
+
+		if (sendTime > TimestampTzPlusMilliseconds(lastMsgReceiptTime, max_clock_skew))
+		{
+			/*
+			 * The primary's clock is more than max_clock_skew + network
+			 * latency ahead of the standby's clock.  (If the primary's clock
+			 * is more than max_clock_skew ahead of the standby's clock, but
+			 * by less than the network latency, then there isn't much we can
+			 * do to detect that; but it still seems useful to have this basic
+			 * sanity check for wildly misconfigured servers.)
+			 */
+			elog(LOG, "the primary server's clock time is too far ahead");
+			causalReadsLease = NULL;
+		}
+		/*
+		 * We could also try to detect cases where sendTime is more than
+		 * max_clock_skew in the past according to the standby's clock, but
+		 * that is indistinguishable from network latency/buffering, so we
+		 * could produce misleading error messages; if we do nothing, the
+		 * consequence is 'standby is not available for causal reads' errors
+		 * which should cause the user to investigate.
+		 */
+	}
+
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
 	if (walrcv->latestWalEnd < walEnd)
@@ -1245,6 +1287,8 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->latestWalEnd = walEnd;
 	walrcv->lastMsgSendTime = sendTime;
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
+	if (causalReadsLease != NULL)
+		walrcv->causalReadsLease = *causalReadsLease;
 	SpinLockRelease(&walrcv->mutex);
 
 	/*
diff --git a/src/backend/replication/walreceiverfuncs.c b/src/backend/replication/walreceiverfuncs.c
index 5f6e423..e502f74 100644
--- a/src/backend/replication/walreceiverfuncs.c
+++ b/src/backend/replication/walreceiverfuncs.c
@@ -28,6 +28,7 @@
 #include "replication/walreceiver.h"
 #include "storage/pmsignal.h"
 #include "storage/shmem.h"
+#include "utils/guc.h"
 #include "utils/timestamp.h"
 
 WalRcvData *WalRcv = NULL;
@@ -374,3 +375,21 @@ GetReplicationTransferLatency(void)
 
 	return ms;
 }
+
+/*
+ * Used by snapmgr to check if this standby has a valid lease, granting it the
+ * right to consider itself available for causal reads.
+ */
+bool
+WalRcvCausalReadsAvailable(void)
+{
+	WalRcvData *walrcv = WalRcv;
+	TimestampTz now = GetCurrentTimestamp();
+	bool result;
+
+	SpinLockAcquire(&walrcv->mutex);
+	result = walrcv->causalReadsLease != 0 && now <= walrcv->causalReadsLease;
+	SpinLockRelease(&walrcv->mutex);
+
+	return result;
+}
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 16d7abc..b4dad72 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -154,9 +154,20 @@ static StringInfoData tmpbuf;
  */
 static TimestampTz last_reply_timestamp = 0;
 
+static TimestampTz last_keepalive_timestamp = 0;
+
 /* Have we sent a heartbeat message asking for reply, since last reply? */
 static bool waiting_for_ping_response = false;
 
+/* How long do need to stay in JOINING state? */
+static XLogRecPtr causal_reads_joining_until = 0;
+
+/* The last causal reads lease sent to the standby. */
+static TimestampTz causal_reads_last_lease = 0;
+
+/* Is this WALSender listed in causal_reads_standby_names? */
+static bool am_potential_causal_reads_standby = false;
+
 /*
  * While streaming WAL in Copy mode, streamingDoneSending is set to true
  * after we have sent CopyDone. We should not send any more CopyData messages
@@ -242,6 +253,57 @@ InitWalSender(void)
 	SendPostmasterSignal(PMSIGNAL_ADVANCE_STATE_MACHINE);
 }
 
+ /*
+ * If we are exiting unexpectedly, we may need to communicate with concurrent
+ * causal_reads commits to maintain the causal consistency guarantee.
+ */
+static void
+PrepareUncleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * We've lost contact with the standby, but it may still be alive.  We
+		 * can't let any causal_reads transactions return until we've stalled
+		 * for long enough for a zombie standby to start raising errors
+		 * because its lease has expired.
+		 */
+		elog(LOG, "standby \"%s\" is lost (no longer available for causal reads)", application_name);
+		CausalReadsBeginStall(causal_reads_last_lease);
+
+		/*
+		 * We set the state to a lower level _after_ beginning the stall,
+		 * otherwise there would be a tiny window where commits could return
+		 * without observing the stall.
+		 */
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
+ * We are shutting down because we received a goodbye message from the
+ * walreceiver.
+ */
+static void
+PrepareCleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * The standby is shutting down, so it won't be running any more
+		 * transactions.  It is therefore safe to stop waiting for it, and no
+		 * stall is necessary.
+		 */
+		elog(LOG, "standby \"%s\" is leaving (no longer available for causal reads)", application_name);
+
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
 /*
  * Clean up after an error.
  *
@@ -266,7 +328,10 @@ WalSndErrorCleanup(void)
 
 	replication_active = false;
 	if (walsender_ready_to_stop)
+	{
+		PrepareUncleanExit();
 		proc_exit(0);
+	}
 
 	/* Revert back to startup state */
 	WalSndSetState(WALSNDSTATE_STARTUP);
@@ -278,6 +343,8 @@ WalSndErrorCleanup(void)
 static void
 WalSndShutdown(void)
 {
+	PrepareUncleanExit();
+
 	/*
 	 * Reset whereToSendOutput to prevent ereport from attempting to send any
 	 * more messages to the standby.
@@ -1388,6 +1455,7 @@ ProcessRepliesIfAny(void)
 		if (r < 0)
 		{
 			/* unexpected error or EOF */
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1404,6 +1472,7 @@ ProcessRepliesIfAny(void)
 		resetStringInfo(&reply_message);
 		if (pq_getmessage(&reply_message, 0))
 		{
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1453,6 +1522,7 @@ ProcessRepliesIfAny(void)
 				 * 'X' means that the standby is closing down the socket.
 				 */
 			case 'X':
+				PrepareCleanExit();
 				proc_exit(0);
 
 			default:
@@ -1584,6 +1654,83 @@ ProcessStandbyReplyMessage(void)
 	 */
 	{
 		WalSnd *walsnd = MyWalSnd;
+		WalSndCausalReadsState causal_reads_state = walsnd->causal_reads_state;
+		bool causal_reads_state_changed = false;
+		bool causal_reads_set_joining_until = false;
+
+		/*
+		 * Handle causal reads state transitions, if a causal_reads_timeout is
+		 * configured, this standby is listed in causal_reads_standby_names,
+		 * and we are a primary database (not a cascading standby).
+		 */
+		if (am_potential_causal_reads_standby &&
+			!am_cascading_walsender &&
+			applyLagUs >= 0)
+		{
+			if (applyLagUs / 1000 < causal_reads_timeout)
+			{
+				if (causal_reads_state == WALSNDCRSTATE_UNAVAILABLE)
+				{
+					/*
+					 * The standby is applying fast enough.  We can't grant a
+					 * lease yet though, we need to wait for everything that
+					 * was committed while this standby was unavailable to be
+					 * applied first.  We move to joining state while we wait
+					 * for the standby to catch up.
+					 */
+					causal_reads_state = WALSNDCRSTATE_JOINING;
+					causal_reads_set_joining_until = true;
+					causal_reads_state_changed = true;
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING &&
+						 applyPtr >= causal_reads_joining_until)
+				{
+					/*
+					 * The standby has applied everything committed before we
+					 * reached joining state, and has been waiting for remote
+					 * apply on this standby while it's been in joining state,
+					 * so it is safe to move to available state and send a
+					 * lease.
+					 */
+					causal_reads_state = WALSNDCRSTATE_AVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+			else
+			{
+				if (causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+					/*
+					 * We are dropping a causal reads available standby, so we
+					 * mustn't let any commit command that is waiting in
+					 * CausalReadsWaitForLSN return until we are sure that the
+					 * standby definitely knows that it's not available and
+					 * starts raising errors for causal_reads transactions.
+					 * TODO: We could just wait until the standby acks that
+					 * its lease has been cancelled, and start numbering
+					 * keepalives and sending the number back in replies, so
+					 * we know it's acking the right message; then lagging
+					 * standbys would be less disruptive, but for now we just
+					 * wait for the lease to expire, as we do when we lose
+					 * contact with a standby, for the sake of simplicity.
+					 */
+					CausalReadsBeginStall(causal_reads_last_lease);
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING)
+				{
+					/*
+					 * Dropping a joining standby doesn't require a stall,
+					 * because the standby doesn't think it's available, so
+					 * it's already raising the error for causal_reads
+					 * transactions.
+					 */
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+		}
 
 		SpinLockAcquire(&walsnd->mutex);
 		walsnd->write = writePtr;
@@ -1591,11 +1738,33 @@ ProcessStandbyReplyMessage(void)
 		walsnd->apply = applyPtr;
 		if (applyLagUs >= 0)
 			walsnd->applyLagUs = applyLagUs;
+		walsnd->causal_reads_state = causal_reads_state;
 		SpinLockRelease(&walsnd->mutex);
+
+		if (causal_reads_set_joining_until)
+		{
+			/*
+			 * Record the end of the primary's WAL at some arbitrary point
+			 * observed _after_ we moved to joining state (so that causal
+			 * reads commits start waiting, closing a race).  The standby
+			 * won't become available until it has replayed up to here.
+			 */
+			causal_reads_joining_until = GetFlushRecPtr();
+		}
+
+		if (causal_reads_state_changed)
+		{
+			WalSndKeepalive(true);
+			elog(LOG, "standby \"%s\" is %s", application_name,
+				 causal_reads_state == WALSNDCRSTATE_UNAVAILABLE ? "unavailable for causal reads" :
+				 causal_reads_state == WALSNDCRSTATE_JOINING ? "joining as a causal reads standby..." :
+				 causal_reads_state == WALSNDCRSTATE_AVAILABLE ? "available for causal reads" :
+				 "UNKNOWN");
+		}
 	}
 
 	if (!am_cascading_walsender)
-		SyncRepReleaseWaiters();
+		SyncRepReleaseWaiters(MyWalSnd->causal_reads_state >= WALSNDCRSTATE_JOINING);
 
 	/*
 	 * Advance our local xmin horizon when the client confirmed a flush.
@@ -1736,33 +1905,53 @@ ProcessStandbyHSFeedbackMessage(void)
  * If wal_sender_timeout is enabled we want to wake up in time to send
  * keepalives and to abort the connection if wal_sender_timeout has been
  * reached.
+ *
+ * But if causal_reads_timeout is enabled, we override that and send
+ * keepalives at a constant rate to replace expiring leases.
  */
 static long
 WalSndComputeSleeptime(TimestampTz now)
 {
 	long		sleeptime = 10000;		/* 10 s */
 
-	if (wal_sender_timeout > 0 && last_reply_timestamp > 0)
+	if ((wal_sender_timeout > 0 && last_reply_timestamp > 0) ||
+		am_potential_causal_reads_standby)
 	{
 		TimestampTz wakeup_time;
 		long		sec_to_timeout;
 		int			microsec_to_timeout;
 
-		/*
-		 * At the latest stop sleeping once wal_sender_timeout has been
-		 * reached.
-		 */
-		wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-												  wal_sender_timeout);
-
-		/*
-		 * If no ping has been sent yet, wakeup when it's time to do so.
-		 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
-		 * the timeout passed without a response.
-		 */
-		if (!waiting_for_ping_response)
+		if (am_potential_causal_reads_standby)
+		{
+			/*
+			 * Leases last for a period of between 50% and 100% of
+			 * causal_reads_timeout, depending on clock skew, assuming clock
+			 * skew is under the 25% of causal_reads_timeout.  We send new
+			 * leases every half a lease, so that there are no gaps between
+			 * leases.
+			 */
+			wakeup_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+													  causal_reads_timeout /
+													  CAUSAL_READS_KEEPALIVE_RATIO);
+		}
+		else
+		{
+			/*
+			 * At the latest stop sleeping once wal_sender_timeout has been
+			 * reached.
+			 */
 			wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-													  wal_sender_timeout / 2);
+													  wal_sender_timeout);
+
+			/*
+			 * If no ping has been sent yet, wakeup when it's time to do so.
+			 * WalSndKeepaliveIfNecessary() wants to send a keepalive once
+			 * half of the timeout passed without a response.
+			 */
+			if (!waiting_for_ping_response)
+				wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+														  wal_sender_timeout / 2);
+		}
 
 		/* Compute relative time until wakeup. */
 		TimestampDifference(now, wakeup_time,
@@ -1778,20 +1967,33 @@ WalSndComputeSleeptime(TimestampTz now)
 /*
  * Check whether there have been responses by the client within
  * wal_sender_timeout and shutdown if not.
+ *
+ * If causal_reads_timeout is configured we override that, so that
+ * unresponsive standbys are detected sooner.
  */
 static void
 WalSndCheckTimeOut(TimestampTz now)
 {
 	TimestampTz timeout;
+	int allowed_time;
 
 	/* don't bail out if we're doing something that doesn't require timeouts */
 	if (last_reply_timestamp <= 0)
 		return;
 
-	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
-										  wal_sender_timeout);
+	/*
+	 * If a causal_reads_timeout is configured, it is used instead of
+	 * wal_sender_timeout, to limit the time before an unresponsive causal
+	 * reads standby is dropped.
+	 */
+	if (am_potential_causal_reads_standby)
+		allowed_time = causal_reads_timeout;
+	else
+		allowed_time = wal_sender_timeout;
 
-	if (wal_sender_timeout > 0 && now >= timeout)
+	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
+										  allowed_time);
+	if (allowed_time > 0 && now >= timeout)
 	{
 		/*
 		 * Since typically expiration of replication timeout means
@@ -1824,6 +2026,9 @@ WalSndLoop(WalSndSendDataCallback send_data)
 	last_reply_timestamp = GetCurrentTimestamp();
 	waiting_for_ping_response = false;
 
+	/* Check if we are managing potential causal_reads standby. */
+	am_potential_causal_reads_standby = CausalReadsPotentialStandby();
+
 	/*
 	 * Loop until we reach the end of this timeline or the client requests to
 	 * stop streaming.
@@ -1984,6 +2189,7 @@ InitWalSenderSlot(void)
 			walsnd->flush = InvalidXLogRecPtr;
 			walsnd->apply = InvalidXLogRecPtr;
 			walsnd->state = WALSNDSTATE_STARTUP;
+			walsnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
 			walsnd->latch = &MyProc->procLatch;
 			SpinLockRelease(&walsnd->mutex);
 			/* don't need the lock anymore */
@@ -2753,6 +2959,24 @@ WalSndGetStateString(WalSndState state)
 	return "UNKNOWN";
 }
 
+/*
+ * Return a string constant representing the causal reads state. This is used
+ * in system views, and should *not* be translated.
+ */
+static const char *
+WalSndGetCausalReadsStateString(WalSndCausalReadsState causal_reads_state)
+{
+	switch (causal_reads_state)
+	{
+		case WALSNDCRSTATE_UNAVAILABLE:
+			return "unavailable";
+		case WALSNDCRSTATE_JOINING:
+			return "joining";
+		case WALSNDCRSTATE_AVAILABLE:
+			return "available";
+	}
+	return "UNKNOWN";
+}
 
 /*
  * Returns activity of walsenders, including pids and xlog locations sent to
@@ -2761,7 +2985,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	9
+#define PG_STAT_GET_WAL_SENDERS_COLS	10
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2812,6 +3036,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		int64		applyLagUs;
 		int			priority;
 		WalSndState state;
+		WalSndCausalReadsState causalReadsState;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
 		bool		nulls[PG_STAT_GET_WAL_SENDERS_COLS];
 
@@ -2821,6 +3046,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		SpinLockAcquire(&walsnd->mutex);
 		sentPtr = walsnd->sentPtr;
 		state = walsnd->state;
+		causalReadsState = walsnd->causal_reads_state;
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
@@ -2895,6 +3121,9 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				values[8] = CStringGetTextDatum("sync");
 			else
 				values[8] = CStringGetTextDatum("potential");
+
+			values[9] =
+				CStringGetTextDatum(WalSndGetCausalReadsStateString(causalReadsState));
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
@@ -2914,14 +3143,52 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 static void
 WalSndKeepalive(bool requestReply)
 {
+	TimestampTz now;
+	TimestampTz causal_reads_lease;
+
 	elog(DEBUG2, "sending replication keepalive");
 
+	/*
+	 * If the walsender currently deems the standby to be available for causal
+	 * reads, then it grants a causal reads lease.  The lease authorizes the
+	 * standby to consider itself available for causal reads until a short
+	 * time in the future.  The primary promises to uphold the causal reads
+	 * guarantee until that time, by stalling commits until the the lease has
+	 * expired if necessary.
+	 */
+	now = GetCurrentTimestamp();
+	if (MyWalSnd->causal_reads_state < WALSNDCRSTATE_AVAILABLE)
+		causal_reads_lease = 0; /* Not available, no lease granted. */
+	else
+	{
+		/*
+		 * Since this timestamp is being sent to the standby where it will be
+		 * compared against a time generated by the standby's system clock, we
+		 * must consider clock skew.  First, we decide on a maximum tolerable
+		 * difference between system clocks.  If the primary's clock is ahead
+		 * of the standby's by more than this, then all bets are off (the
+		 * standby could falsely believe it has a valid lease).  If the
+		 * primary's clock is behind the standby's by more than this, then the
+		 * standby will err the other way and generate spurious errors in
+		 * causal_reads mode.  Rather than having a separate GUC for this, we
+		 * derive it from causal_reads_timeout.
+		 */
+		int max_clock_skew = causal_reads_timeout / CAUSAL_READS_CLOCK_SKEW_RATIO;
+
+		/* Compute and remember the expiry time of the lease we're granting. */
+		causal_reads_last_lease = TimestampTzPlusMilliseconds(now, causal_reads_timeout);
+		/* The version we'll send to the standby is adjusted to tolerate clock skew. */
+		causal_reads_lease =
+			TimestampTzPlusMilliseconds(causal_reads_last_lease, -max_clock_skew);
+	}
+
 	/* construct the message... */
 	resetStringInfo(&output_message);
 	pq_sendbyte(&output_message, 'k');
 	pq_sendint64(&output_message, sentPtr);
-	pq_sendint64(&output_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(now));
 	pq_sendbyte(&output_message, requestReply ? 1 : 0);
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(causal_reads_lease));
 
 	/* ... and send it wrapped in CopyData */
 	pq_putmessage_noblock('d', output_message.data, output_message.len);
@@ -2939,23 +3206,35 @@ WalSndKeepaliveIfNecessary(TimestampTz now)
 	 * Don't send keepalive messages if timeouts are globally disabled or
 	 * we're doing something not partaking in timeouts.
 	 */
-	if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
-		return;
-
-	if (waiting_for_ping_response)
-		return;
+	if (!am_potential_causal_reads_standby)
+	{
+		if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
+			return;
+		if (waiting_for_ping_response)
+			return;
+	}
 
 	/*
 	 * If half of wal_sender_timeout has lapsed without receiving any reply
 	 * from the standby, send a keep-alive message to the standby requesting
 	 * an immediate reply.
+	 *
+	 * If causal_reads_timeout has been configured, use it to control
+	 * keepalive intervals rather than wal_sender_timeout, so that we can keep
+	 * replacing leases at the right frequency.
 	 */
-	ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-											wal_sender_timeout / 2);
+	if (am_potential_causal_reads_standby)
+		ping_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+												causal_reads_timeout /
+												CAUSAL_READS_KEEPALIVE_RATIO);
+	else
+		ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+												wal_sender_timeout / 2);
 	if (now >= ping_time)
 	{
 		WalSndKeepalive(true);
 		waiting_for_ping_response = true;
+		last_keepalive_timestamp = now;
 
 		/* Try to flush pending output to the client */
 		if (pq_flush_if_writable() != 0)
diff --git a/src/backend/utils/errcodes.txt b/src/backend/utils/errcodes.txt
index 49494f9..d81c089 100644
--- a/src/backend/utils/errcodes.txt
+++ b/src/backend/utils/errcodes.txt
@@ -306,6 +306,7 @@ Section: Class 40 - Transaction Rollback
 40001    E    ERRCODE_T_R_SERIALIZATION_FAILURE                              serialization_failure
 40003    E    ERRCODE_T_R_STATEMENT_COMPLETION_UNKNOWN                       statement_completion_unknown
 40P01    E    ERRCODE_T_R_DEADLOCK_DETECTED                                  deadlock_detected
+40P02    E    ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE                         causal_reads_not_available
 
 Section: Class 42 - Syntax Error or Access Rule Violation
 
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 06cb166..ac422e7 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -1634,6 +1634,16 @@ static struct config_bool ConfigureNamesBool[] =
 	},
 
 	{
+		{"causal_reads", PGC_USERSET, REPLICATION_STANDBY,
+		 gettext_noop("Enables causal reads."),
+		 NULL
+		},
+		&causal_reads,
+		false,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"syslog_sequence_numbers", PGC_SIGHUP, LOGGING_WHERE,
 			gettext_noop("Add sequence number to syslog messages to avoid duplicate suppression."),
 			NULL
@@ -1811,6 +1821,17 @@ static struct config_int ConfigureNamesInt[] =
 	},
 
 	{
+		{"causal_reads_timeout", PGC_SIGHUP, REPLICATION_STANDBY,
+			gettext_noop("Sets the maximum apply lag before causal reads standbys are no longer available."),
+			NULL,
+			GUC_UNIT_MS
+		},
+		&causal_reads_timeout,
+		0, 0, INT_MAX,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"max_connections", PGC_POSTMASTER, CONN_AUTH_SETTINGS,
 			gettext_noop("Sets the maximum number of concurrent connections."),
 			NULL
@@ -3454,7 +3475,18 @@ static struct config_string ConfigureNamesString[] =
 		},
 		&SyncRepStandbyNames,
 		"",
-		check_synchronous_standby_names, NULL, NULL
+		check_standby_names, NULL, NULL
+	},
+
+	{
+		{"causal_reads_standby_names", PGC_SIGHUP, REPLICATION_MASTER,
+			gettext_noop("List of names of potential causal reads standbys."),
+			NULL,
+			GUC_LIST_INPUT
+		},
+		&causal_reads_standby_names,
+		"*",
+		check_standby_names, NULL, NULL
 	},
 
 	{
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index ec4427f..fcc2c35 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -244,6 +244,15 @@
 				# from standby(s); '*' = all
 #vacuum_defer_cleanup_age = 0	# number of xacts by which cleanup is delayed
 
+#causal_reads_timeout = 0s      # maximum replication delay to tolerate from
+                                # standbys before dropping them from the set of
+				# available causal reads peers; 0 to disable
+				# causal reads
+
+#causal_reads_standy_names = '*'
+                                # standby servers that can potentially become
+				# available for causal reads; '*' = all
+
 # - Standby Servers -
 
 # These settings are ignored on a master server.
@@ -266,6 +275,14 @@
 #wal_retrieve_retry_interval = 5s	# time to wait before retrying to
 					# retrieve WAL after a failed attempt
 
+# - All Servers -
+
+#causal_reads = off                     # "on" in any pair of consecutive
+                                        # transactions guarantees that the second
+					# can see the first (even if the second
+					# is run on a standby), or will raise an
+					# error to report that the standby is
+					# unavailable for causal reads
 
 #------------------------------------------------------------------------------
 # QUERY TUNING
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index b88e012..6336240 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -46,8 +46,11 @@
 
 #include "access/transam.h"
 #include "access/xact.h"
+#include "access/xlog.h"
 #include "lib/pairingheap.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
+#include "replication/walreceiver.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
 #include "storage/procarray.h"
@@ -209,6 +212,16 @@ GetTransactionSnapshot(void)
 				 "cannot take query snapshot during a parallel operation");
 
 		/*
+		 * In causal_reads mode on a standby, check if we have definitely
+		 * applied WAL for any COMMIT that returned successfully on the
+		 * primary.
+		 */
+		if (causal_reads && RecoveryInProgress() && !WalRcvCausalReadsAvailable())
+			ereport(ERROR,
+					(errcode(ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE),
+					 errmsg("standby is not available for causal reads")));
+
+		/*
 		 * In transaction-snapshot mode, the first snapshot must live until
 		 * end of xact regardless of what the caller does with it, so we must
 		 * make a copy of it rather than returning CurrentSnapshotData
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 4054726..c0d7173 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2712,7 +2712,7 @@ DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f
 DESCR("statistics: information about currently active backends");
 DATA(insert OID = 3318 (  pg_stat_get_progress_info           PGNSP PGUID 12 1 100 0 0 f f f f t t s r 1 0 2249 "25" "{25,23,26,26,20,20,20,20,20,20,20,20,20,20}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{cmdtype,pid,datid,relid,param1,param2,param3,param4,param5,param6,param7,param8,param9,param10}" _null_ _null_ pg_stat_get_progress_info _null_ _null_ _null_ ));
 DESCR("statistics: information about progress of backends running maintenance command");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25}" "{o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25,25}" "{o,o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state,causal_reads_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index c005a42..dbfd601 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -24,14 +24,33 @@
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
 #define SYNC_REP_WAIT_APPLY		2
+#define SYNC_REP_WAIT_CAUSAL_READS 3
 
-#define NUM_SYNC_REP_WAIT_MODE	3
+#define NUM_SYNC_REP_WAIT_MODE	4
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
 #define SYNC_REP_WAITING			1
 #define SYNC_REP_WAIT_COMPLETE		2
 
+/*
+ * ratio of causal_read_timeout to max_clock_skew (4 means than the maximum
+ * tolerated clock difference between primary and standbys using causal_reads
+ * is 1/4 of causal_reads_timeout)
+ */
+#define CAUSAL_READS_CLOCK_SKEW_RATIO 4
+
+/*
+ * ratio of causal_reads_timeout to keepalive time (2 means that the effective
+ * keepalive time is 1/2 of the causal_reads_timeout GUC when it is non-zero)
+ */
+#define CAUSAL_READS_KEEPALIVE_RATIO 2
+
+/* GUC variables */
+extern int causal_reads_timeout;
+extern bool causal_reads;
+extern char *causal_reads_standby_names;
+
 /* user-settable parameters for synchronous replication */
 extern char *SyncRepStandbyNames;
 
@@ -43,16 +62,23 @@ extern void SyncRepCleanupAtProcExit(void);
 
 /* called by wal sender */
 extern void SyncRepInitConfig(void);
-extern void SyncRepReleaseWaiters(void);
+extern void SyncRepReleaseWaiters(bool walsender_cr_available_or_joining);
 
 /* called by checkpointer */
 extern void SyncRepUpdateSyncStandbysDefined(void);
 
+/* called by user backend (xact.c) */
+extern void CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN);
+
+/* called by wal sender */
+extern void CausalReadsBeginStall(TimestampTz lease_expiry_time);
+extern bool CausalReadsPotentialStandby(void);
+
 /* forward declaration to avoid pulling in walsender_private.h */
 struct WalSnd;
 extern struct WalSnd *SyncRepGetSynchronousStandby(void);
 
-extern bool check_synchronous_standby_names(char **newval, void **extra, GucSource source);
+extern bool check_standby_names(char **newval, void **extra, GucSource source);
 extern void assign_synchronous_commit(int newval, void *extra);
 
 #endif   /* _SYNCREP_H */
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index e4a1c3a..056c448 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -80,6 +80,13 @@ typedef struct
 	TimeLineID	receivedTLI;
 
 	/*
+	 * causalReadsLease is the time until which the primary has authorized
+	 * this standby to consider itself available for causal_reads mode, or 0
+	 * for not authorized.
+	 */
+	TimestampTz causalReadsLease;
+
+	/*
 	 * latestChunkStart is the starting byte position of the current "batch"
 	 * of received WAL.  It's actually the same as the previous value of
 	 * receivedUpto before the last flush to disk.  Startup process can use
@@ -165,4 +172,6 @@ extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
 extern void WalRcvWakeup(void);
 
+extern bool WalRcvCausalReadsAvailable(void);
+
 #endif   /* _WALRECEIVER_H */
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 4de43e8..f6e0e9e 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -27,6 +27,13 @@ typedef enum WalSndState
 	WALSNDSTATE_STREAMING
 } WalSndState;
 
+typedef enum WalSndCausalReadsState
+{
+	WALSNDCRSTATE_UNAVAILABLE = 0,
+	WALSNDCRSTATE_JOINING,
+	WALSNDCRSTATE_AVAILABLE
+} WalSndCausalReadsState;
+
 /*
  * Each walsender has a WalSnd struct in shared memory.
  */
@@ -34,6 +41,7 @@ typedef struct WalSnd
 {
 	pid_t		pid;			/* this walsender's process id, or 0 */
 	WalSndState state;			/* this walsender's state */
+	WalSndCausalReadsState causal_reads_state; /* the walsender's causal reads state */
 	XLogRecPtr	sentPtr;		/* WAL has been sent up to this point */
 	bool		needreload;		/* does currently-open file need to be
 								 * reloaded? */
@@ -89,6 +97,12 @@ typedef struct
 	 */
 	bool		sync_standbys_defined;
 
+	/*
+	 * Until when must commits in causal_reads stall?  This is used to wait
+	 * for causal reads leases to expire.
+	 */
+	TimestampTz	stall_causal_reads_until;
+
 	WalSnd		walsnds[FLEXIBLE_ARRAY_MEMBER];
 } WalSndCtlData;
 
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index fc4b765..44f826f 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1785,10 +1785,11 @@ pg_stat_replication| SELECT s.pid,
     w.replay_location,
     w.replay_lag,
     w.sync_priority,
-    w.sync_state
+    w.sync_state,
+    w.causal_reads_state
    FROM pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, wait_event_type, wait_event, xact_start, query_start, backend_start, state_change, client_addr, client_hostname, client_port, backend_xid, backend_xmin, ssl, sslversion, sslcipher, sslbits, sslcompression, sslclientdn),
     pg_authid u,
-    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state)
+    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state, causal_reads_state)
   WHERE ((s.usesysid = u.oid) AND (s.pid = w.pid));
 pg_stat_ssl| SELECT s.pid,
     s.ssl,
#67Michael Paquier
michael.paquier@gmail.com
In reply to: Thomas Munro (#66)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Tue, Mar 29, 2016 at 1:11 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Tue, Mar 29, 2016 at 2:28 AM, Michael Paquier
<michael.paquier@gmail.com> wrote:

I definitely prefer that, that's neater! libpq_select could be
simplified because a timeout does not matter much.

Ok, here is a new version that exits the streaming loop correctly when
endofwal becomes true. To hit that codepath you have to set up a
cascading standby with recovery_target_timeline = 'latest', and then
promote the standby it's talking to. I also got rid of the
PostmasterIsAlive() check which became superfluous.

Yes, I can see the difference.

You're right that libpq_select is now only ever called with timeout =
-1 so could theoretically lose the parameter, but I decided against
cluttering this patch up by touching that for now. It seems like the
only reason it's used by libpqrcv_PQexec is something to do with
interrupts on Windows, which I'm not able to test so that was another
reason not to touch it.

OK. I don't mind if the first patch is bare-bone. That's additional cleanup.

(BTW, isn't the select call in libpq_select
lacking an exceptfds set, and can't it therefore block forever when
there is an error condition on the socket and no timeout?)

Hm. I think you're right here when timeout is NULL... It would loop infinitely.
@Andres (in CC): your thoughts on that regarding the new
WaitEventSetWaitBlock()? The same pattern is used there.

-bool walrcv_receive(int timeout, unsigned char *type, char **buffer, int *len)
-
-Retrieve any message available through the connection, blocking for
Oh, the description of walrcv_receive is actually incorrect in
src/backend/replication/README from the beginning... I am sure you
noticed that as well. Perhaps that's worth fixing in the back-branches
(I think it does matter). Thoughts from others?

OK, so I am switching this patch as "Ready for committer", for 0001.
It is in better shape now.
--
Michael

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#68Robert Haas
robertmhaas@gmail.com
In reply to: Michael Paquier (#67)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Tue, Mar 29, 2016 at 3:17 AM, Michael Paquier
<michael.paquier@gmail.com> wrote:

OK, so I am switching this patch as "Ready for committer", for 0001.
It is in better shape now.

Well... I have a few questions yet.

The new argument to SyncRepWaitForLSN is called "bool commit", but
RecordTransactionAbortPrepared passes true. Either it should be
passing false, or the parameter is misnamed or at the least in need of
a better comment.

I don't understand why this patch is touching the abort paths at all.
XactLogAbortRecord sets XACT_COMPLETION_SYNC_APPLY_FEEDBACK, and
xact_redo_abort honors it. But surely it makes no sense to wait for
an abort to become visible.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#69Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Robert Haas (#68)
1 attachment(s)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Wed, Mar 30, 2016 at 6:04 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Tue, Mar 29, 2016 at 3:17 AM, Michael Paquier
<michael.paquier@gmail.com> wrote:

OK, so I am switching this patch as "Ready for committer", for 0001.
It is in better shape now.

Well... I have a few questions yet.

The new argument to SyncRepWaitForLSN is called "bool commit", but
RecordTransactionAbortPrepared passes true. Either it should be
passing false, or the parameter is misnamed or at the least in need of
a better comment.

I don't understand why this patch is touching the abort paths at all.
XactLogAbortRecord sets XACT_COMPLETION_SYNC_APPLY_FEEDBACK, and
xact_redo_abort honors it. But surely it makes no sense to wait for
an abort to become visible.

You're right, that was totally unnecessary. Here is a version that
removes that (ie XactLogAbortRecord doesn't request apply feedback
from the standby, xact_redo_abort doesn't send apply feedback to the
primary and RecordTransactionAbortPrepared now passes false to
SyncRepWaitForLSN so it doesn't wait for apply feedback from the
standby). Also I fixed a silly bug in SyncRepWaitForLSN when capping
the mode. I have also renamed XACT_COMPLETION_SYNC_APPLY_FEEDBACK to
the more general XACT_COMPLETION_APPLY_FEEDBACK, because the later
0004 patch will use it for a more general purpose than
synchronous_commit.

--
Thomas Munro
http://www.enterprisedb.com

Attachments:

0001-remote-apply-v10.patchapplication/octet-stream; name=0001-remote-apply-v10.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index d48a13f..6ab65a7 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -2143,8 +2143,8 @@ include_dir 'conf.d'
         Specifies whether transaction commit will wait for WAL records
         to be written to disk before the command returns a <quote>success</>
         indication to the client.  Valid values are <literal>on</>,
-        <literal>remote_write</>, <literal>local</>, and <literal>off</>.
-        The default, and safe, setting
+        <literal>remote_write</>, <literal>remote_apply</>, <literal>local</>,
+        and <literal>off</>.  The default, and safe, setting
         is <literal>on</>.  When <literal>off</>, there can be a delay between
         when success is reported to the client and when the transaction is
         really guaranteed to be safe against a server crash.  (The maximum
@@ -2177,6 +2177,10 @@ include_dir 'conf.d'
         ensure data preservation even if the standby instance of
         <productname>PostgreSQL</> were to crash, but not if the standby
         suffers an operating-system-level crash.
+        When set to <literal>remote_apply</>, commits will wait until a reply
+        from the current synchronous standby indicates it has received the
+        commit record of the transaction and applied it, so that it has become
+        visible to queries.
        </para>
        <para>
         When synchronous
diff --git a/doc/src/sgml/high-availability.sgml b/doc/src/sgml/high-availability.sgml
index 19d613e..03c6c30 100644
--- a/doc/src/sgml/high-availability.sgml
+++ b/doc/src/sgml/high-availability.sgml
@@ -1081,6 +1081,9 @@ primary_slot_name = 'node_a_slot'
     WAL record is then sent to the standby. The standby sends reply
     messages each time a new batch of WAL data is written to disk, unless
     <varname>wal_receiver_status_interval</> is set to zero on the standby.
+    In the case that <varname>synchronous_commit</> is set to
+    <literal>remote_apply</>, the standby sends reply messages when the commit
+    record is replayed, making the transaction visible.
     If the standby is the first matching standby, as specified in
     <varname>synchronous_standby_names</> on the primary, the reply
     messages from that standby will be used to wake users waiting for
@@ -1107,6 +1110,14 @@ primary_slot_name = 'node_a_slot'
    </para>
 
    <para>
+    Setting <varname>synchronous_commit</> to <literal>remote_apply</> will
+    cause each commit to wait until the current synchronous standby reports
+    that it has replayed the transaction, making it visible to user queries.
+    In simple cases, this allows for load balancing with causal consistency
+    on a single hot standby.
+   </para>
+
+   <para>
     Users will stop waiting if a fast shutdown is requested.  However, as
     when using asynchronous replication, the server will not fully
     shutdown until all outstanding WAL records are transferred to the currently
@@ -1160,9 +1171,10 @@ primary_slot_name = 'node_a_slot'
     <title>Planning for High Availability</title>
 
    <para>
-    Commits made when <varname>synchronous_commit</> is set to <literal>on</>
-    or <literal>remote_write</> will wait until the synchronous standby responds. The response
-    may never occur if the last, or only, standby should crash.
+    Commits made when <varname>synchronous_commit</> is set to <literal>on</>,
+    <literal>remote_write</> or <literal>remote_apply</> will wait until the
+    synchronous standby responds. The response may never occur if the last, or
+    only, standby should crash.
    </para>
 
    <para>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index e7234c8..a65048b 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -1107,7 +1107,7 @@ EndPrepare(GlobalTransaction gxact)
 	 * Note that at this stage we have marked the prepare, but still show as
 	 * running in the procarray (twice!) and continue to hold locks.
 	 */
-	SyncRepWaitForLSN(gxact->prepare_end_lsn);
+	SyncRepWaitForLSN(gxact->prepare_end_lsn, false);
 
 	records.tail = records.head = NULL;
 	records.num_chunks = 0;
@@ -2103,7 +2103,7 @@ RecordTransactionCommitPrepared(TransactionId xid,
 	 * Note that at this stage we have marked clog, but still show as running
 	 * in the procarray and continue to hold locks.
 	 */
-	SyncRepWaitForLSN(recptr);
+	SyncRepWaitForLSN(recptr, true);
 }
 
 /*
@@ -2156,5 +2156,5 @@ RecordTransactionAbortPrepared(TransactionId xid,
 	 * Note that at this stage we have marked clog, but still show as running
 	 * in the procarray and continue to hold locks.
 	 */
-	SyncRepWaitForLSN(recptr);
+	SyncRepWaitForLSN(recptr, false);
 }
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index e315405..7e37331 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,7 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
-		SyncRepWaitForLSN(XactLastRecEnd);
+		SyncRepWaitForLSN(XactLastRecEnd, true);
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5123,6 +5123,13 @@ XactLogCommitRecord(TimestampTz commit_time,
 		xl_xinfo.xinfo |= XACT_COMPLETION_FORCE_SYNC_COMMIT;
 
 	/*
+	 * Check if the caller would like to ask standbys for immediate feedback
+	 * once this commit is applied.
+	 */
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+		xl_xinfo.xinfo |= XACT_COMPLETION_APPLY_FEEDBACK;
+
+	/*
 	 * Relcache invalidations requires information about the current database
 	 * and so does logical decoding.
 	 */
@@ -5459,6 +5466,13 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 	if (XactCompletionForceSyncCommit(parsed->xinfo))
 		XLogFlush(lsn);
 
+	/*
+	 * If asked by the primary (because someone is waiting for a synchronous
+	 * commit = remote_apply), we will need to ask walreceiver to send a
+	 * reply immediately.
+	 */
+	if (XactCompletionApplyFeedback(parsed->xinfo))
+		XLogRequestWalReceiverReply();
 }
 
 /*
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index b119a47..3e454f5 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -345,6 +345,9 @@ static XLogRecPtr RedoRecPtr;
  */
 static bool doPageWrites;
 
+/* Has the recovery code requested a walreceiver wakeup? */
+static bool doRequestWalReceiverReply;
+
 /*
  * RedoStartLSN points to the checkpoint's REDO location which is specified
  * in a backup label file, backup history file or control file. In standby
@@ -6879,6 +6882,19 @@ StartupXLOG(void)
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
 				SpinLockRelease(&XLogCtl->info_lck);
 
+				/*
+				 * If rm_redo reported that it applied a commit record that
+				 * the master is waiting for by calling
+				 * XLogRequestWalReceiverReply, then we wake up the receiver
+				 * so that it notices the updated lastReplayedEndRecPtr and
+				 * sends a reply to the master.
+				 */
+				if (doRequestWalReceiverReply)
+				{
+					doRequestWalReceiverReply = false;
+					WalRcvWakeup();
+				}
+
 				/* Remember this record as the last-applied one */
 				LastRec = ReadRecPtr;
 
@@ -11594,3 +11610,12 @@ SetWalWriterSleeping(bool sleeping)
 	XLogCtl->WalWriterSleeping = sleeping;
 	SpinLockRelease(&XLogCtl->info_lck);
 }
+
+/*
+ * Schedule a walreceiver wakeup in the main recovery loop.
+ */
+void
+XLogRequestWalReceiverReply(void)
+{
+	doRequestWalReceiverReply = true;
+}
diff --git a/src/backend/replication/README b/src/backend/replication/README
index 8e5bf0d..419a2d7 100644
--- a/src/backend/replication/README
+++ b/src/backend/replication/README
@@ -16,14 +16,16 @@ bool walrcv_connect(char *conninfo, XLogRecPtr startpoint)
 Establish connection to the primary, and starts streaming from 'startpoint'.
 Returns true on success.
 
-bool walrcv_receive(int timeout, unsigned char *type, char **buffer, int *len)
-
-Retrieve any message available through the connection, blocking for
-maximum of 'timeout' ms. If a message was successfully read, returns true,
-otherwise false. On success, a pointer to the message payload is stored in
-*buffer, length in *len, and the type of message received in *type. The
-returned buffer is valid until the next call to walrcv_* functions, the
-caller should not attempt freeing it.
+int walrcv_receive(char **buffer, int *wait_fd)
+
+Retrieve any message available without blocking through the
+connection.  If a message was successfully read, returns its
+length. If the connection is closed, returns -1.  Otherwise returns 0
+to indicate that no data is available, and sets *wait_fd to a file
+descriptor which can be waited on before trying again.  On success, a
+pointer to the message payload is stored in *buffer. The returned
+buffer is valid until the next call to walrcv_* functions, and the
+caller should not attempt to free it.
 
 void walrcv_send(const char *buffer, int nbytes)
 
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index 4ee4d71..a3bec49 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -52,7 +52,7 @@ static void libpqrcv_readtimelinehistoryfile(TimeLineID tli, char **filename, ch
 static bool libpqrcv_startstreaming(TimeLineID tli, XLogRecPtr startpoint,
 						char *slotname);
 static void libpqrcv_endstreaming(TimeLineID *next_tli);
-static int	libpqrcv_receive(int timeout, char **buffer);
+static int	libpqrcv_receive(char **buffer, int *wait_fd);
 static void libpqrcv_send(const char *buffer, int nbytes);
 static void libpqrcv_disconnect(void);
 
@@ -463,8 +463,7 @@ libpqrcv_disconnect(void)
 }
 
 /*
- * Receive a message available from XLOG stream, blocking for
- * maximum of 'timeout' ms.
+ * Receive a message available from XLOG stream.
  *
  * Returns:
  *
@@ -472,15 +471,15 @@ libpqrcv_disconnect(void)
  *	 point to a buffer holding the received message. The buffer is only valid
  *	 until the next libpqrcv_* call.
  *
- *	 0 if no data was available within timeout, or wait was interrupted
- *	 by signal.
+ *	 If no data was available immediately, returns 0, and *wait_fd is set to a
+ *	 file descriptor which can be waited on before trying again.
  *
  *	 -1 if the server ended the COPY.
  *
  * ereports on error.
  */
 static int
-libpqrcv_receive(int timeout, char **buffer)
+libpqrcv_receive(char **buffer, int *wait_fd)
 {
 	int			rawlen;
 
@@ -492,16 +491,7 @@ libpqrcv_receive(int timeout, char **buffer)
 	rawlen = PQgetCopyData(streamConn, &recvBuf, 1);
 	if (rawlen == 0)
 	{
-		/*
-		 * No data available yet. If the caller requested to block, wait for
-		 * more data to arrive.
-		 */
-		if (timeout > 0)
-		{
-			if (!libpq_select(timeout))
-				return 0;
-		}
-
+		/* Try consuming some data. */
 		if (PQconsumeInput(streamConn) == 0)
 			ereport(ERROR,
 					(errmsg("could not receive data from WAL stream: %s",
@@ -510,7 +500,11 @@ libpqrcv_receive(int timeout, char **buffer)
 		/* Now that we've consumed some input, try again */
 		rawlen = PQgetCopyData(streamConn, &recvBuf, 1);
 		if (rawlen == 0)
+		{
+			/* Tell caller to try again when our socket is ready. */
+			*wait_fd = PQsocket(streamConn);
 			return 0;
+		}
 	}
 	if (rawlen == -1)			/* end-of-streaming or error */
 	{
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 92faf4e..2da9cba 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -91,13 +91,24 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * to the wait queue. During SyncRepWakeQueue() a WALSender changes
  * the state to SYNC_REP_WAIT_COMPLETE once replication is confirmed.
  * This backend then resets its state to SYNC_REP_NOT_WAITING.
+ *
+ * 'lsn' represents the LSN to wait for.  'commit' indicates whether this LSN
+ * represents a commit record.  If it doesn't, then we wait only for the WAL
+ * to be flushed if synchronous_commit is set to the higher level of
+ * remote_apply, because only commit records provide apply feedback.
  */
 void
-SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
+SyncRepWaitForLSN(XLogRecPtr lsn, bool commit)
 {
 	char	   *new_status = NULL;
 	const char *old_status;
-	int			mode = SyncRepWaitMode;
+	int			mode;
+
+	/* Cap the level for anything other than commit to remote flush only. */
+	if (commit)
+		mode = SyncRepWaitMode;
+	else
+		mode = Min(SyncRepWaitMode, SYNC_REP_WAIT_FLUSH);
 
 	/*
 	 * Fast exit if user has not requested sync replication, or there are no
@@ -122,7 +133,7 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 	 * to be a low cost check.
 	 */
 	if (!WalSndCtl->sync_standbys_defined ||
-		XactCommitLSN <= WalSndCtl->lsn[mode])
+		lsn <= WalSndCtl->lsn[mode])
 	{
 		LWLockRelease(SyncRepLock);
 		return;
@@ -132,7 +143,7 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 	 * Set our waitLSN so WALSender will know when to wake us, and add
 	 * ourselves to the queue.
 	 */
-	MyProc->waitLSN = XactCommitLSN;
+	MyProc->waitLSN = lsn;
 	MyProc->syncRepState = SYNC_REP_WAITING;
 	SyncRepQueueInsert(mode);
 	Assert(SyncRepQueueIsOrderedByLSN(mode));
@@ -147,7 +158,7 @@ SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
 		new_status = (char *) palloc(len + 32 + 1);
 		memcpy(new_status, old_status, len);
 		sprintf(new_status + len, " waiting for %X/%X",
-				(uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN);
+				(uint32) (lsn >> 32), (uint32) lsn);
 		set_ps_display(new_status, false);
 		new_status[len] = '\0'; /* truncate off " waiting ..." */
 	}
@@ -416,6 +427,7 @@ SyncRepReleaseWaiters(void)
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
+	int			numapply = 0;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
@@ -462,12 +474,18 @@ SyncRepReleaseWaiters(void)
 		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
 		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
 	}
+	if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+	{
+		walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+		numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY);
+	}
 
 	LWLockRelease(SyncRepLock);
 
-	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X",
+	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%x",
 		 numwrite, (uint32) (MyWalSnd->write >> 32), (uint32) MyWalSnd->write,
-	   numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush);
+		 numflush, (uint32) (MyWalSnd->flush >> 32), (uint32) MyWalSnd->flush,
+		 numapply, (uint32) (MyWalSnd->apply >> 32), (uint32) MyWalSnd->apply);
 
 	/*
 	 * If we are managing the highest priority standby, though we weren't
@@ -728,6 +746,9 @@ assign_synchronous_commit(int newval, void *extra)
 		case SYNCHRONOUS_COMMIT_REMOTE_FLUSH:
 			SyncRepWaitMode = SYNC_REP_WAIT_FLUSH;
 			break;
+		case SYNCHRONOUS_COMMIT_REMOTE_APPLY:
+			SyncRepWaitMode = SYNC_REP_WAIT_APPLY;
+			break;
 		default:
 			SyncRepWaitMode = SYNC_REP_NO_WAIT;
 			break;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 7b36e02..d1c703c 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -352,8 +352,6 @@ WalReceiverMain(void)
 		if (walrcv_startstreaming(startpointTLI, startpoint,
 								  slotname[0] != '\0' ? slotname : NULL))
 		{
-			bool		endofwal = false;
-
 			if (first_stream)
 				ereport(LOG,
 						(errmsg("started streaming WAL from primary at %X/%X on timeline %u",
@@ -376,18 +374,13 @@ WalReceiverMain(void)
 			ping_sent = false;
 
 			/* Loop until end-of-streaming or error */
-			while (!endofwal)
+			for (;;)
 			{
 				char	   *buf;
 				int			len;
-
-				/*
-				 * Emergency bailout if postmaster has died.  This is to avoid
-				 * the necessity for manual cleanup of all postmaster
-				 * children.
-				 */
-				if (!PostmasterIsAlive())
-					exit(1);
+				bool		endofwal = false;
+				int			wait_fd = PGINVALID_SOCKET;
+				int			rc;
 
 				/*
 				 * Exit walreceiver if we're not in recovery. This should not
@@ -407,8 +400,8 @@ WalReceiverMain(void)
 					XLogWalRcvSendHSFeedback(true);
 				}
 
-				/* Wait a while for data to arrive */
-				len = walrcv_receive(NAPTIME_PER_CYCLE, &buf);
+				/* See if we can read data immediately */
+				len = walrcv_receive(&buf, &wait_fd);
 				if (len != 0)
 				{
 					/*
@@ -439,7 +432,7 @@ WalReceiverMain(void)
 							endofwal = true;
 							break;
 						}
-						len = walrcv_receive(0, &buf);
+						len = walrcv_receive(&buf, &wait_fd);
 					}
 
 					/* Let the master know that we received some data. */
@@ -452,7 +445,44 @@ WalReceiverMain(void)
 					 */
 					XLogWalRcvFlush(false);
 				}
-				else
+
+				/* Check if we need to exit the streaming loop. */
+				if (endofwal)
+					break;
+
+				/*
+				 * Ideally we would reuse a WaitEventSet object repeatedly
+				 * here to avoid the overheads of WaitLatchOrSocket on epoll
+				 * systems, but we can't be sure that libpq (or any other
+				 * walreceiver implementation) has the same socket (even if
+				 * the fd is the same number, it may have been closed and
+				 * reopened since the last time).  In future, if there is a
+				 * function for removing sockets from WaitEventSet, then we
+				 * could add and remove just the socket each time, potentially
+				 * avoiding some system calls.
+				 */
+				Assert(wait_fd != PGINVALID_SOCKET);
+				rc = WaitLatchOrSocket(&walrcv->latch,
+									   WL_POSTMASTER_DEATH | WL_SOCKET_READABLE |
+									   WL_TIMEOUT | WL_LATCH_SET,
+									   wait_fd,
+									   NAPTIME_PER_CYCLE);
+				if (rc & WL_LATCH_SET)
+				{
+					/* The recovery process has asked us to send apply feedback now. */
+					ResetLatch(&walrcv->latch);
+					XLogWalRcvSendReply(true, false);
+				}
+				if (rc & WL_POSTMASTER_DEATH)
+				{
+					/*
+					 * Emergency bailout if postmaster has died.  This is to
+					 * avoid the necessity for manual cleanup of all
+					 * postmaster children.
+					 */
+					exit(1);
+				}
+				if (rc & WL_TIMEOUT)
 				{
 					/*
 					 * We didn't receive anything new. If we haven't heard
@@ -1222,6 +1252,20 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 }
 
 /*
+ * Wake up the walreceiver main loop.
+ *
+ * This is called by the startup process whenever interesting xlog records
+ * are applied, so that walreceiver can check if it needs to send an apply
+ * notification back to the master which may be waiting in a COMMIT with
+ * synchronous_commit = remote_apply.
+ */
+void
+WalRcvWakeup(void)
+{
+	SetLatch(&WalRcv->latch);
+}
+
+/*
  * Return a string constant representing the state. This is used
  * in system functions and views, and should *not* be translated.
  */
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 65a6cd4..06cb166 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -345,12 +345,13 @@ static const struct config_enum_entry constraint_exclusion_options[] = {
 };
 
 /*
- * Although only "on", "off", "remote_write", and "local" are documented, we
- * accept all the likely variants of "on" and "off".
+ * Although only "on", "off", "remote_apply", "remote_write", and "local" are
+ * documented, we accept all the likely variants of "on" and "off".
  */
 static const struct config_enum_entry synchronous_commit_options[] = {
 	{"local", SYNCHRONOUS_COMMIT_LOCAL_FLUSH, false},
 	{"remote_write", SYNCHRONOUS_COMMIT_REMOTE_WRITE, false},
+	{"remote_apply", SYNCHRONOUS_COMMIT_REMOTE_APPLY, false},
 	{"on", SYNCHRONOUS_COMMIT_ON, false},
 	{"off", SYNCHRONOUS_COMMIT_OFF, false},
 	{"true", SYNCHRONOUS_COMMIT_ON, true},
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index 5536012..ec4427f 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -177,7 +177,7 @@
 					# (change requires restart)
 #fsync = on				# turns forced synchronization on or off
 #synchronous_commit = on		# synchronization level;
-					# off, local, remote_write, or on
+					# off, local, remote_write, remote_apply, or on
 #wal_sync_method = fsync		# the default is the first option
 					# supported by the operating system:
 					#   open_datasync
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 3ba23f5..503ae1b 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -60,7 +60,9 @@ typedef enum
 	SYNCHRONOUS_COMMIT_LOCAL_FLUSH,		/* wait for local flush only */
 	SYNCHRONOUS_COMMIT_REMOTE_WRITE,	/* wait for local flush and remote
 										 * write */
-	SYNCHRONOUS_COMMIT_REMOTE_FLUSH		/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_FLUSH,	/* wait for local and remote flush */
+	SYNCHRONOUS_COMMIT_REMOTE_APPLY		/* wait for local flush and remote
+										 * apply */
 }	SyncCommitLevel;
 
 /* Define the default setting for synchonous_commit */
@@ -144,10 +146,13 @@ typedef void (*SubXactCallback) (SubXactEvent event, SubTransactionId mySubid,
  * EOXact... routines which run at the end of the original transaction
  * completion.
  */
+#define XACT_COMPLETION_APPLY_FEEDBACK			(1U << 29)
 #define XACT_COMPLETION_UPDATE_RELCACHE_FILE	(1U << 30)
 #define XACT_COMPLETION_FORCE_SYNC_COMMIT		(1U << 31)
 
 /* Access macros for above flags */
+#define XactCompletionApplyFeedback(xinfo) \
+	((xinfo & XACT_COMPLETION_APPLY_FEEDBACK) != 0)
 #define XactCompletionRelcacheInitFileInval(xinfo) \
 	((xinfo & XACT_COMPLETION_UPDATE_RELCACHE_FILE) != 0)
 #define XactCompletionForceSyncCommit(xinfo) \
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 74a1394..a7dcdae 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -267,6 +267,8 @@ extern bool CheckPromoteSignal(void);
 extern void WakeupRecovery(void);
 extern void SetWalWriterSleeping(bool sleeping);
 
+extern void XLogRequestWalReceiverReply(void);
+
 extern void assign_max_wal_size(int newval, void *extra);
 extern void assign_checkpoint_completion_target(double newval, void *extra);
 
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index 96e059b..c005a42 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -23,8 +23,9 @@
 #define SYNC_REP_NO_WAIT		-1
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
+#define SYNC_REP_WAIT_APPLY		2
 
-#define NUM_SYNC_REP_WAIT_MODE	2
+#define NUM_SYNC_REP_WAIT_MODE	3
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
@@ -35,7 +36,7 @@
 extern char *SyncRepStandbyNames;
 
 /* called by user backend */
-extern void SyncRepWaitForLSN(XLogRecPtr XactCommitLSN);
+extern void SyncRepWaitForLSN(XLogRecPtr lsn, bool commit);
 
 /* called at backend exit */
 extern void SyncRepCleanupAtProcExit(void);
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 6eacb09..1c24563 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -115,7 +115,8 @@ typedef struct
 	/*
 	 * Latch used by startup process to wake up walreceiver after telling it
 	 * where to start streaming (after setting receiveStart and
-	 * receiveStartTLI).
+	 * receiveStartTLI), and also to tell it to send apply feedback to the
+	 * primary whenever specially marked commit records are applied.
 	 */
 	Latch		latch;
 } WalRcvData;
@@ -138,7 +139,7 @@ extern PGDLLIMPORT walrcv_startstreaming_type walrcv_startstreaming;
 typedef void (*walrcv_endstreaming_type) (TimeLineID *next_tli);
 extern PGDLLIMPORT walrcv_endstreaming_type walrcv_endstreaming;
 
-typedef int (*walrcv_receive_type) (int timeout, char **buffer);
+typedef int (*walrcv_receive_type) (char **buffer, int *wait_fd);
 extern PGDLLIMPORT walrcv_receive_type walrcv_receive;
 
 typedef void (*walrcv_send_type) (const char *buffer, int nbytes);
@@ -162,5 +163,6 @@ extern void RequestXLogStreaming(TimeLineID tli, XLogRecPtr recptr,
 extern XLogRecPtr GetWalRcvWriteRecPtr(XLogRecPtr *latestChunkStart, TimeLineID *receiveTLI);
 extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
+extern void WalRcvWakeup(void);
 
 #endif   /* _WALRECEIVER_H */
#70Robert Haas
robertmhaas@gmail.com
In reply to: Thomas Munro (#69)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Tue, Mar 29, 2016 at 5:47 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Wed, Mar 30, 2016 at 6:04 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Tue, Mar 29, 2016 at 3:17 AM, Michael Paquier
<michael.paquier@gmail.com> wrote:

OK, so I am switching this patch as "Ready for committer", for 0001.
It is in better shape now.

Well... I have a few questions yet.

The new argument to SyncRepWaitForLSN is called "bool commit", but
RecordTransactionAbortPrepared passes true. Either it should be
passing false, or the parameter is misnamed or at the least in need of
a better comment.

I don't understand why this patch is touching the abort paths at all.
XactLogAbortRecord sets XACT_COMPLETION_SYNC_APPLY_FEEDBACK, and
xact_redo_abort honors it. But surely it makes no sense to wait for
an abort to become visible.

You're right, that was totally unnecessary. Here is a version that
removes that (ie XactLogAbortRecord doesn't request apply feedback
from the standby, xact_redo_abort doesn't send apply feedback to the
primary and RecordTransactionAbortPrepared now passes false to
SyncRepWaitForLSN so it doesn't wait for apply feedback from the
standby). Also I fixed a silly bug in SyncRepWaitForLSN when capping
the mode. I have also renamed XACT_COMPLETION_SYNC_APPLY_FEEDBACK to
the more general XACT_COMPLETION_APPLY_FEEDBACK, because the later
0004 patch will use it for a more general purpose than
synchronous_commit.

OK, I committed this, with a few tweaks. In particular, I added a
flag variable instead of relying on "latch set" == "need to send
reply"; the other changes were cosmetic.

I'm not sure how much more of this we can realistically get into 9.6;
the latter patches haven't had much review yet. But I'll set this
back to Needs Review in the CommitFest and we'll see where we end up.
But even if we don't get anything more than this, it's still rather
nice: remote_apply turns out to be only slightly slower than remote
flush, and it's a guarantee that a lot of people are looking for.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#71Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Robert Haas (#70)
3 attachment(s)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Wed, Mar 30, 2016 at 2:36 PM, Robert Haas <robertmhaas@gmail.com> wrote:

OK, I committed this, with a few tweaks. In particular, I added a
flag variable instead of relying on "latch set" == "need to send
reply"; the other changes were cosmetic.

I'm not sure how much more of this we can realistically get into 9.6;
the latter patches haven't had much review yet. But I'll set this
back to Needs Review in the CommitFest and we'll see where we end up.
But even if we don't get anything more than this, it's still rather
nice: remote_apply turns out to be only slightly slower than remote
flush, and it's a guarantee that a lot of people are looking for.

Thank you Michael and Robert!

Please find attached the rest of the patch series, rebased against
master. The goal of the 0002 patch is to provide an accurate
indication of the current replay lag on each standby, visible to users
like this:

postgres=# select application_name, replay_lag from pg_stat_replication;
application_name │ replay_lag
──────────────────┼─────────────────
replica1 │ 00:00:00.000299
replica2 │ 00:00:00.000323
replica3 │ 00:00:00.000319
replica4 │ 00:00:00.000303
(4 rows)

It works by maintaining a buffer of (end of WAL, time now) samples
received from the primary, and then eventually feeding those times
back to the primary when the recovery process replays the
corresponding locations.

Compared to approaches based on commit timestamps, this approach has
the advantage of providing non-misleading information between commits.
For example, if you run a batch load job that takes 1 minute to insert
the whole phonebook and no other transactions run, you will see
replay_lag updating regularly throughout that minute, whereas typical
commit timestamp-only approaches will show an increasing lag time
until a commit record is eventually applied. Compared to simple LSN
location comparisons, it reports in time rather than bytes of WAL,
which can be more meaningful for DBAs.

When the standby is entirely caught up and there is no write activity,
the reported time effectively represents the ping time between the
servers, and is updated every wal_sender_timeout / 2, when keepalive
messages are sent. While new WAL traffic is arriving, the walreceiver
records timestamps at most once per second in a circular buffer, and
then sends back replies containing the recorded timestamps as fast as
the recovery process can apply the corresponding xlog. The lag number
you see is computed by the primary server comparing two timestamps
generated by its own system clock, one of which has been on a journey
to the standby and back.

Accurate lag estimates are a prerequisite for the 0004 patch (about
which more later), but I believe users would find this valuable as a
feature on its own.

--
Thomas Munro
http://www.enterprisedb.com

Attachments:

0002-replay-lag-v11.patchapplication/octet-stream; name=0002-replay-lag-v11.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 105d541..7d63782 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1208,6 +1208,12 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
       standby server</entry>
     </row>
     <row>
+     <entry><structfield>replay_lag</></entry>
+     <entry><type>interval</></entry>
+     <entry>Estimated time taken for recent WAL records to be replayed on this
+      standby server</entry>
+    </row>
+    <row>
      <entry><structfield>sync_priority</></entry>
      <entry><type>integer</></entry>
      <entry>Priority of this standby server for being chosen as the
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 7e37331..65a0688 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -5467,6 +5467,12 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 		XLogFlush(lsn);
 
 	/*
+	 * Record the primary's timestamp for the commit record, so it can be used
+	 * for tracking replay lag.
+	 */
+	SetXLogReplayTimestamp(parsed->xact_time);
+
+	/*
 	 * If asked by the primary (because someone is waiting for a synchronous
 	 * commit = remote_apply), we will need to ask walreceiver to send a
 	 * reply immediately.
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 06cefe2..06fc23a 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -81,6 +81,8 @@ extern uint32 bootstrap_data_checksum_version;
 #define PROMOTE_SIGNAL_FILE		"promote"
 #define FALLBACK_PROMOTE_SIGNAL_FILE "fallback_promote"
 
+/* Size of the circular buffer of timestamped LSNs. */
+#define TIMESTAMP_BUFFER_SIZE 8192
 
 /* User-settable parameters */
 int			max_wal_size = 64;	/* 1 GB */
@@ -360,6 +362,13 @@ static bool doRequestWalReceiverReply;
  */
 static XLogRecPtr RedoStartLSN = InvalidXLogRecPtr;
 
+/*
+ * LastReplayedTimestamp can be set by redo handlers when they apply a record
+ * that carries a timestamp, by calling SetXLogReplayedTimestamp.  The xlog
+ * apply loop can then update the value in shared memory.
+ */
+static TimestampTz LastReplayedTimestamp = 0;
+
 /*----------
  * Shared-memory data structures for XLOG control
  *
@@ -518,6 +527,15 @@ typedef struct XLogCtlInsert
 } XLogCtlInsert;
 
 /*
+ * A sample associating a timestamp with a given xlog position.
+ */
+typedef struct XLogTimestamp
+{
+	Timestamp	timestamp;
+	XLogRecPtr	lsn;
+} XLogTimestamp;
+
+/*
  * Total shared-memory state for XLOG.
  */
 typedef struct XLogCtlData
@@ -634,6 +652,20 @@ typedef struct XLogCtlData
 	/* current effective recovery target timeline */
 	TimeLineID	RecoveryTargetTLI;
 
+	/* timestamp from the most recently applied record associated with a timestamp. */
+	TimestampTz lastReplayedTimestamp;
+
+	/*
+	 * We maintain a circular buffer of LSNs and associated timestamps.
+	 * Walreceiver writes into it using information received from an upstream
+	 * server, and the recovery process reads from it and notifies walreceiver
+	 * when LSNs are eventually replayed so that the timestamps can be fed
+	 * back to the upstream server.
+	 */
+	Index			timestampBufferRead;
+	Index			timestampBufferWrite;
+	XLogTimestamp	timestampBuffer[TIMESTAMP_BUFFER_SIZE];
+
 	/*
 	 * timestamp of when we started replaying the current chunk of WAL data,
 	 * only relevant for replication or archive recovery
@@ -6874,12 +6906,42 @@ StartupXLOG(void)
 				error_context_stack = errcallback.previous;
 
 				/*
-				 * Update lastReplayedEndRecPtr after this record has been
-				 * successfully replayed.
+				 * Update lastReplayedEndRecPtr and lastReplayedTimestamp
+				 * after this record has been successfully replayed.
 				 */
 				SpinLockAcquire(&XLogCtl->info_lck);
 				XLogCtl->lastReplayedEndRecPtr = EndRecPtr;
 				XLogCtl->lastReplayedTLI = ThisTimeLineID;
+				if (LastReplayedTimestamp != 0)
+				{
+					/* If replaying a record produced a timestamp, use that. */
+					XLogCtl->lastReplayedTimestamp = LastReplayedTimestamp;
+					LastReplayedTimestamp = 0;
+				}
+				else
+				{
+					/*
+					 * If we have applied LSNs associated with timestamps
+					 * received by walreceiver, then use the recorded
+					 * timestamp.  We consume from the read end of the
+					 * circular buffer.
+					 */
+					while (XLogCtl->timestampBufferRead !=
+						   XLogCtl->timestampBufferWrite &&
+						   XLogCtl->timestampBuffer[XLogCtl->timestampBufferRead].lsn
+						   <= EndRecPtr)
+					{
+						if (XLogCtl->timestampBuffer[XLogCtl->timestampBufferRead].timestamp >
+							XLogCtl->lastReplayedTimestamp)
+						{
+							XLogCtl->lastReplayedTimestamp =
+								XLogCtl->timestampBuffer[XLogCtl->timestampBufferRead].timestamp;
+							doRequestWalReceiverReply = true;
+						}
+						XLogCtl->timestampBufferRead =
+							(XLogCtl->timestampBufferRead + 1) % TIMESTAMP_BUFFER_SIZE;
+					}
+				}
 				SpinLockRelease(&XLogCtl->info_lck);
 
 				/*
@@ -11617,3 +11679,91 @@ XLogRequestWalReceiverReply(void)
 {
 	doRequestWalReceiverReply = true;
 }
+
+/*
+ * Record the timestamp that is associated with a WAL position.
+ *
+ * This is called by walreceiver on standby servers when keepalive messages
+ * arrive, using timestamps generated on the primary server.  The timestamp
+ * will be sent back to the primary server when the standby had applied this
+ * WAL position.  The primary can use the elapsed time to estimate the replay
+ * lag.
+ */
+void
+SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn)
+{
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn == XLogCtl->lastReplayedEndRecPtr)
+	{
+		/*
+		 * That is the last replayed LSN: we are fully replayed, so we can
+		 * update the replay timestamp immediately.
+		 */
+		XLogCtl->lastReplayedTimestamp = timestamp;
+	}
+	else
+	{
+		/*
+		 * There is WAL still to be applied.  We will associate the timestamp
+		 * with this WAL position and wait for it to be replayed.  We add it
+		 * at the 'write' end of the circular buffer of LSN/timestamp
+		 * mappings, which the replay loop will eventually read.
+		 */
+		Index w = XLogCtl->timestampBufferWrite;
+		Index r = XLogCtl->timestampBufferRead;
+
+		XLogCtl->timestampBuffer[w].lsn = lsn;
+		XLogCtl->timestampBuffer[w].timestamp = timestamp;
+
+		/* Advance the write point. */
+		w = (w + 1) % TIMESTAMP_BUFFER_SIZE;
+		XLogCtl->timestampBufferWrite = w;
+		if (w == r)
+		{
+			/*
+			 * The buffer is full.  Advance the read point (throwing away
+			 * oldest values; we will begin to overestimate replay lag, until
+			 * lag decreases to a size our buffer can manage, or the next
+			 * commit record is replayed).
+			 */
+			r = (r + 1) % TIMESTAMP_BUFFER_SIZE;
+			XLogCtl->timestampBufferRead = r;
+		}
+	}
+	SpinLockRelease(&XLogCtl->info_lck);
+}
+
+/*
+ * Set the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary.  This can be called by redo handlers that have
+ * an appropriate timestamp (currently only commit records).  Updating the
+ * shared memory value is deferred until after the redo handler returns.
+ */
+void
+SetXLogReplayTimestamp(TimestampTz timestamp)
+{
+	LastReplayedTimestamp = timestamp;
+}
+
+/*
+ * Get the timestamp for the most recently applied WAL record that carried a
+ * timestamp from the primary, and also the most recently applied LSN.  (Note
+ * that the timestamp and the LSN don't necessarily relate to the same
+ * record.)
+ *
+ * This is similar to GetLatestXTime, except that it is not only advanced by
+ * commit records (see SetXLogReplayTimestampAtLsn).
+ */
+TimestampTz
+GetXLogReplayTimestamp(XLogRecPtr *lsn)
+{
+	TimestampTz result;
+
+	SpinLockAcquire(&XLogCtl->info_lck);
+	if (lsn)
+		*lsn = XLogCtl->lastReplayedEndRecPtr;
+	result = XLogCtl->lastReplayedTimestamp;
+	SpinLockRelease(&XLogCtl->info_lck);
+
+	return result;
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 9ae1ef4..a53f07b 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -662,6 +662,7 @@ CREATE VIEW pg_stat_replication AS
             W.write_location,
             W.flush_location,
             W.replay_location,
+            W.replay_lag,
             W.sync_priority,
             W.sync_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 057c250..7eab357 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -85,6 +85,8 @@ walrcv_disconnect_type walrcv_disconnect = NULL;
 
 #define NAPTIME_PER_CYCLE 100	/* max sleep time between cycles (100ms) */
 
+#define MIN_TIME_BETWEEN_TIMESTAMPED_LSNS 1000 /* 1s */
+
 /*
  * These variables are used similarly to openLogFile/SegNo/Off,
  * but for walreceiver to write the XLOG. recvFileTLI is the TimeLineID
@@ -102,6 +104,8 @@ static uint32 recvOff = 0;
 static volatile sig_atomic_t got_SIGHUP = false;
 static volatile sig_atomic_t got_SIGTERM = false;
 
+static bool recovery_active = false;
+
 /*
  * LogstreamResult indicates the byte positions that we have already
  * written/fsynced.
@@ -143,7 +147,7 @@ static void WalRcvDie(int code, Datum arg);
 static void XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len);
 static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
-static void XLogWalRcvSendReply(bool force, bool requestReply);
+static void XLogWalRcvSendReply(bool force, bool requestReply, bool includeApplyTimestamp);
 static void XLogWalRcvSendHSFeedback(bool immed);
 static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
 
@@ -436,7 +440,7 @@ WalReceiverMain(void)
 					}
 
 					/* Let the master know that we received some data. */
-					XLogWalRcvSendReply(false, false);
+					XLogWalRcvSendReply(false, false, false);
 
 					/*
 					 * If we've written some records, flush them to disk and
@@ -480,7 +484,8 @@ WalReceiverMain(void)
 						 */
 						walrcv->force_reply = false;
 						pg_memory_barrier();
-						XLogWalRcvSendReply(true, false);
+						XLogWalRcvSendReply(true, false, true);
+						recovery_active = true;
 					}
 				}
 				if (rc & WL_POSTMASTER_DEATH)
@@ -538,7 +543,7 @@ WalReceiverMain(void)
 						}
 					}
 
-					XLogWalRcvSendReply(requestReply, requestReply);
+					XLogWalRcvSendReply(requestReply, requestReply, false);
 					XLogWalRcvSendHSFeedback(false);
 				}
 			}
@@ -869,6 +874,8 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 			}
 		case 'k':				/* Keepalive */
 			{
+				bool reportApplyTimestamp = false;
+
 				/* copy message to StringInfo */
 				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
 				if (len != hdrlen)
@@ -885,9 +892,22 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 
 				ProcessWalSndrMessage(walEnd, sendTime);
 
+				/*
+				 * If no apply timestamps have been sent at the request of the
+				 * recovery process since we last received a keepalive, then
+				 * we will send one now.  This allows us to feed back
+				 * timestamps in response to pings if we are idle or if the
+				 * recovery process is somehow blocked, but we don't want to
+				 * do that if it's actively applying and periodically waking
+				 * us up with accurate apply timestamps.
+				 */
+				if (!recovery_active)
+					reportApplyTimestamp = true;
+				recovery_active = false;
+
 				/* If the primary requested a reply, send one immediately */
-				if (replyRequested)
-					XLogWalRcvSendReply(true, false);
+				if (replyRequested || reportApplyTimestamp)
+					XLogWalRcvSendReply(true, false, reportApplyTimestamp);
 				break;
 			}
 		default:
@@ -1050,7 +1070,7 @@ XLogWalRcvFlush(bool dying)
 		/* Also let the master know that we made some progress */
 		if (!dying)
 		{
-			XLogWalRcvSendReply(false, false);
+			XLogWalRcvSendReply(false, false, false);
 			XLogWalRcvSendHSFeedback(false);
 		}
 	}
@@ -1068,15 +1088,18 @@ XLogWalRcvFlush(bool dying)
  * If 'requestReply' is true, requests the server to reply immediately upon
  * receiving this message. This is used for heartbearts, when approaching
  * wal_receiver_timeout.
+ *
+ * If 'reportApplyTimestamp' is true, the latest apply timestamp is included.
  */
 static void
-XLogWalRcvSendReply(bool force, bool requestReply)
+XLogWalRcvSendReply(bool force, bool requestReply, bool reportApplyTimestamp)
 {
 	static XLogRecPtr writePtr = 0;
 	static XLogRecPtr flushPtr = 0;
 	XLogRecPtr	applyPtr;
 	static TimestampTz sendTime = 0;
 	TimestampTz now;
+	TimestampTz applyTimestamp = 0;
 
 	/*
 	 * If the user doesn't want status to be reported to the master, be sure
@@ -1092,10 +1115,8 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	 * We can compare the write and flush positions to the last message we
 	 * sent without taking any lock, but the apply position requires a spin
 	 * lock, so we don't check that unless something else has changed or 10
-	 * seconds have passed.  This means that the apply log position will
-	 * appear, from the master's point of view, to lag slightly, but since
-	 * this is only for reporting purposes and only on idle systems, that's
-	 * probably OK.
+	 * seconds have passed, or the force flag has been set (which happens when
+	 * apply feedback has been requested by the primary).
 	 */
 	if (!force
 		&& writePtr == LogstreamResult.Write
@@ -1108,7 +1129,10 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	/* Construct a new message */
 	writePtr = LogstreamResult.Write;
 	flushPtr = LogstreamResult.Flush;
-	applyPtr = GetXLogReplayRecPtr(NULL);
+	if (reportApplyTimestamp)
+		applyTimestamp = GetXLogReplayTimestamp(&applyPtr);
+	else
+		applyPtr = GetXLogReplayRecPtr(NULL);
 
 	resetStringInfo(&reply_message);
 	pq_sendbyte(&reply_message, 'r');
@@ -1116,6 +1140,7 @@ XLogWalRcvSendReply(bool force, bool requestReply)
 	pq_sendint64(&reply_message, flushPtr);
 	pq_sendint64(&reply_message, applyPtr);
 	pq_sendint64(&reply_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&reply_message, TimestampTzToIntegerTimestamp(applyTimestamp));
 	pq_sendbyte(&reply_message, requestReply ? 1 : 0);
 
 	/* Send it */
@@ -1220,8 +1245,8 @@ static void
 ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 {
 	WalRcvData *walrcv = WalRcv;
-
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
+	static TimestampTz lastRecordedTimestamp = 0;
 
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
@@ -1232,6 +1257,18 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
 	SpinLockRelease(&walrcv->mutex);
 
+	/*
+	 * Remember primary's timestamp at this WAL location.  We throw away
+	 * samples if they are coming too fast because we don't want to fill up
+	 * the finite circular buffer and have to throw away older samples.
+	 */
+	if (lastRecordedTimestamp < TimestampTzPlusMilliseconds(sendTime,
+															-MIN_TIME_BETWEEN_TIMESTAMPED_LSNS))
+	{
+		SetXLogReplayTimestampAtLsn(sendTime, walEnd);
+		lastRecordedTimestamp = sendTime;
+	}
+
 	if (log_min_messages <= DEBUG2)
 	{
 		char	   *sendtime;
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index f98475c..16d7abc 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1545,15 +1545,29 @@ ProcessStandbyReplyMessage(void)
 	XLogRecPtr	writePtr,
 				flushPtr,
 				applyPtr;
+	int64		applyLagUs;
 	bool		replyRequested;
+	TimestampTz now = GetCurrentTimestamp();
+	TimestampTz applyTimestamp;
 
 	/* the caller already consumed the msgtype byte */
 	writePtr = pq_getmsgint64(&reply_message);
 	flushPtr = pq_getmsgint64(&reply_message);
 	applyPtr = pq_getmsgint64(&reply_message);
 	(void) pq_getmsgint64(&reply_message);		/* sendTime; not used ATM */
+	applyTimestamp = IntegerTimestampToTimestampTz(pq_getmsgint64(&reply_message));
 	replyRequested = pq_getmsgbyte(&reply_message);
 
+	/* Compute the apply lag in milliseconds. */
+	if (applyTimestamp == 0)
+		applyLagUs = -1;
+	else
+#ifdef HAVE_INT64_TIMESTAMP
+		applyLagUs = now - applyTimestamp;
+#else
+		applyLagUs = (now - applyTimestamp) * 1000000;
+#endif
+
 	elog(DEBUG2, "write %X/%X flush %X/%X apply %X/%X%s",
 		 (uint32) (writePtr >> 32), (uint32) writePtr,
 		 (uint32) (flushPtr >> 32), (uint32) flushPtr,
@@ -1575,6 +1589,8 @@ ProcessStandbyReplyMessage(void)
 		walsnd->write = writePtr;
 		walsnd->flush = flushPtr;
 		walsnd->apply = applyPtr;
+		if (applyLagUs >= 0)
+			walsnd->applyLagUs = applyLagUs;
 		SpinLockRelease(&walsnd->mutex);
 	}
 
@@ -2745,7 +2761,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	8
+#define PG_STAT_GET_WAL_SENDERS_COLS	9
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2793,6 +2809,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		XLogRecPtr	write;
 		XLogRecPtr	flush;
 		XLogRecPtr	apply;
+		int64		applyLagUs;
 		int			priority;
 		WalSndState state;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
@@ -2807,6 +2824,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
+		applyLagUs = walsnd->applyLagUs;
 		priority = walsnd->sync_standby_priority;
 		SpinLockRelease(&walsnd->mutex);
 
@@ -2841,6 +2859,23 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				nulls[5] = true;
 			values[5] = LSNGetDatum(apply);
 
+			if (applyLagUs < 0)
+				nulls[6] = true;
+			else
+			{
+				Interval *applyLagInterval = palloc(sizeof(Interval));
+
+				applyLagInterval->month = 0;
+				applyLagInterval->day = 0;
+#ifdef HAVE_INT64_TIMESTAMP
+				applyLagInterval->time = applyLagUs;
+#else
+				applyLagInterval->time = applyLagUs / 1000000.0;
+#endif
+				nulls[6] = false;
+				values[6] = IntervalPGetDatum(applyLagInterval);
+			}
+
 			/*
 			 * Treat a standby such as a pg_basebackup background process
 			 * which always returns an invalid flush location, as an
@@ -2848,18 +2883,18 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 			 */
 			priority = XLogRecPtrIsInvalid(walsnd->flush) ? 0 : priority;
 
-			values[6] = Int32GetDatum(priority);
+			values[7] = Int32GetDatum(priority);
 
 			/*
 			 * More easily understood version of standby state. This is purely
 			 * informational, not different from priority.
 			 */
 			if (priority == 0)
-				values[7] = CStringGetTextDatum("async");
+				values[8] = CStringGetTextDatum("async");
 			else if (walsnd == sync_standby)
-				values[7] = CStringGetTextDatum("sync");
+				values[8] = CStringGetTextDatum("sync");
 			else
-				values[7] = CStringGetTextDatum("potential");
+				values[8] = CStringGetTextDatum("potential");
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
diff --git a/src/backend/utils/adt/timestamp.c b/src/backend/utils/adt/timestamp.c
index 7456808..b1501e2 100644
--- a/src/backend/utils/adt/timestamp.c
+++ b/src/backend/utils/adt/timestamp.c
@@ -1687,6 +1687,20 @@ IntegerTimestampToTimestampTz(int64 timestamp)
 #endif
 
 /*
+ * TimestampTzToIntegerTimestamp -- convert a native timestamp to int64 format
+ *
+ * When compiled with --enable-integer-datetimes, this is implemented as a
+ * no-op macro.
+ */
+#ifndef HAVE_INT64_TIMESTAMP
+int64
+TimestampTzToIntegerTimestamp(TimestampTz timestamp)
+{
+	return timestamp * 1000000;
+}
+#endif
+
+/*
  * TimestampDifference -- convert the difference between two timestamps
  *		into integer seconds and microseconds
  *
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index a7dcdae..c8be3ce 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -235,6 +235,9 @@ extern void GetXLogReceiptTime(TimestampTz *rtime, bool *fromStream);
 extern XLogRecPtr GetXLogReplayRecPtr(TimeLineID *replayTLI);
 extern XLogRecPtr GetXLogInsertRecPtr(void);
 extern XLogRecPtr GetXLogWriteRecPtr(void);
+extern void SetXLogReplayTimestamp(TimestampTz timestamp);
+extern void SetXLogReplayTimestampAtLsn(TimestampTz timestamp, XLogRecPtr lsn);
+extern TimestampTz GetXLogReplayTimestamp(XLogRecPtr *lsn);
 extern bool RecoveryIsPaused(void);
 extern void SetRecoveryPause(bool recoveryPause);
 extern TimestampTz GetLatestXTime(void);
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 7619c40..425b93b 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2712,7 +2712,7 @@ DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f
 DESCR("statistics: information about currently active backends");
 DATA(insert OID = 3318 (  pg_stat_get_progress_info           PGNSP PGUID 12 1 100 0 0 f f f f t t s r 1 0 2249 "25" "{25,23,26,26,20,20,20,20,20,20,20,20,20,20}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{cmdtype,pid,datid,relid,param1,param2,param3,param4,param5,param6,param7,param8,param9,param10}" _null_ _null_ pg_stat_get_progress_info _null_ _null_ _null_ ));
 DESCR("statistics: information about progress of backends running maintenance command");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,23,25}" "{o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25}" "{o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 7794aa5..4de43e8 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -46,6 +46,7 @@ typedef struct WalSnd
 	XLogRecPtr	write;
 	XLogRecPtr	flush;
 	XLogRecPtr	apply;
+	int64		applyLagUs;
 
 	/* Protects shared variables shown above. */
 	slock_t		mutex;
diff --git a/src/include/utils/timestamp.h b/src/include/utils/timestamp.h
index 865e993..31d6c62 100644
--- a/src/include/utils/timestamp.h
+++ b/src/include/utils/timestamp.h
@@ -228,9 +228,11 @@ extern bool TimestampDifferenceExceeds(TimestampTz start_time,
 #ifndef HAVE_INT64_TIMESTAMP
 extern int64 GetCurrentIntegerTimestamp(void);
 extern TimestampTz IntegerTimestampToTimestampTz(int64 timestamp);
+extern int64 TimestampTzToIntegerTimestamp(TimestampTz timestamp);
 #else
 #define GetCurrentIntegerTimestamp()	GetCurrentTimestamp()
 #define IntegerTimestampToTimestampTz(timestamp) (timestamp)
+#define TimestampTzToIntegerTimestamp(timestamp) (timestamp)
 #endif
 
 extern TimestampTz time_t_to_timestamptz(pg_time_t tm);
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 79f9b23..fc4b765 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1783,11 +1783,12 @@ pg_stat_replication| SELECT s.pid,
     w.write_location,
     w.flush_location,
     w.replay_location,
+    w.replay_lag,
     w.sync_priority,
     w.sync_state
    FROM pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, wait_event_type, wait_event, xact_start, query_start, backend_start, state_change, client_addr, client_hostname, client_port, backend_xid, backend_xmin, ssl, sslversion, sslcipher, sslbits, sslcompression, sslclientdn),
     pg_authid u,
-    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, sync_priority, sync_state)
+    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state)
   WHERE ((s.usesysid = u.oid) AND (s.pid = w.pid));
 pg_stat_ssl| SELECT s.pid,
     s.ssl,
0003-refactor-syncrep-exit-v11.patchapplication/octet-stream; name=0003-refactor-syncrep-exit-v11.patchDownload
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 2da9cba..57bde43 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -83,6 +83,64 @@ static bool SyncRepQueueIsOrderedByLSN(int mode);
  * ===========================================================
  */
 
+static bool
+SyncRepCheckEarlyExit(void)
+{
+	/*
+	 * If a wait for synchronous replication is pending, we can neither
+	 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
+	 * lead the client to believe that the transaction aborted, which
+	 * is not true: it's already committed locally. The former is no good
+	 * either: the client has requested synchronous replication, and is
+	 * entitled to assume that an acknowledged commit is also replicated,
+	 * which might not be true. So in this case we issue a WARNING (which
+	 * some clients may be able to interpret) and shut off further output.
+	 * We do NOT reset ProcDiePending, so that the process will die after
+	 * the commit is cleaned up.
+	 */
+	if (ProcDiePending)
+	{
+		ereport(WARNING,
+				(errcode(ERRCODE_ADMIN_SHUTDOWN),
+				 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * It's unclear what to do if a query cancel interrupt arrives.  We
+	 * can't actually abort at this point, but ignoring the interrupt
+	 * altogether is not helpful, so we just terminate the wait with a
+	 * suitable warning.
+	 */
+	if (QueryCancelPending)
+	{
+		QueryCancelPending = false;
+		ereport(WARNING,
+				(errmsg("canceling wait for synchronous replication due to user request"),
+				 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
+		SyncRepCancelWait();
+		return true;
+	}
+
+	/*
+	 * If the postmaster dies, we'll probably never get an
+	 * acknowledgement, because all the wal sender processes will exit. So
+	 * just bail out.
+	 */
+	if (!PostmasterIsAlive())
+	{
+		ProcDiePending = true;
+		whereToSendOutput = DestNone;
+		SyncRepCancelWait();
+		return true;
+	}
+
+	return false;
+}
+
 /*
  * Wait for synchronous replication, if requested by user.
  *
@@ -191,57 +249,9 @@ SyncRepWaitForLSN(XLogRecPtr lsn, bool commit)
 		if (syncRepState == SYNC_REP_WAIT_COMPLETE)
 			break;
 
-		/*
-		 * If a wait for synchronous replication is pending, we can neither
-		 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
-		 * lead the client to believe that the transaction aborted, which
-		 * is not true: it's already committed locally. The former is no good
-		 * either: the client has requested synchronous replication, and is
-		 * entitled to assume that an acknowledged commit is also replicated,
-		 * which might not be true. So in this case we issue a WARNING (which
-		 * some clients may be able to interpret) and shut off further output.
-		 * We do NOT reset ProcDiePending, so that the process will die after
-		 * the commit is cleaned up.
-		 */
-		if (ProcDiePending)
-		{
-			ereport(WARNING,
-					(errcode(ERRCODE_ADMIN_SHUTDOWN),
-					 errmsg("canceling the wait for synchronous replication and terminating connection due to administrator command"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit())
 			break;
-		}
-
-		/*
-		 * It's unclear what to do if a query cancel interrupt arrives.  We
-		 * can't actually abort at this point, but ignoring the interrupt
-		 * altogether is not helpful, so we just terminate the wait with a
-		 * suitable warning.
-		 */
-		if (QueryCancelPending)
-		{
-			QueryCancelPending = false;
-			ereport(WARNING,
-					(errmsg("canceling wait for synchronous replication due to user request"),
-					 errdetail("The transaction has already committed locally, but might not have been replicated to the standby.")));
-			SyncRepCancelWait();
-			break;
-		}
-
-		/*
-		 * If the postmaster dies, we'll probably never get an
-		 * acknowledgement, because all the wal sender processes will exit. So
-		 * just bail out.
-		 */
-		if (!PostmasterIsAlive())
-		{
-			ProcDiePending = true;
-			whereToSendOutput = DestNone;
-			SyncRepCancelWait();
-			break;
-		}
 
 		/*
 		 * Wait on latch.  Any condition that should wake us up will set the
0004-causal-reads-v11.patchapplication/octet-stream; name=0004-causal-reads-v11.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 1c4f796..27d5800 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -2749,6 +2749,35 @@ include_dir 'conf.d'
      across the cluster without problems if that is required.
     </para>
 
+    <sect2 id="runtime-config-replication-all">
+     <title>All Servers</title>
+     <para>
+      These parameters can be set on the primary or any standby.
+     </para>
+     <variablelist>
+      <varlistentry id="guc-causal-reads" xreflabel="causal_reads">
+       <term><varname>causal_reads</varname> (<type>boolean</type>)
+       <indexterm>
+        <primary><varname>causal_reads</> configuration parameter</primary>
+       </indexterm>
+       </term>
+       <listitem>
+        <para>
+         Enables causal consistency between transactions run on different
+         servers.  A transaction that is run on a standby
+         with <varname>causal_reads</> set to <literal>on</> is guaranteed
+         either to see the effects of all completed transactions run on the
+         primary with the setting on, or to receive an error "standby is not
+         available for causal reads".  Note that both transactions involved in
+         a causal dependency (a write on the primary followed by a read on any
+         server which must see the write) must be run with the setting on.
+         See <xref linkend="causal-reads"> for more details.
+        </para>
+       </listitem>
+      </varlistentry>
+     </variablelist>     
+    </sect2>
+
     <sect2 id="runtime-config-replication-sender">
      <title>Sending Server(s)</title>
 
@@ -2980,6 +3009,48 @@ include_dir 'conf.d'
       </listitem>
      </varlistentry>
 
+     <varlistentry>
+      <term><varname>causal_reads_timeout</varname> (<type>integer</type>)
+       <indexterm>
+        <primary><varname>causal_reads_timeout</> configuration parameter</primary>
+       </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies the maximum replay lag the primary will tolerate from a
+        standby before dropping it from the set of standbys available for
+        causal reads.
+       </para>
+       <para>
+        This setting is also used to control the <firstterm>leases</> used to
+        maintain the causal reads guarantee.  It must be set to a value which
+        is at least 4 times the maximum possible difference in system clocks
+        between the primary and standby servers, as described
+        in <xref linkend="causal-reads">.
+       </para>
+      </listitem>
+     </varlistentry>
+
+     <varlistentry id="guc-causal-reads-standby-names" xreflabel="causal-reads-standby-names">
+      <term><varname>causal_reads_standby_names</varname> (<type>string</type>)
+      <indexterm>
+       <primary><varname>causal_reads_standby_names</> configuration parameter</primary>
+      </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Specifies a comma-separated list of standby names that can support
+        <firstterm>causal reads</>, as described in
+        <xref linkend="causal-reads">.  Follows the same convention
+        as <link linkend="guc-synchronous-standby-names"><literal>synchronous_standby_name</></>.
+        The default is <literal>*</>, matching all standbys.
+       </para>
+       <para>
+        This setting has no effect if <varname>causal_reads_timeout</> is not set.
+       </para>
+      </listitem>
+     </varlistentry>
+
      </variablelist>
     </sect2>
 
diff --git a/doc/src/sgml/high-availability.sgml b/doc/src/sgml/high-availability.sgml
index d32ceb1..1b70808 100644
--- a/doc/src/sgml/high-availability.sgml
+++ b/doc/src/sgml/high-availability.sgml
@@ -1115,6 +1115,9 @@ primary_slot_name = 'node_a_slot'
     that it has replayed the transaction, making it visible to user queries.
     In simple cases, this allows for load balancing with causal consistency
     on a single hot standby.
+    (See also
+    <xref linkend="causal-reads"> which deals with multiple standbys and
+    standby failure.)
    </para>
 
    <para>
@@ -1233,6 +1236,119 @@ primary_slot_name = 'node_a_slot'
    </sect3>
   </sect2>
 
+  <sect2 id="causal-reads">
+   <title>Causal reads</title>
+   <indexterm>
+    <primary>causal reads</primary>
+    <secondary>in standby</secondary>
+   </indexterm>
+
+   <para>
+    The causal reads feature allows read-only queries to run on hot standby
+    servers without exposing stale data to the client, providing a form of
+    causal consistency.  Transactions can run on any standby with the
+    following guarantee about the visibility of preceding transactions: If you
+    set <varname>causal_reads</> to <literal>on</> in any pair of consecutive
+    transactions tx1, tx2 where tx2 begins after tx1 successfully returns,
+    then tx2 will either see tx1 or fail with a new error "standby is not
+    available for causal reads", no matter which server it runs on.  Although
+    the guarantee is expressed in terms of two individual transactions, the
+    GUC can also be set at session, role or system level to make the guarantee
+    generally, allowing for load balancing of applications that were not
+    designed with load balancing in mind.
+   </para>
+
+   <para>
+    In order to enable the feature, <varname>causal_reads_timeout</> must be
+    set to a non-zero value on the primary server.  The
+    GUC <varname>causal_reads_standby_names</> can be used to limit the set of
+    standbys that can join the dynamic set of causal reads standbys by
+    providing a comma-separated list of application names.  By default, all
+    standbys are candidates, if the feature is enabled.
+   </para>
+
+   <para>
+    The current set of servers that the primary considers to be available for
+    causal reads can be seen in
+    the <link linkend="monitoring-stats-views-table"> <literal>pg_stat_replication</></>
+    view.  Administrators, applications and load balancing middleware can use
+    this view to discover standbys that can currently handle causal reads
+    transactions without raising the error.  Since that information is only an
+    instantantaneous snapshot, clients should still be prepared for the error
+    to be raised at any time, and consider redirecting transactions to another
+    standby.
+   </para>
+
+   <para>
+    The advantages of the causal reads feature over simply
+    setting <varname>synchronous_commit</> to <literal>remote_apply</> are:
+    <orderedlist>
+      <listitem>
+       <para>
+        It allows the primary to wait for multiple standbys to replay
+        transactions.
+       </para>
+      </listitem>
+      <listitem>
+       <para>
+        It places a configurable limit on how much replay lag (and therefore
+        delay at commit time) the primary tolerates from standbys before it
+        drops them from the dynamic set of standbys it waits for.
+       </para>   
+      </listitem>
+      <listitem>
+       <para>
+        It upholds the causal reads guarantee during the transitions that
+        occur when new standbys are added or removed from the set of standbys,
+        including scenarios where contact has been lost between the primary
+        and standbys but the standby is still alive and running client
+        queries.
+       </para>
+      </listitem>
+    </orderedlist>
+   </para>
+
+   <para>
+    The protocol used to uphold the guarantee even in the case of network
+    failure depends on the system clocks of the primary and standby servers
+    being synchronized, with an allowance for a difference up to one quarter
+    of <varname>causal_reads_timeout</>.  For example,
+    if <varname>causal_reads_timeout</> is set to <literal>4s</>, then the
+    clocks must not be further than 1 second apart for the guarantee to be
+    upheld reliably during transitions.  The ubiquity of the Network Time
+    Protocol (NTP) on modern operating systems and availability of high
+    quality time servers makes it possible to choose a tolerance significantly
+    higher than the maximum expected clock difference.  An effort is
+    nevertheless made to detect and report misconfigured and faulty systems
+    with clock differences greater than the configured tolerance.
+   </para>
+
+   <note>
+    <para>
+     Current hardware clocks, NTP implementations and public time servers are
+     unlikely to allow the system clocks to differ more than tens or hundreds
+     of milliseconds, and systems synchronized with dedicated local time
+     servers may be considerably more accurate, but you should only consider
+     setting <varname>causal_reads_timeout</> below 4 seconds (allowing up to
+     1 second of clock difference) after researching your time synchronization
+     infrastructure thoroughly.
+    </para>  
+   </note>
+
+   <note>
+    <para>
+      While similar to synchronous replication in the sense that both involve
+      the primary server waiting for responses from standby servers, the
+      causal reads feature is not concerned with avoiding data loss.  A
+      primary configured for causal reads will drop all standbys that stop
+      responding or replay too slowly from the dynamic set that it waits for,
+      so you should consider configuring both synchronous replication and
+      causal reads if you need data loss avoidance guarantees and causal
+      consistency guarantees for load balancing.
+    </para>
+   </note>
+  </sect2>
+
   <sect2 id="continuous-archiving-in-standby">
    <title>Continuous archiving in standby</title>
 
@@ -1581,7 +1697,16 @@ if (!triggered)
     so there will be a measurable delay between primary and standby. Running the
     same query nearly simultaneously on both primary and standby might therefore
     return differing results. We say that data on the standby is
-    <firstterm>eventually consistent</firstterm> with the primary.  Once the
+    <firstterm>eventually consistent</firstterm> with the primary by default.
+    The data visible to a transaction running on a standby can be
+    made <firstterm>causally consistent</> with respect to a transaction that
+    has completed on the primary by setting <varname>causal_reads</>
+    to <literal>on</> in both transactions.  For more details,
+    see <xref linkend="causal-reads">.
+   </para>
+
+   <para>
+    Once the    
     commit record for a transaction is replayed on the standby, the changes
     made by that transaction will be visible to any new snapshots taken on
     the standby.  Snapshots may be taken at the start of each query or at the
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 7d63782..23d68d5 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1224,6 +1224,17 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
      <entry><type>text</></entry>
      <entry>Synchronous state of this standby server</entry>
     </row>
+    <row>
+     <entry><structfield>causal_reads_state</></entry>
+     <entry><type>text</></entry>
+     <entry>Causal reads state of this standby server.  This field will be
+     non-null only if <varname>cause_reads_timeout</> is set.  If a standby is
+     in <literal>available</> state, then it can currently serve causal reads
+     queries.  If it is not replaying fast enough or not responding to
+     keepalive messages, it will be in <literal>unavailable</> state, and if
+     it is currently transitioning to availability it will be
+     in <literal>joining</> state for a short time.</entry>
+    </row>
    </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index a65048b..b34a81d 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -2098,11 +2098,12 @@ RecordTransactionCommitPrepared(TransactionId xid,
 	END_CRIT_SECTION();
 
 	/*
-	 * Wait for synchronous replication, if required.
+	 * Wait for causal reads and synchronous replication, if required.
 	 *
 	 * Note that at this stage we have marked clog, but still show as running
 	 * in the procarray and continue to hold locks.
 	 */
+	CausalReadsWaitForLSN(recptr);
 	SyncRepWaitForLSN(recptr, true);
 }
 
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 65a0688..7b5a636 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1324,7 +1324,10 @@ RecordTransactionCommit(void)
 	 * in the procarray and continue to hold locks.
 	 */
 	if (wrote_xlog && markXidCommitted)
+	{
+		CausalReadsWaitForLSN(XactLastRecEnd);
 		SyncRepWaitForLSN(XactLastRecEnd, true);
+	}
 
 	/* remember end of last commit record */
 	XactLastCommitEnd = XactLastRecEnd;
@@ -5126,7 +5129,7 @@ XactLogCommitRecord(TimestampTz commit_time,
 	 * Check if the caller would like to ask standbys for immediate feedback
 	 * once this commit is applied.
 	 */
-	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY)
+	if (synchronous_commit >= SYNCHRONOUS_COMMIT_REMOTE_APPLY || causal_reads)
 		xl_xinfo.xinfo |= XACT_COMPLETION_APPLY_FEEDBACK;
 
 	/*
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index a53f07b..276ac12 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -664,7 +664,8 @@ CREATE VIEW pg_stat_replication AS
             W.replay_location,
             W.replay_lag,
             W.sync_priority,
-            W.sync_state
+            W.sync_state,
+            W.causal_reads_state
     FROM pg_stat_get_activity(NULL) AS S, pg_authid U,
             pg_stat_get_wal_senders() AS W
     WHERE S.usesysid = U.oid AND
diff --git a/src/backend/replication/README.causal_reads b/src/backend/replication/README.causal_reads
new file mode 100644
index 0000000..1fddd62
--- /dev/null
+++ b/src/backend/replication/README.causal_reads
@@ -0,0 +1,193 @@
+The causal reads guarantee says: If you run any two consecutive
+transactions tx1, tx2 where tx1 completes before tx2 begins, with
+causal_reads set to "on" in both transactions, tx2 will see tx1 or
+raise an error to complain that it can't guarantee causal consistency,
+no matter which servers (primary or any standby) you run each
+transaction on.
+
+When both transactions run on the primary, the guarantee is trivially
+upheld.
+
+To deal with read-only physical streaming standbys, the primary keeps
+track of a set of standbys that it considers to be currently
+"available" for causal reads, and sends a stream of "leases" to those
+standbys granting them the right to handle causal reads transactions
+for a short time without any further communication with the primary.
+
+In general, the primary provides the guarantee by waiting for all of
+the "available" standbys to report that they have applied a
+transaction.  However, the set of available standbys is dynamic, and
+things get more complicated during state transitions.  There are two
+types of transitions to consider:
+
+1.  unavailable->joining->available
+
+Standbys start out as "unavailable".  If a standby is unavailable and
+is applying fast enough and matches causal_reads_standby_names, the
+primary transitions it to "available", but first it sets it to
+"joining" until it is sure that any transaction committed while it was
+unavailable has definitely been applied on the standby.  This closes a
+race that would otherwise exist if we moved directly to available
+state: tx1 might not wait for a given standby because it's
+unavailable, then a lease might be granted, and then tx2 might run a
+causal reads transaction without error but see stale data.  The
+joining state acts as an airlock: while in joining state, the primary
+waits for that standby to replay causal reads transactions in
+anticipation of the move to available, but it doesn't progress to
+available state and grant a lease to the standby until everything
+preceding joining state has also been applied.
+
+2.  available->unavailable
+
+If a standby is not applying fast enough or not responding to
+keepalive messages, then the primary kicks that standby out of the
+dynamic set of available standbys, that is, marks it as "unavailable".
+In order to make sure that the standby has started rejecting causal
+reads transactions, it needs to revoke the lease it most recently
+granted.  It does that by waiting for the lease to expire before
+allowing any causal reads commits to return.  (In future there could
+be a fast-path revocation message which waits for a serial-numbered
+acknowledgement to reduce waiting in the case where the standby is
+lagging but still reachable and responding).
+
+The rest of this document illustrates how clock skew affects the
+available->unavailable transition.
+
+The following 4 variables are derived from a single GUC, and these
+values will be used in the following illustrations:
+
+causal_reads_timeout = 4s
+lease_time           = 4s (= causal_reads_timeout)
+keepalive_time       = 2s (= lease_time / 2)
+max_clock_skew       = 1s (= lease_time / 4)
+
+Every keepalive_time, the primary transmits a lease that expires at
+local_clock_time + lease_time - max_clock_skew, shown in the following
+diagram as 't' for transmission time and '|' for expiry time.  If
+contact is lost with a standby, the primary will wait until sent_time
++ lease_time for the most recently granted lease to expire, shown on
+the following diagram 'x', to be sure that the standby's clock has
+reached the expiry time even if its clock differs by up to
+max_clock_skew.  In other words, the primary tells the standby that
+the expiry time is at one time, but it trusts that the standby will
+surely agree if it gives it some extra time.  The extra time is
+max_clock_skew.  If the clocks differ by more than max_clock_skew, all
+bets are off (but see below for attempt to detect obvious cases).
+
+0     1     2     3     4     5     6     7     8     9
+t-----------------|-----x
+            t-----------------|-----x
+                        t-----------------|-----x
+                                    t-----------------|...
+                                                t------...
+
+A standby whose clock is 2 seconds ahead of the primary's clock
+perceives gaps in the stream of leases, and will reject causal_reads
+transactions in those intervals.  The causal reads guarantee is
+upheld, but spurious errors are raised between leases, as a
+consequence of the clock skew being greater than max_clock_skew.  In
+the following diagram 'r' shows reception time, and the timeline along
+the top shows the standby's local clock time.
+
+2     3     4     5     6     7     8     9    10    11
+r-----|
+            r-----|
+                        r-----|
+                                    r-----|
+                                                r-----|
+
+If there were no network latency, a standby whose clock is exactly 1
+second ahead of the primary's clock would perceive the stream of
+leases as being replaced just in time, so there is no gap.  Since in
+reality the time of receipt is some time after the time of
+transmission due to network latency, if the standby's clock is exactly
+1 second behind, then there will be small network-latency-sized gaps
+before the next lease arrives, but still no correctness problem with
+respect to the causal reads guarantee.
+
+1     2     3     4     5     6     7     8     9    10
+r-----------|
+            r-----------|
+                        r-----------|
+                                    r-----------|
+                                                r------...
+
+A standby whose clock is perfectly in sync with the primary's
+perceives the stream of leases overlapping (this matches the primary's
+perception of the leases it sent):
+
+0     1     2     3     4     5     6     7     8     9
+r-----------------|
+            r-----------------|
+                        r-----------------|
+                                    r-----------------|
+                                                r------...
+
+A standby whose clock is exactly 1 second behind the primary's
+perceives the stream of leases as overlapping even more, but the time
+of expiry as judged by the standby is no later than the time the
+primary will wait for if required ('x').  That is, if contact is lost
+with the standby, the primary can still reliably hold up causal reads
+commits until the standby has started raising the error in
+causal_reads transactions.
+
+-1    0     1     2     3     4     5     6     7     8
+r-----------------------|
+            r-----------------------|
+                        r-----------------------|
+                                    r------------------...
+                                                r------...
+
+
+A standby whose clock is 2 seconds behind the primary's would perceive
+the stream of leases overlapping even more, and the primary would no
+longer be able to wait for a lease to expire if it wanted to revoke
+it.  But because the expiry time is after local_clock_time +
+lease_time, the standby can immediately see that its own clock must be
+more than 1 second behind the primary's, so it ignores the lease and
+logs a clock skew warning.  In the following diagram a lease expiry
+time that is obviously generated by a primary with a clock set too far
+in the future compared to the local clock is shown with a '!'.
+
+-2    -1    0     1     2     3     4     5     6     7
+r-----------------------------!
+            r-----------------------------!
+                        r-----------------------------!
+                                    r------------------...
+                                                r------...
+
+A danger window exists when the standby's clock is more than
+max_clock_skew behind the primary's clock, but not more than
+max_clock_skew + network latency time behind.  If the clock difference
+is in that range, then the algorithm presented above which is based on
+time of receipt cannot detect that the local clock is too far behind.
+The consequence of this problem could be as follows:
+
+1.  The standby loses contact with the primary due to a network fault.
+
+2.  The primary decides to drop the standby from the set of available
+    causal reads standbys due to lack of keepalive responses or
+    excessive lag, which necessitates holding up commits of causal
+    reads transactions until the most recently sent lease expires, in
+    the belief that the standby will definitely have started raising
+    the 'causal reads unavailable' error in causal reads transactions
+    by that time, if it is still alive and servicing requests.
+
+3.  The standby still has clients connected and running queries.
+
+4.  Due to clock skew in the problematic range, in the standby's
+    opinion the lease lasts slightly longer than the primary waits.
+
+5.  For a short window at most the duration of the network latency
+    time, clients running causal reads transactions are allowed to see
+    potentially stale data.
+
+For this reason we say that the causal reads guarantee only holds as
+long as the absolute difference between the system clocks of the
+machines is no more than max_clock_skew.  The theory is that NTP makes
+it possible to reason about the maximum possible clock difference
+between machines and choose a value that allows for a much larger
+difference.  However, we do make a best effort attempt to detect
+wildly divergent systems as described above, to catch the case of
+servers not running a correctly configured ntp daemon, or with a clock
+so far out of whack that ntp refuses to fix it.
\ No newline at end of file
diff --git a/src/backend/replication/syncrep.c b/src/backend/replication/syncrep.c
index 57bde43..eb46e49 100644
--- a/src/backend/replication/syncrep.c
+++ b/src/backend/replication/syncrep.c
@@ -57,6 +57,11 @@
 #include "utils/builtins.h"
 #include "utils/ps_status.h"
 
+/* GUC variables */
+int causal_reads_timeout;
+bool causal_reads;
+char *causal_reads_standby_names;
+
 /* User-settable parameters for sync rep */
 char	   *SyncRepStandbyNames;
 
@@ -69,7 +74,7 @@ static int	SyncRepWaitMode = SYNC_REP_NO_WAIT;
 
 static void SyncRepQueueInsert(int mode);
 static void SyncRepCancelWait(void);
-static int	SyncRepWakeQueue(bool all, int mode);
+static int	SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn);
 
 static int	SyncRepGetStandbyPriority(void);
 
@@ -142,6 +147,198 @@ SyncRepCheckEarlyExit(void)
 }
 
 /*
+ * Check if we can stop waiting for causal consistency.  We can stop waiting
+ * when the following conditions are met:
+ *
+ * 1.  All walsenders currently in 'joining' or 'available' state have
+ * applied the target LSN.
+ *
+ * 2.  Any stall periods caused by standbys dropping out of 'available' state
+ * have passed, so that we can be sure that their leases have expired and they
+ * have started rejecting causal reads transactions.
+ *
+ * The output parameter 'waitingFor' is set to the number of nodes we are
+ * currently waiting for.  The output parameters 'stallTimeMillis' is set to
+ * the number of milliseconds we need to wait for to observe any current
+ * commit stall.
+ *
+ * Returns true if commit can return control, because every standby has either
+ * applied the LSN or started rejecting causal_reads transactions.
+ */
+static bool
+CausalReadsCommitCanReturn(XLogRecPtr XactCommitLSN,
+						   int *waitingFor,
+						   long *stallTimeMillis)
+{
+	int i;
+	TimestampTz now;
+
+	/* Count how many joining/available nodes we are waiting for. */
+	*waitingFor = 0;
+	for (i = 0; i < max_wal_senders; ++i)
+	{
+		WalSnd *walsnd = &WalSndCtl->walsnds[i];
+
+		/*
+		 * Assuming atomic read of pid_t, we can check walsnd->pid without
+		 * acquiring the spinlock to avoid memory synchronization costs for
+		 * unused walsender slots.  We see a value that existed sometime at
+		 * least as recently as the last memory barrier.
+		 */
+		if (walsnd->pid != 0)
+		{
+			/*
+			 * We need to hold the spinlock to read LSNs, because we can't be
+			 * sure they can be read atomically.
+			 */
+			SpinLockAcquire(&walsnd->mutex);
+			if (walsnd->pid != 0 && walsnd->causal_reads_state >= WALSNDCRSTATE_JOINING)
+			{
+				if (walsnd->apply < XactCommitLSN)
+					++*waitingFor;
+			}
+			SpinLockRelease(&walsnd->mutex);
+		}
+	}
+
+	/* Check if there is a stall in progress that we need to observe. */
+	now = GetCurrentTimestamp();
+	LWLockAcquire(SyncRepLock, LW_SHARED);
+	if (WalSndCtl->stall_causal_reads_until > now)
+	{
+		long seconds;
+		int usecs;
+
+		/* Compute how long we have to wait, rounded up to nearest ms. */
+		TimestampDifference(now, WalSndCtl->stall_causal_reads_until,
+							&seconds, &usecs);
+		*stallTimeMillis = seconds * 1000 + (usecs + 999) / 1000;
+	}
+	else
+		*stallTimeMillis = 0;
+	LWLockRelease(SyncRepLock);
+
+	/* We are done if we are not waiting for any nodes or stalls. */
+	return *waitingFor == 0 && *stallTimeMillis == 0;
+}
+
+/*
+ * Wait for causal consistency in causal_reads mode, if requested by user.
+ */
+void
+CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN)
+{
+	long stallTimeMillis;
+	int waitingFor;
+	char *ps_display_buffer = NULL;
+
+	/* Leave if we aren't in causal_reads mode. */
+	if (!causal_reads)
+		return;
+
+	for (;;)
+	{
+		/* Reset latch before checking state. */
+		ResetLatch(MyLatch);
+
+		/*
+		 * Join the queue to be woken up if any causal reads joining/available
+		 * standby applies XactCommitLSN or the set of causal reads standbys
+		 * changes (if we aren't already in the queue).  We don't actually know
+		 * if we need to wait for any peers to reach the target LSN yet, but
+		 * we have to register just in case before checking the walsenders'
+		 * state to avoid a race condition that could occur if we did it after
+		 * calling CausalReadsCommitCanReturn.  (SyncRepWaitForLSN doesn't
+		 * have to do this because it can check the highest-seen LSN in
+		 * walsndctl->lsn[mode] which is protected by SyncRepLock, the same
+		 * lock as the queues.  We can't do that here, because there is no
+		 * single highest-seen LSN that is useful.  We must check
+		 * walsnd->apply for all relevant walsenders.  Therefore we must
+		 * register for notifications first, so that we can be notified via
+		 * our latch of any standby applying the LSN we're interested in after
+		 * we check but before we start waiting, or we could wait forever for
+		 * something that has already happened.)
+		 */
+		LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+		if (MyProc->syncRepState != SYNC_REP_WAITING)
+		{
+			MyProc->waitLSN = XactCommitLSN;
+			MyProc->syncRepState = SYNC_REP_WAITING;
+			SyncRepQueueInsert(SYNC_REP_WAIT_CAUSAL_READS);
+			Assert(SyncRepQueueIsOrderedByLSN(SYNC_REP_WAIT_CAUSAL_READS));
+		}
+		LWLockRelease(SyncRepLock);
+
+		/* Check if we're done. */
+		if (CausalReadsCommitCanReturn(XactCommitLSN, &waitingFor, &stallTimeMillis))
+		{
+			SyncRepCancelWait();
+			break;
+		}
+
+		Assert(waitingFor > 0 || stallTimeMillis > 0);
+
+		/* If we aren't actually waiting for any standbys, leave the queue. */
+		if (waitingFor == 0)
+			SyncRepCancelWait();
+
+		/* Update the ps title. */
+		if (update_process_title)
+		{
+			char buffer[80];
+
+			/* Remember the old value if this is our first update. */
+			if (ps_display_buffer == NULL)
+			{
+				int len;
+				const char *ps_display = get_ps_display(&len);
+
+				ps_display_buffer = palloc(len + 1);
+				memcpy(ps_display_buffer, ps_display, len);
+				ps_display_buffer[len] = '\0';
+			}
+
+			snprintf(buffer, sizeof(buffer),
+					 "waiting for %d peer(s) to apply %X/%X%s",
+					 waitingFor,
+					 (uint32) (XactCommitLSN >> 32), (uint32) XactCommitLSN,
+					 stallTimeMillis > 0 ? " (stalling)" : "");
+			set_ps_display(buffer, false);
+		}
+
+		/* Check if we need to exit early due to postmaster death etc. */
+		if (SyncRepCheckEarlyExit()) /* Calls SyncRepCancelWait() if true. */
+			break;
+
+		/*
+		 * If are still waiting for peers, then we wait for any joining or
+		 * available peer to reach the LSN (or possibly stop being in one of
+		 * those states or go away).
+		 *
+		 * If not, there must be a non-zero stall time, so we wait for that to
+		 * elapse.
+		 */
+		if (waitingFor > 0)
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH, -1);
+		else
+			WaitLatch(MyLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH | WL_TIMEOUT,
+					  stallTimeMillis);
+	}
+
+	/* There is no way out of the loop that could leave us in the queue. */
+	Assert(SHMQueueIsDetached(&(MyProc->syncRepLinks)));
+	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
+	MyProc->waitLSN = 0;
+
+	/* Restore the ps display. */
+	if (ps_display_buffer != NULL)
+	{
+		set_ps_display(ps_display_buffer, false);
+		pfree(ps_display_buffer);
+	}
+}
+
+/*
  * Wait for synchronous replication, if requested by user.
  *
  * Initially backends start in state SYNC_REP_NOT_WAITING and then
@@ -424,6 +621,53 @@ SyncRepGetSynchronousStandby(void)
 }
 
 /*
+ * Check if the current WALSender process's application_name matches a name in
+ * causal_reads_standby_names (including '*' for wildcard).
+ */
+bool
+CausalReadsPotentialStandby(void)
+{
+	char *rawstring;
+	List	   *elemlist;
+	ListCell   *l;
+	bool		found = false;
+
+	/* If the feature is disable, then no. */
+	if (causal_reads_timeout == 0)
+		return false;
+
+	/* Need a modifiable copy of string */
+	rawstring = pstrdup(causal_reads_standby_names);
+
+	/* Parse string into list of identifiers */
+	if (!SplitIdentifierString(rawstring, ',', &elemlist))
+	{
+		/* syntax error in list */
+		pfree(rawstring);
+		list_free(elemlist);
+		/* GUC machinery will have already complained - no need to do again */
+		return 0;
+	}
+
+	foreach(l, elemlist)
+	{
+		char	   *standby_name = (char *) lfirst(l);
+
+		if (pg_strcasecmp(standby_name, application_name) == 0 ||
+			pg_strcasecmp(standby_name, "*") == 0)
+		{
+			found = true;
+			break;
+		}
+	}
+
+	pfree(rawstring);
+	list_free(elemlist);
+
+	return found;
+}
+
+/*
  * Update the LSNs on each queue based upon our latest state. This
  * implements a simple policy of first-valid-standby-releases-waiter.
  *
@@ -431,23 +675,27 @@ SyncRepGetSynchronousStandby(void)
  * perhaps also which information we store as well.
  */
 void
-SyncRepReleaseWaiters(void)
+SyncRepReleaseWaiters(bool walsender_cr_available_or_joining)
 {
 	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	WalSnd	   *syncWalSnd;
 	int			numwrite = 0;
 	int			numflush = 0;
 	int			numapply = 0;
+	int			numcausalreadsapply = 0;
+	bool		is_highest_priority_sync_standby;
 
 	/*
 	 * If this WALSender is serving a standby that is not on the list of
-	 * potential sync standbys then we have nothing to do. If we are still
-	 * starting up, still running base backup or the current flush position
-	 * is still invalid, then leave quickly also.
+	 * potential sync standbys and not in a state that causal_reads waits for,
+	 * then we have nothing to do. If we are still starting up, still running
+	 * base backup or the current flush position is still invalid, then leave
+	 * quickly also.
 	 */
-	if (MyWalSnd->sync_standby_priority == 0 ||
-		MyWalSnd->state < WALSNDSTATE_STREAMING ||
-		XLogRecPtrIsInvalid(MyWalSnd->flush))
+	if (!walsender_cr_available_or_joining &&
+		(MyWalSnd->sync_standby_priority == 0 ||
+		 MyWalSnd->state < WALSNDSTATE_STREAMING ||
+		 XLogRecPtrIsInvalid(MyWalSnd->flush)))
 		return;
 
 	/*
@@ -457,13 +705,19 @@ SyncRepReleaseWaiters(void)
 	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
 	syncWalSnd = SyncRepGetSynchronousStandby();
 
-	/* We should have found ourselves at least */
-	Assert(syncWalSnd != NULL);
+	/*
+	 * If we aren't managing the highest priority standby then make a note of
+	 * that so we can announce a takeover in the log if we ever get that job.
+	 */
+	is_highest_priority_sync_standby = syncWalSnd == MyWalSnd;
+	if (!is_highest_priority_sync_standby)
+		announce_next_takeover = true;
 
 	/*
-	 * If we aren't managing the highest priority standby then just leave.
+	 * If we aren't managing the highest priority standby or a standby in
+	 * causal reads 'joining' or 'available' state, then just leave.
 	 */
-	if (syncWalSnd != MyWalSnd)
+	if (!is_highest_priority_sync_standby && !walsender_cr_available_or_joining)
 	{
 		LWLockRelease(SyncRepLock);
 		announce_next_takeover = true;
@@ -472,24 +726,45 @@ SyncRepReleaseWaiters(void)
 
 	/*
 	 * Set the lsn first so that when we wake backends they will release up to
-	 * this location.
+	 * this location.  For the single-standby synchronous commit levels, we
+	 * only do this if we are the current synchronous standby and we are
+	 * advancing the LSN further than it has been advanced before, so that
+	 * SyncRepWaitForLSN can skip waiting in some cases.
 	 */
-	if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
-		numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
-	{
-		walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
-		numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH);
-	}
-	if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+	if (is_highest_priority_sync_standby)
 	{
-		walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
-		numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY);
+		if (walsndctl->lsn[SYNC_REP_WAIT_WRITE] < MyWalSnd->write)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_WRITE] = MyWalSnd->write;
+			numwrite = SyncRepWakeQueue(false, SYNC_REP_WAIT_WRITE,
+										MyWalSnd->write);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_FLUSH] < MyWalSnd->flush)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_FLUSH] = MyWalSnd->flush;
+			numflush = SyncRepWakeQueue(false, SYNC_REP_WAIT_FLUSH,
+										MyWalSnd->flush);
+		}
+		if (walsndctl->lsn[SYNC_REP_WAIT_APPLY] < MyWalSnd->apply)
+		{
+			walsndctl->lsn[SYNC_REP_WAIT_APPLY] = MyWalSnd->apply;
+			numapply = SyncRepWakeQueue(false, SYNC_REP_WAIT_APPLY,
+										MyWalSnd->apply);
+		}
 	}
 
+	/*
+	 * For causal_reads, all walsenders currently in available or joining
+	 * state must reach the LSN on their own, and standbys will reach LSNs in
+	 * any order.  It doesn't make sense to keep the highest seen LSN in a
+	 * single walsndctl->lsn element.  (CausalReadsWaitForLSN has handling for
+	 * LSNs that have already been reached).
+	 */
+	if (walsender_cr_available_or_joining)
+		numcausalreadsapply =
+			SyncRepWakeQueue(false, SYNC_REP_WAIT_CAUSAL_READS,
+							 MyWalSnd->apply);
+
 	LWLockRelease(SyncRepLock);
 
 	elog(DEBUG3, "released %d procs up to write %X/%X, %d procs up to flush %X/%X, %d procs up to apply %X/%x",
@@ -501,7 +776,7 @@ SyncRepReleaseWaiters(void)
 	 * If we are managing the highest priority standby, though we weren't
 	 * prior to this, then announce we are now the sync standby.
 	 */
-	if (announce_next_takeover)
+	if (is_highest_priority_sync_standby && announce_next_takeover)
 	{
 		announce_next_takeover = false;
 		ereport(LOG,
@@ -576,9 +851,8 @@ SyncRepGetStandbyPriority(void)
  * Must hold SyncRepLock.
  */
 static int
-SyncRepWakeQueue(bool all, int mode)
+SyncRepWakeQueue(bool all, int mode, XLogRecPtr lsn)
 {
-	volatile WalSndCtlData *walsndctl = WalSndCtl;
 	PGPROC	   *proc = NULL;
 	PGPROC	   *thisproc = NULL;
 	int			numprocs = 0;
@@ -595,7 +869,7 @@ SyncRepWakeQueue(bool all, int mode)
 		/*
 		 * Assume the queue is ordered by LSN
 		 */
-		if (!all && walsndctl->lsn[mode] < proc->waitLSN)
+		if (!all && lsn < proc->waitLSN)
 			return numprocs;
 
 		/*
@@ -655,7 +929,7 @@ SyncRepUpdateSyncStandbysDefined(void)
 			int			i;
 
 			for (i = 0; i < NUM_SYNC_REP_WAIT_MODE; i++)
-				SyncRepWakeQueue(true, i);
+				SyncRepWakeQueue(true, i, InvalidXLogRecPtr);
 		}
 
 		/*
@@ -707,13 +981,31 @@ SyncRepQueueIsOrderedByLSN(int mode)
 #endif
 
 /*
+ * Make sure that CausalReadsWaitForLSN can't return until after the given
+ * lease expiry time has been reached.  In other words, revoke the lease.
+ *
+ * Wake up all backends waiting in CausalReadsWaitForLSN, because the set of
+ * available/joining peers has changed, and there is a new stall time they
+ * need to observe.
+ */
+void
+CausalReadsBeginStall(TimestampTz lease_expiry_time)
+{
+	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
+	WalSndCtl->stall_causal_reads_until =
+		Max(WalSndCtl->stall_causal_reads_until, lease_expiry_time);
+	SyncRepWakeQueue(true, SYNC_REP_WAIT_CAUSAL_READS, InvalidXLogRecPtr);
+	LWLockRelease(SyncRepLock);
+}
+
+/*
  * ===========================================================
  * Synchronous Replication functions executed by any process
  * ===========================================================
  */
 
 bool
-check_synchronous_standby_names(char **newval, void **extra, GucSource source)
+check_standby_names(char **newval, void **extra, GucSource source)
 {
 	char	   *rawstring;
 	List	   *elemlist;
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 7eab357..267465c 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -55,6 +55,7 @@
 #include "libpq/pqformat.h"
 #include "libpq/pqsignal.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
 #include "replication/walreceiver.h"
 #include "replication/walsender.h"
 #include "storage/ipc.h"
@@ -149,7 +150,8 @@ static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr);
 static void XLogWalRcvFlush(bool dying);
 static void XLogWalRcvSendReply(bool force, bool requestReply, bool includeApplyTimestamp);
 static void XLogWalRcvSendHSFeedback(bool immed);
-static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
+static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+								  TimestampTz *causalReadsUntil);
 
 /* Signal handlers */
 static void WalRcvSigHupHandler(SIGNAL_ARGS);
@@ -845,6 +847,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 	XLogRecPtr	walEnd;
 	TimestampTz sendTime;
 	bool		replyRequested;
+	TimestampTz causalReadsLease;
 
 	resetStringInfo(&incoming_message);
 
@@ -865,7 +868,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				walEnd = pq_getmsgint64(&incoming_message);
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, NULL);
 
 				buf += hdrlen;
 				len -= hdrlen;
@@ -877,7 +880,7 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				bool reportApplyTimestamp = false;
 
 				/* copy message to StringInfo */
-				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
+				hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char) + sizeof(int64);
 				if (len != hdrlen)
 					ereport(ERROR,
 							(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -889,8 +892,10 @@ XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len)
 				sendTime = IntegerTimestampToTimestampTz(
 										  pq_getmsgint64(&incoming_message));
 				replyRequested = pq_getmsgbyte(&incoming_message);
+				causalReadsLease = IntegerTimestampToTimestampTz(
+					pq_getmsgint64(&incoming_message));
 
-				ProcessWalSndrMessage(walEnd, sendTime);
+				ProcessWalSndrMessage(walEnd, sendTime, &causalReadsLease);
 
 				/*
 				 * If no apply timestamps have been sent at the request of the
@@ -1239,15 +1244,52 @@ XLogWalRcvSendHSFeedback(bool immed)
  * Update shared memory status upon receiving a message from primary.
  *
  * 'walEnd' and 'sendTime' are the end-of-WAL and timestamp of the latest
- * message, reported by primary.
+ * message, reported by primary.  'causalReadsLease' is a pointer to
+ * the time the primary promises that this standby can safely claim to be
+ * causally consistent, to 0 if it cannot, or a NULL pointer for no change.
  */
 static void
-ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
+ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime,
+					  TimestampTz *causalReadsLease)
 {
 	WalRcvData *walrcv = WalRcv;
 	TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
 	static TimestampTz lastRecordedTimestamp = 0;
 
+	/* Sanity check for the causalReadsLease time. */
+	if (causalReadsLease != NULL && *causalReadsLease != 0)
+	{
+		/* Deduce max_clock_skew from the causalReadsLease and sendTime. */
+#ifdef HAVE_INT64_TIMESTAMP
+		int64 diffMillis = (*causalReadsLease - sendTime) / 1000;
+#else
+		int64 diffMillis = (*causalReadsLease - sendTime) * 1000;
+#endif
+		int64 max_clock_skew = diffMillis / (CAUSAL_READS_CLOCK_SKEW_RATIO - 1);
+
+		if (sendTime > TimestampTzPlusMilliseconds(lastMsgReceiptTime, max_clock_skew))
+		{
+			/*
+			 * The primary's clock is more than max_clock_skew + network
+			 * latency ahead of the standby's clock.  (If the primary's clock
+			 * is more than max_clock_skew ahead of the standby's clock, but
+			 * by less than the network latency, then there isn't much we can
+			 * do to detect that; but it still seems useful to have this basic
+			 * sanity check for wildly misconfigured servers.)
+			 */
+			elog(LOG, "the primary server's clock time is too far ahead");
+			causalReadsLease = NULL;
+		}
+		/*
+		 * We could also try to detect cases where sendTime is more than
+		 * max_clock_skew in the past according to the standby's clock, but
+		 * that is indistinguishable from network latency/buffering, so we
+		 * could produce misleading error messages; if we do nothing, the
+		 * consequence is 'standby is not available for causal reads' errors
+		 * which should cause the user to investigate.
+		 */
+	}
+
 	/* Update shared-memory status */
 	SpinLockAcquire(&walrcv->mutex);
 	if (walrcv->latestWalEnd < walEnd)
@@ -1255,6 +1297,8 @@ ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
 	walrcv->latestWalEnd = walEnd;
 	walrcv->lastMsgSendTime = sendTime;
 	walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
+	if (causalReadsLease != NULL)
+		walrcv->causalReadsLease = *causalReadsLease;
 	SpinLockRelease(&walrcv->mutex);
 
 	/*
diff --git a/src/backend/replication/walreceiverfuncs.c b/src/backend/replication/walreceiverfuncs.c
index 5f6e423..e502f74 100644
--- a/src/backend/replication/walreceiverfuncs.c
+++ b/src/backend/replication/walreceiverfuncs.c
@@ -28,6 +28,7 @@
 #include "replication/walreceiver.h"
 #include "storage/pmsignal.h"
 #include "storage/shmem.h"
+#include "utils/guc.h"
 #include "utils/timestamp.h"
 
 WalRcvData *WalRcv = NULL;
@@ -374,3 +375,21 @@ GetReplicationTransferLatency(void)
 
 	return ms;
 }
+
+/*
+ * Used by snapmgr to check if this standby has a valid lease, granting it the
+ * right to consider itself available for causal reads.
+ */
+bool
+WalRcvCausalReadsAvailable(void)
+{
+	WalRcvData *walrcv = WalRcv;
+	TimestampTz now = GetCurrentTimestamp();
+	bool result;
+
+	SpinLockAcquire(&walrcv->mutex);
+	result = walrcv->causalReadsLease != 0 && now <= walrcv->causalReadsLease;
+	SpinLockRelease(&walrcv->mutex);
+
+	return result;
+}
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 16d7abc..b4dad72 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -154,9 +154,20 @@ static StringInfoData tmpbuf;
  */
 static TimestampTz last_reply_timestamp = 0;
 
+static TimestampTz last_keepalive_timestamp = 0;
+
 /* Have we sent a heartbeat message asking for reply, since last reply? */
 static bool waiting_for_ping_response = false;
 
+/* How long do need to stay in JOINING state? */
+static XLogRecPtr causal_reads_joining_until = 0;
+
+/* The last causal reads lease sent to the standby. */
+static TimestampTz causal_reads_last_lease = 0;
+
+/* Is this WALSender listed in causal_reads_standby_names? */
+static bool am_potential_causal_reads_standby = false;
+
 /*
  * While streaming WAL in Copy mode, streamingDoneSending is set to true
  * after we have sent CopyDone. We should not send any more CopyData messages
@@ -242,6 +253,57 @@ InitWalSender(void)
 	SendPostmasterSignal(PMSIGNAL_ADVANCE_STATE_MACHINE);
 }
 
+ /*
+ * If we are exiting unexpectedly, we may need to communicate with concurrent
+ * causal_reads commits to maintain the causal consistency guarantee.
+ */
+static void
+PrepareUncleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * We've lost contact with the standby, but it may still be alive.  We
+		 * can't let any causal_reads transactions return until we've stalled
+		 * for long enough for a zombie standby to start raising errors
+		 * because its lease has expired.
+		 */
+		elog(LOG, "standby \"%s\" is lost (no longer available for causal reads)", application_name);
+		CausalReadsBeginStall(causal_reads_last_lease);
+
+		/*
+		 * We set the state to a lower level _after_ beginning the stall,
+		 * otherwise there would be a tiny window where commits could return
+		 * without observing the stall.
+		 */
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
+/*
+ * We are shutting down because we received a goodbye message from the
+ * walreceiver.
+ */
+static void
+PrepareCleanExit(void)
+{
+	if (MyWalSnd->causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+	{
+		/*
+		 * The standby is shutting down, so it won't be running any more
+		 * transactions.  It is therefore safe to stop waiting for it, and no
+		 * stall is necessary.
+		 */
+		elog(LOG, "standby \"%s\" is leaving (no longer available for causal reads)", application_name);
+
+		SpinLockAcquire(&MyWalSnd->mutex);
+		MyWalSnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+		SpinLockRelease(&MyWalSnd->mutex);
+	}
+}
+
 /*
  * Clean up after an error.
  *
@@ -266,7 +328,10 @@ WalSndErrorCleanup(void)
 
 	replication_active = false;
 	if (walsender_ready_to_stop)
+	{
+		PrepareUncleanExit();
 		proc_exit(0);
+	}
 
 	/* Revert back to startup state */
 	WalSndSetState(WALSNDSTATE_STARTUP);
@@ -278,6 +343,8 @@ WalSndErrorCleanup(void)
 static void
 WalSndShutdown(void)
 {
+	PrepareUncleanExit();
+
 	/*
 	 * Reset whereToSendOutput to prevent ereport from attempting to send any
 	 * more messages to the standby.
@@ -1388,6 +1455,7 @@ ProcessRepliesIfAny(void)
 		if (r < 0)
 		{
 			/* unexpected error or EOF */
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1404,6 +1472,7 @@ ProcessRepliesIfAny(void)
 		resetStringInfo(&reply_message);
 		if (pq_getmessage(&reply_message, 0))
 		{
+			PrepareUncleanExit();
 			ereport(COMMERROR,
 					(errcode(ERRCODE_PROTOCOL_VIOLATION),
 					 errmsg("unexpected EOF on standby connection")));
@@ -1453,6 +1522,7 @@ ProcessRepliesIfAny(void)
 				 * 'X' means that the standby is closing down the socket.
 				 */
 			case 'X':
+				PrepareCleanExit();
 				proc_exit(0);
 
 			default:
@@ -1584,6 +1654,83 @@ ProcessStandbyReplyMessage(void)
 	 */
 	{
 		WalSnd *walsnd = MyWalSnd;
+		WalSndCausalReadsState causal_reads_state = walsnd->causal_reads_state;
+		bool causal_reads_state_changed = false;
+		bool causal_reads_set_joining_until = false;
+
+		/*
+		 * Handle causal reads state transitions, if a causal_reads_timeout is
+		 * configured, this standby is listed in causal_reads_standby_names,
+		 * and we are a primary database (not a cascading standby).
+		 */
+		if (am_potential_causal_reads_standby &&
+			!am_cascading_walsender &&
+			applyLagUs >= 0)
+		{
+			if (applyLagUs / 1000 < causal_reads_timeout)
+			{
+				if (causal_reads_state == WALSNDCRSTATE_UNAVAILABLE)
+				{
+					/*
+					 * The standby is applying fast enough.  We can't grant a
+					 * lease yet though, we need to wait for everything that
+					 * was committed while this standby was unavailable to be
+					 * applied first.  We move to joining state while we wait
+					 * for the standby to catch up.
+					 */
+					causal_reads_state = WALSNDCRSTATE_JOINING;
+					causal_reads_set_joining_until = true;
+					causal_reads_state_changed = true;
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING &&
+						 applyPtr >= causal_reads_joining_until)
+				{
+					/*
+					 * The standby has applied everything committed before we
+					 * reached joining state, and has been waiting for remote
+					 * apply on this standby while it's been in joining state,
+					 * so it is safe to move to available state and send a
+					 * lease.
+					 */
+					causal_reads_state = WALSNDCRSTATE_AVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+			else
+			{
+				if (causal_reads_state == WALSNDCRSTATE_AVAILABLE)
+				{
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+					/*
+					 * We are dropping a causal reads available standby, so we
+					 * mustn't let any commit command that is waiting in
+					 * CausalReadsWaitForLSN return until we are sure that the
+					 * standby definitely knows that it's not available and
+					 * starts raising errors for causal_reads transactions.
+					 * TODO: We could just wait until the standby acks that
+					 * its lease has been cancelled, and start numbering
+					 * keepalives and sending the number back in replies, so
+					 * we know it's acking the right message; then lagging
+					 * standbys would be less disruptive, but for now we just
+					 * wait for the lease to expire, as we do when we lose
+					 * contact with a standby, for the sake of simplicity.
+					 */
+					CausalReadsBeginStall(causal_reads_last_lease);
+				}
+				else if (causal_reads_state == WALSNDCRSTATE_JOINING)
+				{
+					/*
+					 * Dropping a joining standby doesn't require a stall,
+					 * because the standby doesn't think it's available, so
+					 * it's already raising the error for causal_reads
+					 * transactions.
+					 */
+					causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
+					causal_reads_state_changed = true;
+				}
+			}
+		}
 
 		SpinLockAcquire(&walsnd->mutex);
 		walsnd->write = writePtr;
@@ -1591,11 +1738,33 @@ ProcessStandbyReplyMessage(void)
 		walsnd->apply = applyPtr;
 		if (applyLagUs >= 0)
 			walsnd->applyLagUs = applyLagUs;
+		walsnd->causal_reads_state = causal_reads_state;
 		SpinLockRelease(&walsnd->mutex);
+
+		if (causal_reads_set_joining_until)
+		{
+			/*
+			 * Record the end of the primary's WAL at some arbitrary point
+			 * observed _after_ we moved to joining state (so that causal
+			 * reads commits start waiting, closing a race).  The standby
+			 * won't become available until it has replayed up to here.
+			 */
+			causal_reads_joining_until = GetFlushRecPtr();
+		}
+
+		if (causal_reads_state_changed)
+		{
+			WalSndKeepalive(true);
+			elog(LOG, "standby \"%s\" is %s", application_name,
+				 causal_reads_state == WALSNDCRSTATE_UNAVAILABLE ? "unavailable for causal reads" :
+				 causal_reads_state == WALSNDCRSTATE_JOINING ? "joining as a causal reads standby..." :
+				 causal_reads_state == WALSNDCRSTATE_AVAILABLE ? "available for causal reads" :
+				 "UNKNOWN");
+		}
 	}
 
 	if (!am_cascading_walsender)
-		SyncRepReleaseWaiters();
+		SyncRepReleaseWaiters(MyWalSnd->causal_reads_state >= WALSNDCRSTATE_JOINING);
 
 	/*
 	 * Advance our local xmin horizon when the client confirmed a flush.
@@ -1736,33 +1905,53 @@ ProcessStandbyHSFeedbackMessage(void)
  * If wal_sender_timeout is enabled we want to wake up in time to send
  * keepalives and to abort the connection if wal_sender_timeout has been
  * reached.
+ *
+ * But if causal_reads_timeout is enabled, we override that and send
+ * keepalives at a constant rate to replace expiring leases.
  */
 static long
 WalSndComputeSleeptime(TimestampTz now)
 {
 	long		sleeptime = 10000;		/* 10 s */
 
-	if (wal_sender_timeout > 0 && last_reply_timestamp > 0)
+	if ((wal_sender_timeout > 0 && last_reply_timestamp > 0) ||
+		am_potential_causal_reads_standby)
 	{
 		TimestampTz wakeup_time;
 		long		sec_to_timeout;
 		int			microsec_to_timeout;
 
-		/*
-		 * At the latest stop sleeping once wal_sender_timeout has been
-		 * reached.
-		 */
-		wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-												  wal_sender_timeout);
-
-		/*
-		 * If no ping has been sent yet, wakeup when it's time to do so.
-		 * WalSndKeepaliveIfNecessary() wants to send a keepalive once half of
-		 * the timeout passed without a response.
-		 */
-		if (!waiting_for_ping_response)
+		if (am_potential_causal_reads_standby)
+		{
+			/*
+			 * Leases last for a period of between 50% and 100% of
+			 * causal_reads_timeout, depending on clock skew, assuming clock
+			 * skew is under the 25% of causal_reads_timeout.  We send new
+			 * leases every half a lease, so that there are no gaps between
+			 * leases.
+			 */
+			wakeup_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+													  causal_reads_timeout /
+													  CAUSAL_READS_KEEPALIVE_RATIO);
+		}
+		else
+		{
+			/*
+			 * At the latest stop sleeping once wal_sender_timeout has been
+			 * reached.
+			 */
 			wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-													  wal_sender_timeout / 2);
+													  wal_sender_timeout);
+
+			/*
+			 * If no ping has been sent yet, wakeup when it's time to do so.
+			 * WalSndKeepaliveIfNecessary() wants to send a keepalive once
+			 * half of the timeout passed without a response.
+			 */
+			if (!waiting_for_ping_response)
+				wakeup_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+														  wal_sender_timeout / 2);
+		}
 
 		/* Compute relative time until wakeup. */
 		TimestampDifference(now, wakeup_time,
@@ -1778,20 +1967,33 @@ WalSndComputeSleeptime(TimestampTz now)
 /*
  * Check whether there have been responses by the client within
  * wal_sender_timeout and shutdown if not.
+ *
+ * If causal_reads_timeout is configured we override that, so that
+ * unresponsive standbys are detected sooner.
  */
 static void
 WalSndCheckTimeOut(TimestampTz now)
 {
 	TimestampTz timeout;
+	int allowed_time;
 
 	/* don't bail out if we're doing something that doesn't require timeouts */
 	if (last_reply_timestamp <= 0)
 		return;
 
-	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
-										  wal_sender_timeout);
+	/*
+	 * If a causal_reads_timeout is configured, it is used instead of
+	 * wal_sender_timeout, to limit the time before an unresponsive causal
+	 * reads standby is dropped.
+	 */
+	if (am_potential_causal_reads_standby)
+		allowed_time = causal_reads_timeout;
+	else
+		allowed_time = wal_sender_timeout;
 
-	if (wal_sender_timeout > 0 && now >= timeout)
+	timeout = TimestampTzPlusMilliseconds(last_reply_timestamp,
+										  allowed_time);
+	if (allowed_time > 0 && now >= timeout)
 	{
 		/*
 		 * Since typically expiration of replication timeout means
@@ -1824,6 +2026,9 @@ WalSndLoop(WalSndSendDataCallback send_data)
 	last_reply_timestamp = GetCurrentTimestamp();
 	waiting_for_ping_response = false;
 
+	/* Check if we are managing potential causal_reads standby. */
+	am_potential_causal_reads_standby = CausalReadsPotentialStandby();
+
 	/*
 	 * Loop until we reach the end of this timeline or the client requests to
 	 * stop streaming.
@@ -1984,6 +2189,7 @@ InitWalSenderSlot(void)
 			walsnd->flush = InvalidXLogRecPtr;
 			walsnd->apply = InvalidXLogRecPtr;
 			walsnd->state = WALSNDSTATE_STARTUP;
+			walsnd->causal_reads_state = WALSNDCRSTATE_UNAVAILABLE;
 			walsnd->latch = &MyProc->procLatch;
 			SpinLockRelease(&walsnd->mutex);
 			/* don't need the lock anymore */
@@ -2753,6 +2959,24 @@ WalSndGetStateString(WalSndState state)
 	return "UNKNOWN";
 }
 
+/*
+ * Return a string constant representing the causal reads state. This is used
+ * in system views, and should *not* be translated.
+ */
+static const char *
+WalSndGetCausalReadsStateString(WalSndCausalReadsState causal_reads_state)
+{
+	switch (causal_reads_state)
+	{
+		case WALSNDCRSTATE_UNAVAILABLE:
+			return "unavailable";
+		case WALSNDCRSTATE_JOINING:
+			return "joining";
+		case WALSNDCRSTATE_AVAILABLE:
+			return "available";
+	}
+	return "UNKNOWN";
+}
 
 /*
  * Returns activity of walsenders, including pids and xlog locations sent to
@@ -2761,7 +2985,7 @@ WalSndGetStateString(WalSndState state)
 Datum
 pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_SENDERS_COLS	9
+#define PG_STAT_GET_WAL_SENDERS_COLS	10
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	TupleDesc	tupdesc;
 	Tuplestorestate *tupstore;
@@ -2812,6 +3036,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		int64		applyLagUs;
 		int			priority;
 		WalSndState state;
+		WalSndCausalReadsState causalReadsState;
 		Datum		values[PG_STAT_GET_WAL_SENDERS_COLS];
 		bool		nulls[PG_STAT_GET_WAL_SENDERS_COLS];
 
@@ -2821,6 +3046,7 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 		SpinLockAcquire(&walsnd->mutex);
 		sentPtr = walsnd->sentPtr;
 		state = walsnd->state;
+		causalReadsState = walsnd->causal_reads_state;
 		write = walsnd->write;
 		flush = walsnd->flush;
 		apply = walsnd->apply;
@@ -2895,6 +3121,9 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 				values[8] = CStringGetTextDatum("sync");
 			else
 				values[8] = CStringGetTextDatum("potential");
+
+			values[9] =
+				CStringGetTextDatum(WalSndGetCausalReadsStateString(causalReadsState));
 		}
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
@@ -2914,14 +3143,52 @@ pg_stat_get_wal_senders(PG_FUNCTION_ARGS)
 static void
 WalSndKeepalive(bool requestReply)
 {
+	TimestampTz now;
+	TimestampTz causal_reads_lease;
+
 	elog(DEBUG2, "sending replication keepalive");
 
+	/*
+	 * If the walsender currently deems the standby to be available for causal
+	 * reads, then it grants a causal reads lease.  The lease authorizes the
+	 * standby to consider itself available for causal reads until a short
+	 * time in the future.  The primary promises to uphold the causal reads
+	 * guarantee until that time, by stalling commits until the the lease has
+	 * expired if necessary.
+	 */
+	now = GetCurrentTimestamp();
+	if (MyWalSnd->causal_reads_state < WALSNDCRSTATE_AVAILABLE)
+		causal_reads_lease = 0; /* Not available, no lease granted. */
+	else
+	{
+		/*
+		 * Since this timestamp is being sent to the standby where it will be
+		 * compared against a time generated by the standby's system clock, we
+		 * must consider clock skew.  First, we decide on a maximum tolerable
+		 * difference between system clocks.  If the primary's clock is ahead
+		 * of the standby's by more than this, then all bets are off (the
+		 * standby could falsely believe it has a valid lease).  If the
+		 * primary's clock is behind the standby's by more than this, then the
+		 * standby will err the other way and generate spurious errors in
+		 * causal_reads mode.  Rather than having a separate GUC for this, we
+		 * derive it from causal_reads_timeout.
+		 */
+		int max_clock_skew = causal_reads_timeout / CAUSAL_READS_CLOCK_SKEW_RATIO;
+
+		/* Compute and remember the expiry time of the lease we're granting. */
+		causal_reads_last_lease = TimestampTzPlusMilliseconds(now, causal_reads_timeout);
+		/* The version we'll send to the standby is adjusted to tolerate clock skew. */
+		causal_reads_lease =
+			TimestampTzPlusMilliseconds(causal_reads_last_lease, -max_clock_skew);
+	}
+
 	/* construct the message... */
 	resetStringInfo(&output_message);
 	pq_sendbyte(&output_message, 'k');
 	pq_sendint64(&output_message, sentPtr);
-	pq_sendint64(&output_message, GetCurrentIntegerTimestamp());
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(now));
 	pq_sendbyte(&output_message, requestReply ? 1 : 0);
+	pq_sendint64(&output_message, TimestampTzToIntegerTimestamp(causal_reads_lease));
 
 	/* ... and send it wrapped in CopyData */
 	pq_putmessage_noblock('d', output_message.data, output_message.len);
@@ -2939,23 +3206,35 @@ WalSndKeepaliveIfNecessary(TimestampTz now)
 	 * Don't send keepalive messages if timeouts are globally disabled or
 	 * we're doing something not partaking in timeouts.
 	 */
-	if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
-		return;
-
-	if (waiting_for_ping_response)
-		return;
+	if (!am_potential_causal_reads_standby)
+	{
+		if (wal_sender_timeout <= 0 || last_reply_timestamp <= 0)
+			return;
+		if (waiting_for_ping_response)
+			return;
+	}
 
 	/*
 	 * If half of wal_sender_timeout has lapsed without receiving any reply
 	 * from the standby, send a keep-alive message to the standby requesting
 	 * an immediate reply.
+	 *
+	 * If causal_reads_timeout has been configured, use it to control
+	 * keepalive intervals rather than wal_sender_timeout, so that we can keep
+	 * replacing leases at the right frequency.
 	 */
-	ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
-											wal_sender_timeout / 2);
+	if (am_potential_causal_reads_standby)
+		ping_time = TimestampTzPlusMilliseconds(last_keepalive_timestamp,
+												causal_reads_timeout /
+												CAUSAL_READS_KEEPALIVE_RATIO);
+	else
+		ping_time = TimestampTzPlusMilliseconds(last_reply_timestamp,
+												wal_sender_timeout / 2);
 	if (now >= ping_time)
 	{
 		WalSndKeepalive(true);
 		waiting_for_ping_response = true;
+		last_keepalive_timestamp = now;
 
 		/* Try to flush pending output to the client */
 		if (pq_flush_if_writable() != 0)
diff --git a/src/backend/utils/errcodes.txt b/src/backend/utils/errcodes.txt
index 49494f9..d81c089 100644
--- a/src/backend/utils/errcodes.txt
+++ b/src/backend/utils/errcodes.txt
@@ -306,6 +306,7 @@ Section: Class 40 - Transaction Rollback
 40001    E    ERRCODE_T_R_SERIALIZATION_FAILURE                              serialization_failure
 40003    E    ERRCODE_T_R_STATEMENT_COMPLETION_UNKNOWN                       statement_completion_unknown
 40P01    E    ERRCODE_T_R_DEADLOCK_DETECTED                                  deadlock_detected
+40P02    E    ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE                         causal_reads_not_available
 
 Section: Class 42 - Syntax Error or Access Rule Violation
 
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 06cb166..ac422e7 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -1634,6 +1634,16 @@ static struct config_bool ConfigureNamesBool[] =
 	},
 
 	{
+		{"causal_reads", PGC_USERSET, REPLICATION_STANDBY,
+		 gettext_noop("Enables causal reads."),
+		 NULL
+		},
+		&causal_reads,
+		false,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"syslog_sequence_numbers", PGC_SIGHUP, LOGGING_WHERE,
 			gettext_noop("Add sequence number to syslog messages to avoid duplicate suppression."),
 			NULL
@@ -1811,6 +1821,17 @@ static struct config_int ConfigureNamesInt[] =
 	},
 
 	{
+		{"causal_reads_timeout", PGC_SIGHUP, REPLICATION_STANDBY,
+			gettext_noop("Sets the maximum apply lag before causal reads standbys are no longer available."),
+			NULL,
+			GUC_UNIT_MS
+		},
+		&causal_reads_timeout,
+		0, 0, INT_MAX,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"max_connections", PGC_POSTMASTER, CONN_AUTH_SETTINGS,
 			gettext_noop("Sets the maximum number of concurrent connections."),
 			NULL
@@ -3454,7 +3475,18 @@ static struct config_string ConfigureNamesString[] =
 		},
 		&SyncRepStandbyNames,
 		"",
-		check_synchronous_standby_names, NULL, NULL
+		check_standby_names, NULL, NULL
+	},
+
+	{
+		{"causal_reads_standby_names", PGC_SIGHUP, REPLICATION_MASTER,
+			gettext_noop("List of names of potential causal reads standbys."),
+			NULL,
+			GUC_LIST_INPUT
+		},
+		&causal_reads_standby_names,
+		"*",
+		check_standby_names, NULL, NULL
 	},
 
 	{
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index ec4427f..fcc2c35 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -244,6 +244,15 @@
 				# from standby(s); '*' = all
 #vacuum_defer_cleanup_age = 0	# number of xacts by which cleanup is delayed
 
+#causal_reads_timeout = 0s      # maximum replication delay to tolerate from
+                                # standbys before dropping them from the set of
+				# available causal reads peers; 0 to disable
+				# causal reads
+
+#causal_reads_standy_names = '*'
+                                # standby servers that can potentially become
+				# available for causal reads; '*' = all
+
 # - Standby Servers -
 
 # These settings are ignored on a master server.
@@ -266,6 +275,14 @@
 #wal_retrieve_retry_interval = 5s	# time to wait before retrying to
 					# retrieve WAL after a failed attempt
 
+# - All Servers -
+
+#causal_reads = off                     # "on" in any pair of consecutive
+                                        # transactions guarantees that the second
+					# can see the first (even if the second
+					# is run on a standby), or will raise an
+					# error to report that the standby is
+					# unavailable for causal reads
 
 #------------------------------------------------------------------------------
 # QUERY TUNING
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index b88e012..6336240 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -46,8 +46,11 @@
 
 #include "access/transam.h"
 #include "access/xact.h"
+#include "access/xlog.h"
 #include "lib/pairingheap.h"
 #include "miscadmin.h"
+#include "replication/syncrep.h"
+#include "replication/walreceiver.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
 #include "storage/procarray.h"
@@ -209,6 +212,16 @@ GetTransactionSnapshot(void)
 				 "cannot take query snapshot during a parallel operation");
 
 		/*
+		 * In causal_reads mode on a standby, check if we have definitely
+		 * applied WAL for any COMMIT that returned successfully on the
+		 * primary.
+		 */
+		if (causal_reads && RecoveryInProgress() && !WalRcvCausalReadsAvailable())
+			ereport(ERROR,
+					(errcode(ERRCODE_T_R_CAUSAL_READS_NOT_AVAILABLE),
+					 errmsg("standby is not available for causal reads")));
+
+		/*
 		 * In transaction-snapshot mode, the first snapshot must live until
 		 * end of xact regardless of what the caller does with it, so we must
 		 * make a copy of it rather than returning CurrentSnapshotData
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 425b93b..3892dd3 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2712,7 +2712,7 @@ DATA(insert OID = 2022 (  pg_stat_get_activity			PGNSP PGUID 12 1 100 0 0 f f f
 DESCR("statistics: information about currently active backends");
 DATA(insert OID = 3318 (  pg_stat_get_progress_info           PGNSP PGUID 12 1 100 0 0 f f f f t t s r 1 0 2249 "25" "{25,23,26,26,20,20,20,20,20,20,20,20,20,20}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{cmdtype,pid,datid,relid,param1,param2,param3,param4,param5,param6,param7,param8,param9,param10}" _null_ _null_ pg_stat_get_progress_info _null_ _null_ _null_ ));
 DESCR("statistics: information about progress of backends running maintenance command");
-DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25}" "{o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
+DATA(insert OID = 3099 (  pg_stat_get_wal_senders	PGNSP PGUID 12 1 10 0 0 f f f f f t s r 0 0 2249 "" "{23,25,3220,3220,3220,3220,1186,23,25,25}" "{o,o,o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,replay_lag,sync_priority,sync_state,causal_reads_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
 DESCR("statistics: information about currently active replication");
 DATA(insert OID = 3317 (  pg_stat_get_wal_receiver	PGNSP PGUID 12 1 0 0 0 f f f f f f s r 0 0 2249 "" "{23,25,3220,23,3220,23,1184,1184,3220,1184,25}" "{o,o,o,o,o,o,o,o,o,o,o}" "{pid,status,receive_start_lsn,receive_start_tli,received_lsn,received_tli,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,slot_name}" _null_ _null_ pg_stat_get_wal_receiver _null_ _null_ _null_ ));
 DESCR("statistics: information about WAL receiver");
diff --git a/src/include/replication/syncrep.h b/src/include/replication/syncrep.h
index c005a42..dbfd601 100644
--- a/src/include/replication/syncrep.h
+++ b/src/include/replication/syncrep.h
@@ -24,14 +24,33 @@
 #define SYNC_REP_WAIT_WRITE		0
 #define SYNC_REP_WAIT_FLUSH		1
 #define SYNC_REP_WAIT_APPLY		2
+#define SYNC_REP_WAIT_CAUSAL_READS 3
 
-#define NUM_SYNC_REP_WAIT_MODE	3
+#define NUM_SYNC_REP_WAIT_MODE	4
 
 /* syncRepState */
 #define SYNC_REP_NOT_WAITING		0
 #define SYNC_REP_WAITING			1
 #define SYNC_REP_WAIT_COMPLETE		2
 
+/*
+ * ratio of causal_read_timeout to max_clock_skew (4 means than the maximum
+ * tolerated clock difference between primary and standbys using causal_reads
+ * is 1/4 of causal_reads_timeout)
+ */
+#define CAUSAL_READS_CLOCK_SKEW_RATIO 4
+
+/*
+ * ratio of causal_reads_timeout to keepalive time (2 means that the effective
+ * keepalive time is 1/2 of the causal_reads_timeout GUC when it is non-zero)
+ */
+#define CAUSAL_READS_KEEPALIVE_RATIO 2
+
+/* GUC variables */
+extern int causal_reads_timeout;
+extern bool causal_reads;
+extern char *causal_reads_standby_names;
+
 /* user-settable parameters for synchronous replication */
 extern char *SyncRepStandbyNames;
 
@@ -43,16 +62,23 @@ extern void SyncRepCleanupAtProcExit(void);
 
 /* called by wal sender */
 extern void SyncRepInitConfig(void);
-extern void SyncRepReleaseWaiters(void);
+extern void SyncRepReleaseWaiters(bool walsender_cr_available_or_joining);
 
 /* called by checkpointer */
 extern void SyncRepUpdateSyncStandbysDefined(void);
 
+/* called by user backend (xact.c) */
+extern void CausalReadsWaitForLSN(XLogRecPtr XactCommitLSN);
+
+/* called by wal sender */
+extern void CausalReadsBeginStall(TimestampTz lease_expiry_time);
+extern bool CausalReadsPotentialStandby(void);
+
 /* forward declaration to avoid pulling in walsender_private.h */
 struct WalSnd;
 extern struct WalSnd *SyncRepGetSynchronousStandby(void);
 
-extern bool check_synchronous_standby_names(char **newval, void **extra, GucSource source);
+extern bool check_standby_names(char **newval, void **extra, GucSource source);
 extern void assign_synchronous_commit(int newval, void *extra);
 
 #endif   /* _SYNCREP_H */
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 36bcb47..e721c6e 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -80,6 +80,13 @@ typedef struct
 	TimeLineID	receivedTLI;
 
 	/*
+	 * causalReadsLease is the time until which the primary has authorized
+	 * this standby to consider itself available for causal_reads mode, or 0
+	 * for not authorized.
+	 */
+	TimestampTz causalReadsLease;
+
+	/*
 	 * latestChunkStart is the starting byte position of the current "batch"
 	 * of received WAL.  It's actually the same as the previous value of
 	 * receivedUpto before the last flush to disk.  Startup process can use
@@ -171,4 +178,6 @@ extern int	GetReplicationApplyDelay(void);
 extern int	GetReplicationTransferLatency(void);
 extern void WalRcvForceReply(void);
 
+extern bool WalRcvCausalReadsAvailable(void);
+
 #endif   /* _WALRECEIVER_H */
diff --git a/src/include/replication/walsender_private.h b/src/include/replication/walsender_private.h
index 4de43e8..f6e0e9e 100644
--- a/src/include/replication/walsender_private.h
+++ b/src/include/replication/walsender_private.h
@@ -27,6 +27,13 @@ typedef enum WalSndState
 	WALSNDSTATE_STREAMING
 } WalSndState;
 
+typedef enum WalSndCausalReadsState
+{
+	WALSNDCRSTATE_UNAVAILABLE = 0,
+	WALSNDCRSTATE_JOINING,
+	WALSNDCRSTATE_AVAILABLE
+} WalSndCausalReadsState;
+
 /*
  * Each walsender has a WalSnd struct in shared memory.
  */
@@ -34,6 +41,7 @@ typedef struct WalSnd
 {
 	pid_t		pid;			/* this walsender's process id, or 0 */
 	WalSndState state;			/* this walsender's state */
+	WalSndCausalReadsState causal_reads_state; /* the walsender's causal reads state */
 	XLogRecPtr	sentPtr;		/* WAL has been sent up to this point */
 	bool		needreload;		/* does currently-open file need to be
 								 * reloaded? */
@@ -89,6 +97,12 @@ typedef struct
 	 */
 	bool		sync_standbys_defined;
 
+	/*
+	 * Until when must commits in causal_reads stall?  This is used to wait
+	 * for causal reads leases to expire.
+	 */
+	TimestampTz	stall_causal_reads_until;
+
 	WalSnd		walsnds[FLEXIBLE_ARRAY_MEMBER];
 } WalSndCtlData;
 
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index fc4b765..44f826f 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1785,10 +1785,11 @@ pg_stat_replication| SELECT s.pid,
     w.replay_location,
     w.replay_lag,
     w.sync_priority,
-    w.sync_state
+    w.sync_state,
+    w.causal_reads_state
    FROM pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, wait_event_type, wait_event, xact_start, query_start, backend_start, state_change, client_addr, client_hostname, client_port, backend_xid, backend_xmin, ssl, sslversion, sslcipher, sslbits, sslcompression, sslclientdn),
     pg_authid u,
-    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state)
+    pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, replay_lag, sync_priority, sync_state, causal_reads_state)
   WHERE ((s.usesysid = u.oid) AND (s.pid = w.pid));
 pg_stat_ssl| SELECT s.pid,
     s.ssl,
#72Robert Haas
robertmhaas@gmail.com
In reply to: Thomas Munro (#71)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Wed, Mar 30, 2016 at 2:22 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Wed, Mar 30, 2016 at 2:36 PM, Robert Haas <robertmhaas@gmail.com> wrote:

OK, I committed this, with a few tweaks. In particular, I added a
flag variable instead of relying on "latch set" == "need to send
reply"; the other changes were cosmetic.

I'm not sure how much more of this we can realistically get into 9.6;
the latter patches haven't had much review yet. But I'll set this
back to Needs Review in the CommitFest and we'll see where we end up.
But even if we don't get anything more than this, it's still rather
nice: remote_apply turns out to be only slightly slower than remote
flush, and it's a guarantee that a lot of people are looking for.

Thank you Michael and Robert!

Please find attached the rest of the patch series, rebased against
master. The goal of the 0002 patch is to provide an accurate
indication of the current replay lag on each standby, visible to users
like this:

postgres=# select application_name, replay_lag from pg_stat_replication;
application_name │ replay_lag
──────────────────┼─────────────────
replica1 │ 00:00:00.000299
replica2 │ 00:00:00.000323
replica3 │ 00:00:00.000319
replica4 │ 00:00:00.000303
(4 rows)

It works by maintaining a buffer of (end of WAL, time now) samples
received from the primary, and then eventually feeding those times
back to the primary when the recovery process replays the
corresponding locations.

Compared to approaches based on commit timestamps, this approach has
the advantage of providing non-misleading information between commits.
For example, if you run a batch load job that takes 1 minute to insert
the whole phonebook and no other transactions run, you will see
replay_lag updating regularly throughout that minute, whereas typical
commit timestamp-only approaches will show an increasing lag time
until a commit record is eventually applied. Compared to simple LSN
location comparisons, it reports in time rather than bytes of WAL,
which can be more meaningful for DBAs.

When the standby is entirely caught up and there is no write activity,
the reported time effectively represents the ping time between the
servers, and is updated every wal_sender_timeout / 2, when keepalive
messages are sent. While new WAL traffic is arriving, the walreceiver
records timestamps at most once per second in a circular buffer, and
then sends back replies containing the recorded timestamps as fast as
the recovery process can apply the corresponding xlog. The lag number
you see is computed by the primary server comparing two timestamps
generated by its own system clock, one of which has been on a journey
to the standby and back.

Accurate lag estimates are a prerequisite for the 0004 patch (about
which more later), but I believe users would find this valuable as a
feature on its own.

Well, one problem with this is that you can't put a loop inside of a
spinlock-protected critical section.

In general, I think this is a pretty reasonable way of attacking this
problem, but I'd say it's significantly under-commented. Where should
someone go to get a general overview of this mechanism? The answer is
not "at place XXX within the patch". (I think it might merit some
more extensive documentation, too, although I'm not exactly sure what
that should look like.)

When you overflow the buffer, you could thin in out in a smarter way,
like by throwing away every other entry instead of the oldest one. I
guess you'd need to be careful how you coded that, though, because
replaying an entry with a timestamp invalidates some of the saved
entries without formally throwing them out.

Conceivably, 0002 could be split into two patches, one of which
computes "stupid replay lag" considering only records that naturally
carry timestamps, and a second adding the circular buffer to handle
the case where much time passes without finding such a record.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#73Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Robert Haas (#72)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Tue, Apr 5, 2016 at 4:17 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Wed, Mar 30, 2016 at 2:22 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Wed, Mar 30, 2016 at 2:36 PM, Robert Haas <robertmhaas@gmail.com> wrote:

OK, I committed this, with a few tweaks. In particular, I added a
flag variable instead of relying on "latch set" == "need to send
reply"; the other changes were cosmetic.

I'm not sure how much more of this we can realistically get into 9.6;
the latter patches haven't had much review yet. But I'll set this
back to Needs Review in the CommitFest and we'll see where we end up.
But even if we don't get anything more than this, it's still rather
nice: remote_apply turns out to be only slightly slower than remote
flush, and it's a guarantee that a lot of people are looking for.

Thank you Michael and Robert!

Please find attached the rest of the patch series, rebased against
master. The goal of the 0002 patch is to provide an accurate
indication of the current replay lag on each standby, visible to users
like this:

postgres=# select application_name, replay_lag from pg_stat_replication;
application_name │ replay_lag
──────────────────┼─────────────────
replica1 │ 00:00:00.000299
replica2 │ 00:00:00.000323
replica3 │ 00:00:00.000319
replica4 │ 00:00:00.000303
(4 rows)

It works by maintaining a buffer of (end of WAL, time now) samples
received from the primary, and then eventually feeding those times
back to the primary when the recovery process replays the
corresponding locations.

Compared to approaches based on commit timestamps, this approach has
the advantage of providing non-misleading information between commits.
For example, if you run a batch load job that takes 1 minute to insert
the whole phonebook and no other transactions run, you will see
replay_lag updating regularly throughout that minute, whereas typical
commit timestamp-only approaches will show an increasing lag time
until a commit record is eventually applied. Compared to simple LSN
location comparisons, it reports in time rather than bytes of WAL,
which can be more meaningful for DBAs.

When the standby is entirely caught up and there is no write activity,
the reported time effectively represents the ping time between the
servers, and is updated every wal_sender_timeout / 2, when keepalive
messages are sent. While new WAL traffic is arriving, the walreceiver
records timestamps at most once per second in a circular buffer, and
then sends back replies containing the recorded timestamps as fast as
the recovery process can apply the corresponding xlog. The lag number
you see is computed by the primary server comparing two timestamps
generated by its own system clock, one of which has been on a journey
to the standby and back.

Accurate lag estimates are a prerequisite for the 0004 patch (about
which more later), but I believe users would find this valuable as a
feature on its own.

Well, one problem with this is that you can't put a loop inside of a
spinlock-protected critical section.

In general, I think this is a pretty reasonable way of attacking this
problem, but I'd say it's significantly under-commented. Where should
someone go to get a general overview of this mechanism? The answer is
not "at place XXX within the patch". (I think it might merit some
more extensive documentation, too, although I'm not exactly sure what
that should look like.)

When you overflow the buffer, you could thin in out in a smarter way,
like by throwing away every other entry instead of the oldest one. I
guess you'd need to be careful how you coded that, though, because
replaying an entry with a timestamp invalidates some of the saved
entries without formally throwing them out.

Conceivably, 0002 could be split into two patches, one of which
computes "stupid replay lag" considering only records that naturally
carry timestamps, and a second adding the circular buffer to handle
the case where much time passes without finding such a record.

Thanks. I see a way to move that loop and change the overflow
behaviour along those lines but due to other commitments I won't be
able to post a well tested patch and still leave time for reviewers
and committer before the looming deadline. After the freeze I will
post an updated version that addresses these problems for the next CF.

--
Thomas Munro
http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#74Robert Haas
robertmhaas@gmail.com
In reply to: Thomas Munro (#73)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Tue, Apr 5, 2016 at 7:21 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

Thanks. I see a way to move that loop and change the overflow
behaviour along those lines but due to other commitments I won't be
able to post a well tested patch and still leave time for reviewers
and committer before the looming deadline. After the freeze I will
post an updated version that addresses these problems for the next CF.

OK, I've marked this Returned with Feedback for now.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#75Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Michael Paquier (#67)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On Tue, Mar 29, 2016 at 8:17 PM, Michael Paquier
<michael.paquier@gmail.com> wrote:

On Tue, Mar 29, 2016 at 1:11 PM, Thomas Munro <thomas.munro@enterprisedb.com> wrote:

(BTW, isn't the select call in libpq_select
lacking an exceptfds set, and can't it therefore block forever when
there is an error condition on the socket and no timeout?)

Hm. I think you're right here when timeout is NULL... It would loop infinitely.
@Andres (in CC): your thoughts on that regarding the new
WaitEventSetWaitBlock()? The same pattern is used there.

That was a red herring. I was confused because SUSv2 and POSIX call
this argument 'errorfds' and say that sockets *also* tell you about
errors this way. (Many/most real OSs call the argument 'exceptfds'
instead and only use it to tell you about out-of-band data and
possibly implementation specific events for devices, pseudo-terminals
etc. If you want to know about errors on a socket it's enough to have
it in readfds/writefds, and insufficient to have it only in
errorfds/exceptfds unless you can find a computer that actually
conforms to POSIX.)

--
Thomas Munro
http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#76Andres Freund
andres@anarazel.de
In reply to: Thomas Munro (#75)
Re: Proposal: "Causal reads" mode for load balancing reads without stale data

On 2016-05-05 13:30:42 +1200, Thomas Munro wrote:

That was a red herring. I was confused because SUSv2 and POSIX call
this argument 'errorfds' and say that sockets *also* tell you about
errors this way. (Many/most real OSs call the argument 'exceptfds'
instead and only use it to tell you about out-of-band data and
possibly implementation specific events for devices, pseudo-terminals
etc. If you want to know about errors on a socket it's enough to have
it in readfds/writefds, and insufficient to have it only in
errorfds/exceptfds unless you can find a computer that actually
conforms to POSIX.)

Correct, exceptfds is pretty much meaningless for anything we do in
postgres. We rely on select returning a socket as read/writeable if the
socket has hung up. That's been the case *before* the recent
WaitEventSet refactoring, so I think we're fairly solid on relying on
that.

Greetings,

Andres Freund

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers