diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 5415604..d03ab07 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -45,8 +45,8 @@
  *		  fsynced
  *		* If COMMIT happens after checkpoint then backend reads state data from
  *		  files
- *		* In case of crash replay will move data from xlog to files, if that
- *		  hasn't happened before. XXX TODO - move to shmem in replay also
+ *
+ *		The same procedure happens during WAL replay.
  *
  *-------------------------------------------------------------------------
  */
@@ -578,6 +578,46 @@ LockGXact(const char *gid, Oid user)
 }
 
 /*
+ * LockGXactByXid
+ *
+ * Find prepared transaction by xid and lock corresponding GXACT.
+ * This is used during recovery as an alternative to LockGXact(), and
+ * should only be used in recovery. No entries found is sane situation,
+ * see comments inside XlogRedoFinishPrepared().
+ *
+ * Returns the transaction data if found, or NULL if nothing has been locked.
+ */
+static GlobalTransaction
+LockGXactByXid(TransactionId xid)
+{
+	int		i;
+	GlobalTransaction gxact = NULL;
+
+	Assert(RecoveryInProgress());
+
+	LWLockAcquire(TwoPhaseStateLock, LW_EXCLUSIVE);
+	for (i = 0; i < TwoPhaseState->numPrepXacts; i++)
+	{
+		PGXACT	   *pgxact;
+
+		gxact = TwoPhaseState->prepXacts[i];
+		pgxact = &ProcGlobal->allPgXact[gxact->pgprocno];
+
+		if (TransactionIdEquals(xid, pgxact->xid))
+		{
+			/* ok to lock it */
+			gxact->locking_backend = MyBackendId;
+			MyLockedGxact = gxact;
+			LWLockRelease(TwoPhaseStateLock);
+			return gxact;
+		}
+	}
+	LWLockRelease(TwoPhaseStateLock);
+
+	return NULL;
+}
+
+/*
  * RemoveGXact
  *		Remove the prepared transaction from the shared memory array.
  *
@@ -1241,9 +1281,9 @@ ReadTwoPhaseFile(TransactionId xid, bool give_warnings)
  * Reads 2PC data from xlog. During checkpoint this data will be moved to
  * twophase files and ReadTwoPhaseFile should be used instead.
  *
- * Note clearly that this function accesses WAL during normal operation, similarly
- * to the way WALSender or Logical Decoding would do. It does not run during
- * crash recovery or standby processing.
+ * Note that this function accesses WAL not only during recovery but also
+ * during normal operation, similarly to the way WALSender or Logical
+ * Decoding would do.
  */
 static void
 XlogReadTwoPhaseData(XLogRecPtr lsn, char **buf, int *len)
@@ -1252,8 +1292,6 @@ XlogReadTwoPhaseData(XLogRecPtr lsn, char **buf, int *len)
 	XLogReaderState *xlogreader;
 	char	   *errormsg;
 
-	Assert(!RecoveryInProgress());
-
 	xlogreader = XLogReaderAllocate(&read_local_xlog_page, NULL);
 	if (!xlogreader)
 		ereport(ERROR,
@@ -1288,40 +1326,54 @@ XlogReadTwoPhaseData(XLogRecPtr lsn, char **buf, int *len)
 
 
 /*
- * Confirms an xid is prepared, during recovery
+ * Confirms an xid is prepared, during recovery.
+ *
+ * Here we know that all info about old prepared transactions is already
+ * loaded by RecoverPreparedFromFiles() so we can check only PGPROC.
  */
 bool
 StandbyTransactionIdIsPrepared(TransactionId xid)
 {
-	char	   *buf;
-	TwoPhaseFileHeader *hdr;
-	bool		result;
+	int			i;
 
 	Assert(TransactionIdIsValid(xid));
 
 	if (max_prepared_xacts <= 0)
 		return false;			/* nothing to do */
 
-	/* Read and validate file */
-	buf = ReadTwoPhaseFile(xid, false);
-	if (buf == NULL)
-		return false;
+	/*
+	 * Check if this prepared transaction has its information in
+	 * shared memory, and use it.
+	 */
+	LWLockAcquire(TwoPhaseStateLock, LW_SHARED);
+	for (i = 0; i < TwoPhaseState->numPrepXacts; i++)
+	{
+		GlobalTransaction gxact = TwoPhaseState->prepXacts[i];
+		PGXACT *pgxact = &ProcGlobal->allPgXact[gxact->pgprocno];
 
-	/* Check header also */
-	hdr = (TwoPhaseFileHeader *) buf;
-	result = TransactionIdEquals(hdr->xid, xid);
-	pfree(buf);
+		if (TransactionIdEquals(pgxact->xid, xid))
+		{
+			LWLockRelease(TwoPhaseStateLock);
+			return true;
+		}
+	}
+	LWLockRelease(TwoPhaseStateLock);
 
-	return result;
+	return false;
 }
 
 /*
- * FinishPreparedTransaction: execute COMMIT PREPARED or ROLLBACK PREPARED
+ * FinishGXact
+ *
+ * Do the actual finish of COMMIT/ABORT PREPARED. Calls is responsible
+ * for locking the transaction this routine is working on.
+ *
+ * This function can be called during replay to clean memory state for
+ * previously prepared xact. In that case actions are the same as in
+ * normal operations but without any writes to WAL or files.
  */
-void
-FinishPreparedTransaction(const char *gid, bool isCommit)
+static void FinishGXact(GlobalTransaction gxact, bool isCommit)
 {
-	GlobalTransaction gxact;
 	PGPROC	   *proc;
 	PGXACT	   *pgxact;
 	TransactionId xid;
@@ -1332,16 +1384,9 @@ FinishPreparedTransaction(const char *gid, bool isCommit)
 	TransactionId *children;
 	RelFileNode *commitrels;
 	RelFileNode *abortrels;
-	RelFileNode *delrels;
-	int			ndelrels;
 	SharedInvalidationMessage *invalmsgs;
 	int			i;
 
-	/*
-	 * Validate the GID, and lock the GXACT to ensure that two backends do not
-	 * try to commit the same GID at once.
-	 */
-	gxact = LockGXact(gid, GetUserId());
 	proc = &ProcGlobal->allProcs[gxact->pgprocno];
 	pgxact = &ProcGlobal->allPgXact[gxact->pgprocno];
 	xid = pgxact->xid;
@@ -1383,17 +1428,23 @@ FinishPreparedTransaction(const char *gid, bool isCommit)
 	 * TransactionIdIsInProgress will stop saying the prepared xact is in
 	 * progress), then run the post-commit or post-abort callbacks. The
 	 * callbacks will release the locks the transaction held.
+	 *
+	 * In recovery nothing needs to happen here as this generates WAL
+	 * records.
 	 */
-	if (isCommit)
-		RecordTransactionCommitPrepared(xid,
+	if (!RecoveryInProgress())
+	{
+		if (isCommit)
+			RecordTransactionCommitPrepared(xid,
 										hdr->nsubxacts, children,
 										hdr->ncommitrels, commitrels,
 										hdr->ninvalmsgs, invalmsgs,
 										hdr->initfileinval);
-	else
-		RecordTransactionAbortPrepared(xid,
+		else
+			RecordTransactionAbortPrepared(xid,
 									   hdr->nsubxacts, children,
 									   hdr->nabortrels, abortrels);
+	}
 
 	ProcArrayRemove(proc, latestXid);
 
@@ -1408,41 +1459,50 @@ FinishPreparedTransaction(const char *gid, bool isCommit)
 	gxact->valid = false;
 
 	/*
-	 * We have to remove any files that were supposed to be dropped. For
-	 * consistency with the regular xact.c code paths, must do this before
-	 * releasing locks, so do it before running the callbacks.
-	 *
-	 * NB: this code knows that we couldn't be dropping any temp rels ...
+	 * Perform actions needed only during normal operation, but *not* recovery.
 	 */
-	if (isCommit)
+	if (!RecoveryInProgress())
 	{
-		delrels = commitrels;
-		ndelrels = hdr->ncommitrels;
-	}
-	else
-	{
-		delrels = abortrels;
-		ndelrels = hdr->nabortrels;
-	}
-	for (i = 0; i < ndelrels; i++)
-	{
-		SMgrRelation srel = smgropen(delrels[i], InvalidBackendId);
+		RelFileNode *delrels;
+		int			ndelrels;
 
-		smgrdounlink(srel, false);
-		smgrclose(srel);
-	}
+		/*
+		 * We have to remove any files that were supposed to be dropped. For
+		 * consistency with the regular xact.c code paths, must do this before
+		 * releasing locks, so do it before running the callbacks.
+		 *
+		 * NB: this code knows that we couldn't be dropping any temp rels ...
+		 */
+		if (isCommit)
+		{
+			delrels = commitrels;
+			ndelrels = hdr->ncommitrels;
+		}
+		else
+		{
+			delrels = abortrels;
+			ndelrels = hdr->nabortrels;
+		}
+		for (i = 0; i < ndelrels; i++)
+		{
+			SMgrRelation srel = smgropen(delrels[i], InvalidBackendId);
 
-	/*
-	 * Handle cache invalidation messages.
-	 *
-	 * Relcache init file invalidation requires processing both before and
-	 * after we send the SI messages. See AtEOXact_Inval()
-	 */
-	if (hdr->initfileinval)
-		RelationCacheInitFilePreInvalidate();
-	SendSharedInvalidMessages(invalmsgs, hdr->ninvalmsgs);
-	if (hdr->initfileinval)
-		RelationCacheInitFilePostInvalidate();
+			smgrdounlink(srel, false);
+			smgrclose(srel);
+		}
+
+		/*
+		 * Handle cache invalidation messages.
+		 *
+		 * Relcache init file invalidation requires processing both before and
+		 * after we send the SI messages. See AtEOXact_Inval()
+		 */
+		if (hdr->initfileinval)
+			RelationCacheInitFilePreInvalidate();
+		SendSharedInvalidMessages(invalmsgs, hdr->ninvalmsgs);
+		if (hdr->initfileinval)
+			RelationCacheInitFilePostInvalidate();
+	}
 
 	/* And now do the callbacks */
 	if (isCommit)
@@ -1468,6 +1528,51 @@ FinishPreparedTransaction(const char *gid, bool isCommit)
 }
 
 /*
+ * FinishPreparedTransaction: execute COMMIT PREPARED or ROLLBACK PREPARED
+ */
+void
+FinishPreparedTransaction(const char *gid, bool isCommit)
+{
+	GlobalTransaction gxact;
+
+	/*
+	 * Validate the GID, and lock the GXACT to ensure that two backends do not
+	 * try to commit the same GID at once.
+	 */
+	gxact = LockGXact(gid, GetUserId());
+	FinishGXact(gxact, isCommit);
+}
+
+/*
+ * XlogRedoFinishPrepared()
+ *
+ * This function is called during recovery for WAL records working on COMMIT
+ * PREPARED or ABORT PREPARED. That function cleans up memory state that was
+ * created while replaying its corresponding PREPARE record if its information
+ * was not on disk in a twophase file.
+ */
+void
+XlogRedoFinishPrepared(TransactionId xid, bool isCommit)
+{
+	GlobalTransaction gxact;
+
+	Assert(RecoveryInProgress());
+
+	gxact = LockGXactByXid(xid);
+
+	/*
+	 * If requested xid was not found that means that PREPARE record was before
+	 * checkpoint that we are replaying from and COMMIT managed to remove
+	 * twophase file before crash. So we know that no memory state for prepared
+	 * tx was created and no files left and everything is fine.
+	 */
+	if (!gxact)
+		return;
+
+	FinishGXact(gxact, isCommit);
+}
+
+/*
  * Scan 2PC state data in memory and call the indicated callbacks for each 2PC record.
  */
 static void
@@ -1654,43 +1759,21 @@ CheckPointTwoPhase(XLogRecPtr redo_horizon)
 }
 
 /*
- * PrescanPreparedTransactions
- *
- * Scan the pg_twophase directory and determine the range of valid XIDs
- * present.  This is run during database startup, after we have completed
- * reading WAL.  ShmemVariableCache->nextXid has been set to one more than
- * the highest XID for which evidence exists in WAL.
- *
  * We throw away any prepared xacts with main XID beyond nextXid --- if any
  * are present, it suggests that the DBA has done a PITR recovery to an
  * earlier point in time without cleaning out pg_twophase.  We dare not
  * try to recover such prepared xacts since they likely depend on database
  * state that doesn't exist now.
- *
- * However, we will advance nextXid beyond any subxact XIDs belonging to
- * valid prepared xacts.  We need to do this since subxact commit doesn't
- * write a WAL entry, and so there might be no evidence in WAL of those
- * subxact XIDs.
- *
- * Our other responsibility is to determine and return the oldest valid XID
- * among the prepared xacts (if none, return ShmemVariableCache->nextXid).
- * This is needed to synchronize pg_subtrans startup properly.
- *
- * If xids_p and nxids_p are not NULL, pointer to a palloc'd array of all
- * top-level xids is stored in *xids_p. The number of entries in the array
- * is returned in *nxids_p.
  */
-TransactionId
-PrescanPreparedTransactions(TransactionId **xids_p, int *nxids_p)
+void
+CleanupPreparedTransactionsOnPITR(void)
 {
-	TransactionId origNextXid = ShmemVariableCache->nextXid;
-	TransactionId result = origNextXid;
 	DIR		   *cldir;
 	struct dirent *clde;
-	TransactionId *xids = NULL;
-	int			nxids = 0;
-	int			allocsize = 0;
 
+	/*
+	 * Scan files in pg_twophase directory
+	 */
 	cldir = AllocateDir(TWOPHASE_DIR);
 	while ((clde = ReadDir(cldir, TWOPHASE_DIR)) != NULL)
 	{
@@ -1698,15 +1781,11 @@ PrescanPreparedTransactions(TransactionId **xids_p, int *nxids_p)
 			strspn(clde->d_name, "0123456789ABCDEF") == 8)
 		{
 			TransactionId xid;
-			char	   *buf;
-			TwoPhaseFileHeader *hdr;
-			TransactionId *subxids;
-			int			i;
 
 			xid = (TransactionId) strtoul(clde->d_name, NULL, 16);
 
 			/* Reject XID if too new */
-			if (TransactionIdFollowsOrEquals(xid, origNextXid))
+			if (TransactionIdFollowsOrEquals(xid, ShmemVariableCache->nextXid))
 			{
 				ereport(WARNING,
 						(errmsg("removing future two-phase state file \"%s\"",
@@ -1714,201 +1793,163 @@ PrescanPreparedTransactions(TransactionId **xids_p, int *nxids_p)
 				RemoveTwoPhaseFile(xid, true);
 				continue;
 			}
-
-			/*
-			 * Note: we can't check if already processed because clog
-			 * subsystem isn't up yet.
-			 */
-
-			/* Read and validate file */
-			buf = ReadTwoPhaseFile(xid, true);
-			if (buf == NULL)
-			{
-				ereport(WARNING,
-					  (errmsg("removing corrupt two-phase state file \"%s\"",
-							  clde->d_name)));
-				RemoveTwoPhaseFile(xid, true);
-				continue;
-			}
-
-			/* Deconstruct header */
-			hdr = (TwoPhaseFileHeader *) buf;
-			if (!TransactionIdEquals(hdr->xid, xid))
-			{
-				ereport(WARNING,
-					  (errmsg("removing corrupt two-phase state file \"%s\"",
-							  clde->d_name)));
-				RemoveTwoPhaseFile(xid, true);
-				pfree(buf);
-				continue;
-			}
-
-			/*
-			 * OK, we think this file is valid.  Incorporate xid into the
-			 * running-minimum result.
-			 */
-			if (TransactionIdPrecedes(xid, result))
-				result = xid;
-
-			/*
-			 * Examine subtransaction XIDs ... they should all follow main
-			 * XID, and they may force us to advance nextXid.
-			 *
-			 * We don't expect anyone else to modify nextXid, hence we don't
-			 * need to hold a lock while examining it.  We still acquire the
-			 * lock to modify it, though.
-			 */
-			subxids = (TransactionId *) (buf +
-								MAXALIGN(sizeof(TwoPhaseFileHeader)) +
-								MAXALIGN(hdr->gidlen));
-			for (i = 0; i < hdr->nsubxacts; i++)
-			{
-				TransactionId subxid = subxids[i];
-
-				Assert(TransactionIdFollows(subxid, xid));
-				if (TransactionIdFollowsOrEquals(subxid,
-												 ShmemVariableCache->nextXid))
-				{
-					LWLockAcquire(XidGenLock, LW_EXCLUSIVE);
-					ShmemVariableCache->nextXid = subxid;
-					TransactionIdAdvance(ShmemVariableCache->nextXid);
-					LWLockRelease(XidGenLock);
-				}
-			}
-
-
-			if (xids_p)
-			{
-				if (nxids == allocsize)
-				{
-					if (nxids == 0)
-					{
-						allocsize = 10;
-						xids = palloc(allocsize * sizeof(TransactionId));
-					}
-					else
-					{
-						allocsize = allocsize * 2;
-						xids = repalloc(xids, allocsize * sizeof(TransactionId));
-					}
-				}
-				xids[nxids++] = xid;
-			}
-
-			pfree(buf);
 		}
 	}
 	FreeDir(cldir);
+}
+
+
+/*
+ * Returns array of currently prepared two-phase transactions.
+ *
+ * If we faced shutdown checkpoint during replay we need that to create
+ * RunningTransactionsData structure and pass it to ProcArrayApplyRecoveryInfo.
+ */
+void
+GetPreparedTransactions(TransactionId **xids_p, int *nxids_p)
+{
+	TransactionId *xids = NULL;
+	int			nxids = 0;
+	int			allocsize = 0;
+	int			i;
+
+	Assert(RecoveryInProgress());
+
+	/*
+	 * We need to check the PGXACT array for prepared transactions that doesn't
+	 * have any state file in case of a slave restart with the master being off.
+	 */
+	LWLockAcquire(TwoPhaseStateLock, LW_SHARED);
+	for (i = 0; i < TwoPhaseState->numPrepXacts; i++)
+	{
+		GlobalTransaction gxact = TwoPhaseState->prepXacts[i];
+		PGXACT	   *pgxact = &ProcGlobal->allPgXact[gxact->pgprocno];
+
+		if (!gxact->valid)
+			continue;
+
+		if (xids_p)
+		{
+			if (nxids == allocsize)
+			{
+				if (nxids == 0)
+				{
+					allocsize = 10;
+					xids = palloc(allocsize * sizeof(TransactionId));
+				}
+				else
+				{
+					allocsize = allocsize * 2;
+					xids = repalloc(xids, allocsize * sizeof(TransactionId));
+				}
+			}
+			xids[nxids++] = pgxact->xid;
+		}
+	}
+	LWLockRelease(TwoPhaseStateLock);
 
 	if (xids_p)
 	{
 		*xids_p = xids;
 		*nxids_p = nxids;
 	}
-
-	return result;
 }
 
+
 /*
- * StandbyRecoverPreparedTransactions
+ * RecoverPreparedFromBuffer
  *
- * Scan the pg_twophase directory and setup all the required information to
- * allow standby queries to treat prepared transactions as still active.
- * This is never called at the end of recovery - we use
- * RecoverPreparedTransactions() at that point.
+ * Parse data in given buffer (that can be a pointer to WAL record holding
+ * this information or data read from a twophase file) and build the
+ * shared-memory state for that prepared transaction.
  *
- * Currently we simply call SubTransSetParent() for any subxids of prepared
- * transactions. If overwriteOK is true, it's OK if some XIDs have already
- * been marked in pg_subtrans.
+ * Caller is responsible for calling MarkAsPrepared() on the returned gxact.
  */
-void
-StandbyRecoverPreparedTransactions(bool overwriteOK)
+static GlobalTransaction
+RecoverPreparedFromBuffer(char *buf)
 {
-	DIR		   *cldir;
-	struct dirent *clde;
+	char			*bufptr;
+	const char		*gid;
+	TransactionId	*subxids;
+	bool			overwriteOK = false;
+	int				i;
+	GlobalTransaction gxact;
+	TwoPhaseFileHeader	*hdr;
 
-	cldir = AllocateDir(TWOPHASE_DIR);
-	while ((clde = ReadDir(cldir, TWOPHASE_DIR)) != NULL)
-	{
-		if (strlen(clde->d_name) == 8 &&
-			strspn(clde->d_name, "0123456789ABCDEF") == 8)
-		{
-			TransactionId xid;
-			char	   *buf;
-			TwoPhaseFileHeader *hdr;
-			TransactionId *subxids;
-			int			i;
+	/* Deconstruct header */
+	hdr = (TwoPhaseFileHeader *) buf;
+	bufptr = buf + MAXALIGN(sizeof(TwoPhaseFileHeader));
+	gid = (const char *) bufptr;
+	bufptr += MAXALIGN(hdr->gidlen);
+	subxids = (TransactionId *) bufptr;
+	bufptr += MAXALIGN(hdr->nsubxacts * sizeof(TransactionId));
+	bufptr += MAXALIGN(hdr->ncommitrels * sizeof(RelFileNode));
+	bufptr += MAXALIGN(hdr->nabortrels * sizeof(RelFileNode));
+	bufptr += MAXALIGN(hdr->ninvalmsgs * sizeof(SharedInvalidationMessage));
 
-			xid = (TransactionId) strtoul(clde->d_name, NULL, 16);
+	/*
+	 * It's possible that SubTransSetParent has been set before, if
+	 * the prepared transaction generated xid assignment records. Test
+	 * here must match one used in AssignTransactionId().
+	 */
+	if (InHotStandby && (hdr->nsubxacts >= PGPROC_MAX_CACHED_SUBXIDS ||
+						 XLogLogicalInfoActive()))
+		overwriteOK = true;
 
-			/* Already processed? */
-			if (TransactionIdDidCommit(xid) || TransactionIdDidAbort(xid))
-			{
-				ereport(WARNING,
-						(errmsg("removing stale two-phase state file \"%s\"",
-								clde->d_name)));
-				RemoveTwoPhaseFile(xid, true);
-				continue;
-			}
+	/*
+	 * Reconstruct subtrans state for the transaction --- needed
+	 * because pg_subtrans is not preserved over a restart.  Note that
+	 * we are linking all the subtransactions directly to the
+	 * top-level XID; there may originally have been a more complex
+	 * hierarchy, but there's no need to restore that exactly.
+	 */
+	for (i = 0; i < hdr->nsubxacts; i++)
+		SubTransSetParent(subxids[i], hdr->xid, overwriteOK);
 
-			/* Read and validate file */
-			buf = ReadTwoPhaseFile(xid, true);
-			if (buf == NULL)
-			{
-				ereport(WARNING,
-					  (errmsg("removing corrupt two-phase state file \"%s\"",
-							  clde->d_name)));
-				RemoveTwoPhaseFile(xid, true);
-				continue;
-			}
+	/*
+	 * Recreate its GXACT and dummy PGPROC
+	 */
+	gxact = MarkAsPreparing(hdr->xid, gid,
+							hdr->prepared_at,
+							hdr->owner, hdr->database);
+	GXactLoadSubxactData(gxact, hdr->nsubxacts, subxids);
 
-			/* Deconstruct header */
-			hdr = (TwoPhaseFileHeader *) buf;
-			if (!TransactionIdEquals(hdr->xid, xid))
-			{
-				ereport(WARNING,
-					  (errmsg("removing corrupt two-phase state file \"%s\"",
-							  clde->d_name)));
-				RemoveTwoPhaseFile(xid, true);
-				pfree(buf);
-				continue;
-			}
+	/*
+	 * Recover other state (notably locks) using resource managers
+	 */
+	ProcessRecords(bufptr, hdr->xid, twophase_recover_callbacks);
 
-			/*
-			 * Examine subtransaction XIDs ... they should all follow main
-			 * XID.
-			 */
-			subxids = (TransactionId *) (buf +
-								MAXALIGN(sizeof(TwoPhaseFileHeader)) +
-								MAXALIGN(hdr->gidlen));
-			for (i = 0; i < hdr->nsubxacts; i++)
-			{
-				TransactionId subxid = subxids[i];
+	/*
+	 * Release locks held by the standby process after we process each
+	 * prepared transaction. As a result, we don't need too many
+	 * additional locks at any one time.
+	 */
+	if (InHotStandby)
+		StandbyReleaseLockTree(hdr->xid, hdr->nsubxacts, subxids);
 
-				Assert(TransactionIdFollows(subxid, xid));
-				SubTransSetParent(xid, subxid, overwriteOK);
-			}
+	/*
+	 * We're done with recovering this transaction. Clear
+	 * MyLockedGxact, like we do in PrepareTransaction() during normal
+	 * operation.
+	 */
+	PostPrepare_Twophase();
 
-			pfree(buf);
-		}
-	}
-	FreeDir(cldir);
+	return gxact;
 }
 
 /*
- * RecoverPreparedTransactions
+ * RecoverPreparedFromFiles
  *
  * Scan the pg_twophase directory and reload shared-memory state for each
  * prepared transaction (reacquire locks, etc).  This is run during database
- * startup.
+ * startup before actual WAL replay started.
  */
 void
-RecoverPreparedTransactions(void)
+RecoverPreparedFromFiles(void)
 {
 	char		dir[MAXPGPATH];
 	DIR		   *cldir;
 	struct dirent *clde;
-	bool		overwriteOK = false;
 
 	snprintf(dir, MAXPGPATH, "%s", TWOPHASE_DIR);
 
@@ -1920,25 +1961,10 @@ RecoverPreparedTransactions(void)
 		{
 			TransactionId xid;
 			char	   *buf;
-			char	   *bufptr;
-			TwoPhaseFileHeader *hdr;
-			TransactionId *subxids;
 			GlobalTransaction gxact;
-			const char *gid;
-			int			i;
 
 			xid = (TransactionId) strtoul(clde->d_name, NULL, 16);
 
-			/* Already processed? */
-			if (TransactionIdDidCommit(xid) || TransactionIdDidAbort(xid))
-			{
-				ereport(WARNING,
-						(errmsg("removing stale two-phase state file \"%s\"",
-								clde->d_name)));
-				RemoveTwoPhaseFile(xid, true);
-				continue;
-			}
-
 			/* Read and validate file */
 			buf = ReadTwoPhaseFile(xid, true);
 			if (buf == NULL)
@@ -1953,73 +1979,39 @@ RecoverPreparedTransactions(void)
 			ereport(LOG,
 					(errmsg("recovering prepared transaction %u", xid)));
 
-			/* Deconstruct header */
-			hdr = (TwoPhaseFileHeader *) buf;
-			Assert(TransactionIdEquals(hdr->xid, xid));
-			bufptr = buf + MAXALIGN(sizeof(TwoPhaseFileHeader));
-			gid = (const char *) bufptr;
-			bufptr += MAXALIGN(hdr->gidlen);
-			subxids = (TransactionId *) bufptr;
-			bufptr += MAXALIGN(hdr->nsubxacts * sizeof(TransactionId));
-			bufptr += MAXALIGN(hdr->ncommitrels * sizeof(RelFileNode));
-			bufptr += MAXALIGN(hdr->nabortrels * sizeof(RelFileNode));
-			bufptr += MAXALIGN(hdr->ninvalmsgs * sizeof(SharedInvalidationMessage));
-
-			/*
-			 * It's possible that SubTransSetParent has been set before, if
-			 * the prepared transaction generated xid assignment records. Test
-			 * here must match one used in AssignTransactionId().
-			 */
-			if (InHotStandby && (hdr->nsubxacts >= PGPROC_MAX_CACHED_SUBXIDS ||
-								 XLogLogicalInfoActive()))
-				overwriteOK = true;
-
-			/*
-			 * Reconstruct subtrans state for the transaction --- needed
-			 * because pg_subtrans is not preserved over a restart.  Note that
-			 * we are linking all the subtransactions directly to the
-			 * top-level XID; there may originally have been a more complex
-			 * hierarchy, but there's no need to restore that exactly.
-			 */
-			for (i = 0; i < hdr->nsubxacts; i++)
-				SubTransSetParent(subxids[i], xid, overwriteOK);
-
-			/*
-			 * Recreate its GXACT and dummy PGPROC
-			 */
-			gxact = MarkAsPreparing(xid, gid,
-									hdr->prepared_at,
-									hdr->owner, hdr->database);
+			gxact = RecoverPreparedFromBuffer(buf);
 			gxact->ondisk = true;
-			GXactLoadSubxactData(gxact, hdr->nsubxacts, subxids);
 			MarkAsPrepared(gxact);
 
-			/*
-			 * Recover other state (notably locks) using resource managers
-			 */
-			ProcessRecords(bufptr, xid, twophase_recover_callbacks);
-
-			/*
-			 * Release locks held by the standby process after we process each
-			 * prepared transaction. As a result, we don't need too many
-			 * additional locks at any one time.
-			 */
-			if (InHotStandby)
-				StandbyReleaseLockTree(xid, hdr->nsubxacts, subxids);
-
-			/*
-			 * We're done with recovering this transaction. Clear
-			 * MyLockedGxact, like we do in PrepareTransaction() during normal
-			 * operation.
-			 */
-			PostPrepare_Twophase();
-
 			pfree(buf);
 		}
 	}
 	FreeDir(cldir);
 }
 
+
+/*
+ * RecoverPreparedFromXLOG
+ *
+ * To avoid the creation of twophase state files during replay we register
+ * WAL records for prepared transactions in shared memory in the same way
+ * during normal operations. If replay faces a WAL record for a COMMIT
+ * PREPARED transaction before a checkpoint or restartpoint happens then
+ * no files are used, limiting the I/O impact of such operations during
+ * recovery.
+ */
+void
+RecoverPreparedFromXLOG(XLogReaderState *record)
+{
+	GlobalTransaction gxact;
+
+	gxact = RecoverPreparedFromBuffer((char *) XLogRecGetData(record));
+	gxact->prepare_start_lsn = record->ReadRecPtr;
+	gxact->prepare_end_lsn = record->EndRecPtr;
+	MarkAsPrepared(gxact);
+}
+
+
 /*
  *	RecordTransactionCommitPrepared
  *
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index e11b229..6a40425 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -5602,7 +5602,7 @@ xact_redo(XLogReaderState *record)
 			Assert(TransactionIdIsValid(parsed.twophase_xid));
 			xact_redo_commit(&parsed, parsed.twophase_xid,
 							 record->EndRecPtr, XLogRecGetOrigin(record));
-			RemoveTwoPhaseFile(parsed.twophase_xid, false);
+			XlogRedoFinishPrepared(parsed.twophase_xid, true);
 		}
 	}
 	else if (info == XLOG_XACT_ABORT || info == XLOG_XACT_ABORT_PREPARED)
@@ -5622,14 +5622,12 @@ xact_redo(XLogReaderState *record)
 		{
 			Assert(TransactionIdIsValid(parsed.twophase_xid));
 			xact_redo_abort(&parsed, parsed.twophase_xid);
-			RemoveTwoPhaseFile(parsed.twophase_xid, false);
+			XlogRedoFinishPrepared(parsed.twophase_xid, false);
 		}
 	}
 	else if (info == XLOG_XACT_PREPARE)
 	{
-		/* the record contents are exactly the 2PC file */
-		RecreateTwoPhaseFile(XLogRecGetXid(record),
-						  XLogRecGetData(record), XLogRecGetDataLen(record));
+		RecoverPreparedFromXLOG(record);
 	}
 	else if (info == XLOG_XACT_ASSIGNMENT)
 	{
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 2189c22..1ba6f14 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -5993,7 +5993,6 @@ StartupXLOG(void)
 	TimeLineID	EndOfLogTLI;
 	TimeLineID	PrevTimeLineID;
 	XLogRecord *record;
-	TransactionId oldestActiveXID;
 	bool		backupEndRequired = false;
 	bool		backupFromStandby = false;
 	DBState		dbstate_at_startup;
@@ -6513,6 +6512,24 @@ StartupXLOG(void)
 		InRecovery = true;
 	}
 
+	/*
+	 * Recover two-phase state of old prepared transactions.
+	 *
+	 * Long-running prepared transaction are moved to filesystem during
+	 * checkpoint as commit will need info from prepare record that will
+	 * be unavalable. So here we are following logical order of things and
+	 * restoring transactions that are behind checkpoint horizon before replay
+	 * started.
+	 *
+	 * RecoverPreparedFromFiles and cosequent replay of prepare records
+	 * will need access to SUBTRANCE structures, so init it here using
+	 * checkpoint's oldestActiveXid as it certainly not higher than latest
+	 * prepared xid.
+	 */
+	StartupCLOG();
+	StartupSUBTRANS(checkPoint.oldestActiveXid);
+	RecoverPreparedFromFiles();
+
 	/* REDO */
 	if (InRecovery)
 	{
@@ -6649,32 +6666,17 @@ StartupXLOG(void)
 		 */
 		if (ArchiveRecoveryRequested && EnableHotStandby)
 		{
-			TransactionId *xids;
-			int			nxids;
-
 			ereport(DEBUG1,
 					(errmsg("initializing for hot standby")));
 
 			InitRecoveryTransactionEnvironment();
 
-			if (wasShutdown)
-				oldestActiveXID = PrescanPreparedTransactions(&xids, &nxids);
-			else
-				oldestActiveXID = checkPoint.oldestActiveXid;
-			Assert(TransactionIdIsValid(oldestActiveXID));
+			Assert(TransactionIdIsValid(checkPoint.oldestActiveXid));
 
 			/* Tell procarray about the range of xids it has to deal with */
 			ProcArrayInitRecovery(ShmemVariableCache->nextXid);
 
 			/*
-			 * Startup commit log and subtrans only.  MultiXact and commit
-			 * timestamp have already been started up and other SLRUs are not
-			 * maintained during recovery and need not be started yet.
-			 */
-			StartupCLOG();
-			StartupSUBTRANS(oldestActiveXID);
-
-			/*
 			 * If we're beginning at a shutdown checkpoint, we know that
 			 * nothing was running on the master at this point. So fake-up an
 			 * empty running-xacts record and use that here and now. Recover
@@ -6683,7 +6685,11 @@ StartupXLOG(void)
 			if (wasShutdown)
 			{
 				RunningTransactionsData running;
-				TransactionId latestCompletedXid;
+				TransactionId   latestCompletedXid;
+				TransactionId  *xids;
+				int				nxids;
+
+				GetPreparedTransactions(&xids, &nxids);
 
 				/*
 				 * Construct a RunningTransactions snapshot representing a
@@ -6695,7 +6701,7 @@ StartupXLOG(void)
 				running.subxcnt = 0;
 				running.subxid_overflow = false;
 				running.nextXid = checkPoint.nextXid;
-				running.oldestRunningXid = oldestActiveXID;
+				running.oldestRunningXid = checkPoint.oldestActiveXid;
 				latestCompletedXid = checkPoint.nextXid;
 				TransactionIdRetreat(latestCompletedXid);
 				Assert(TransactionIdIsNormal(latestCompletedXid));
@@ -6703,8 +6709,6 @@ StartupXLOG(void)
 				running.xids = xids;
 
 				ProcArrayApplyRecoveryInfo(&running);
-
-				StandbyRecoverPreparedTransactions(false);
 			}
 		}
 
@@ -7012,6 +7016,8 @@ StartupXLOG(void)
 					ereport(FATAL,
 							(errmsg("requested recovery stop point is before consistent recovery point")));
 
+				CleanupPreparedTransactionsOnPITR();
+
 				/*
 				 * This is the last point where we can restart recovery with a
 				 * new recovery target, if we shutdown and begin again. After
@@ -7272,9 +7278,6 @@ StartupXLOG(void)
 	XLogCtl->LogwrtRqst.Write = EndOfLog;
 	XLogCtl->LogwrtRqst.Flush = EndOfLog;
 
-	/* Pre-scan prepared transactions to find out the range of XIDs present */
-	oldestActiveXID = PrescanPreparedTransactions(NULL, NULL);
-
 	/*
 	 * Update full_page_writes in shared memory and write an XLOG_FPW_CHANGE
 	 * record before resource manager writes cleanup WAL records or checkpoint
@@ -7447,24 +7450,11 @@ StartupXLOG(void)
 	LWLockRelease(ProcArrayLock);
 
 	/*
-	 * Start up the commit log and subtrans, if not already done for hot
-	 * standby.  (commit timestamps are started below, if necessary.)
-	 */
-	if (standbyState == STANDBY_DISABLED)
-	{
-		StartupCLOG();
-		StartupSUBTRANS(oldestActiveXID);
-	}
-
-	/*
 	 * Perform end of recovery actions for any SLRUs that need it.
 	 */
 	TrimCLOG();
 	TrimMultiXact();
 
-	/* Reload shared-memory state for prepared transactions */
-	RecoverPreparedTransactions();
-
 	/*
 	 * Shutdown the recovery environment. This must occur after
 	 * RecoverPreparedTransactions(), see notes for lock_twophase_recover()
@@ -8311,14 +8301,16 @@ CreateCheckPoint(int flags)
 	checkPoint.time = (pg_time_t) time(NULL);
 
 	/*
-	 * For Hot Standby, derive the oldestActiveXid before we fix the redo
-	 * pointer. This allows us to begin accumulating changes to assemble our
-	 * starting snapshot of locks and transactions.
+	 * Before recovery we need to start SUBTRANS in a way that it will be
+	 * capable of storing info about possible two-phase transactions that we
+	 * will prepare during xlog replay. We can get here value that is less
+	 * then actual latest prepared transaction, but that is okay.
+	 *
+	 * Also for Hot Standby, this allows us to derive the oldestActiveXid
+	 * before we fix the redo pointer and to begin accumulating changes to
+	 * assemble our starting snapshot of locks and transactions.
 	 */
-	if (!shutdown && XLogStandbyInfoActive())
-		checkPoint.oldestActiveXid = GetOldestActiveTransactionId();
-	else
-		checkPoint.oldestActiveXid = InvalidTransactionId;
+	checkPoint.oldestActiveXid = GetOldestActiveTransactionId();
 
 	/*
 	 * We must block concurrent insertions while examining insert state to
@@ -9352,11 +9344,10 @@ xlog_redo(XLogReaderState *record)
 		{
 			TransactionId *xids;
 			int			nxids;
-			TransactionId oldestActiveXID;
 			TransactionId latestCompletedXid;
 			RunningTransactionsData running;
 
-			oldestActiveXID = PrescanPreparedTransactions(&xids, &nxids);
+			GetPreparedTransactions(&xids, &nxids);
 
 			/*
 			 * Construct a RunningTransactions snapshot representing a shut
@@ -9368,7 +9359,7 @@ xlog_redo(XLogReaderState *record)
 			running.subxcnt = 0;
 			running.subxid_overflow = false;
 			running.nextXid = checkPoint.nextXid;
-			running.oldestRunningXid = oldestActiveXID;
+			running.oldestRunningXid = checkPoint.oldestActiveXid;
 			latestCompletedXid = checkPoint.nextXid;
 			TransactionIdRetreat(latestCompletedXid);
 			Assert(TransactionIdIsNormal(latestCompletedXid));
@@ -9376,8 +9367,6 @@ xlog_redo(XLogReaderState *record)
 			running.xids = xids;
 
 			ProcArrayApplyRecoveryInfo(&running);
-
-			StandbyRecoverPreparedTransactions(true);
 		}
 
 		/* ControlFile->checkPointCopy always tracks the latest ckpt XID */
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index e5d487d..ec2371c 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2078,7 +2078,8 @@ GetRunningTransactionData(void)
  * We look at all databases, though there is no need to include WALSender
  * since this has no effect on hot standby conflicts.
  *
- * This is never executed during recovery so there is no need to look at
+ * During recovery we need only minimum anong prepared transaction and since
+ * they are registering dummy PGPROC during replay there is no need to look at
  * KnownAssignedXids.
  *
  * We don't worry about updating other counters, we want to keep this as
@@ -2092,8 +2093,6 @@ GetOldestActiveTransactionId(void)
 	TransactionId oldestRunningXid;
 	int			index;
 
-	Assert(!RecoveryInProgress());
-
 	LWLockAcquire(ProcArrayLock, LW_SHARED);
 
 	/*
diff --git a/src/backend/storage/ipc/standby.c b/src/backend/storage/ipc/standby.c
index 547f1a8..5ea2530 100644
--- a/src/backend/storage/ipc/standby.c
+++ b/src/backend/storage/ipc/standby.c
@@ -608,7 +608,8 @@ StandbyAcquireAccessExclusiveLock(TransactionId xid, Oid dbOid, Oid relOid)
 	/* Already processed? */
 	if (!TransactionIdIsValid(xid) ||
 		TransactionIdDidCommit(xid) ||
-		TransactionIdDidAbort(xid))
+		TransactionIdDidAbort(xid) ||
+		StandbyTransactionIdIsPrepared(xid))
 		return;
 
 	elog(trace_recovery(DEBUG4),
diff --git a/src/include/access/twophase.h b/src/include/access/twophase.h
index b7ce0c6..f6fbf78 100644
--- a/src/include/access/twophase.h
+++ b/src/include/access/twophase.h
@@ -17,6 +17,7 @@
 #include "access/xlogdefs.h"
 #include "datatype/timestamp.h"
 #include "storage/lock.h"
+#include "access/xlogreader.h"
 
 /*
  * GlobalTransactionData is defined in twophase.c; other places have no
@@ -44,10 +45,11 @@ extern void StartPrepare(GlobalTransaction gxact);
 extern void EndPrepare(GlobalTransaction gxact);
 extern bool StandbyTransactionIdIsPrepared(TransactionId xid);
 
-extern TransactionId PrescanPreparedTransactions(TransactionId **xids_p,
+extern void CleanupPreparedTransactionsOnPITR(void);
+extern void GetPreparedTransactions(TransactionId **xids_p,
 							int *nxids_p);
-extern void StandbyRecoverPreparedTransactions(bool overwriteOK);
-extern void RecoverPreparedTransactions(void);
+extern void RecoverPreparedFromFiles(void);
+extern void RecoverPreparedFromXLOG(XLogReaderState *record);
 
 extern void RecreateTwoPhaseFile(TransactionId xid, void *content, int len);
 extern void RemoveTwoPhaseFile(TransactionId xid, bool giveWarning);
@@ -56,4 +58,5 @@ extern void CheckPointTwoPhase(XLogRecPtr redo_horizon);
 
 extern void FinishPreparedTransaction(const char *gid, bool isCommit);
 
+extern void XlogRedoFinishPrepared(TransactionId xid, bool isCommit);
 #endif   /* TWOPHASE_H */
diff --git a/src/include/catalog/pg_control.h b/src/include/catalog/pg_control.h
index 0bc41ab..c42e05e 100644
--- a/src/include/catalog/pg_control.h
+++ b/src/include/catalog/pg_control.h
@@ -50,14 +50,7 @@ typedef struct CheckPoint
 										 * timestamp */
 	TransactionId newestCommitTsXid;	/* newest Xid with valid commit
 										 * timestamp */
-
-	/*
-	 * Oldest XID still running. This is only needed to initialize hot standby
-	 * mode from an online checkpoint, so we only bother calculating this for
-	 * online checkpoints and only when wal_level is replica. Otherwise it's
-	 * set to InvalidTransactionId.
-	 */
-	TransactionId oldestActiveXid;
+	TransactionId oldestActiveXid;		/* Oldest XID still running. */
 } CheckPoint;
 
 /* XLOG info values for XLOG rmgr */
diff --git a/src/test/recovery/t/008_twophase.pl b/src/test/recovery/t/008_twophase.pl
new file mode 100644
index 0000000..7ada751
--- /dev/null
+++ b/src/test/recovery/t/008_twophase.pl
@@ -0,0 +1,314 @@
+# Tests dedicated to two-phase commit in recovery
+use strict;
+use warnings;
+use PostgresNode;
+use TestLib;
+use Test::More tests => 13;
+
+# Setup master node
+my $node_master = get_new_node("master");
+$node_master->init(allows_streaming => 1);
+$node_master->append_conf('postgresql.conf', qq(
+	max_prepared_transactions = 10
+));
+$node_master->start;
+$node_master->backup('master_backup');
+$node_master->psql('postgres', "create table t(id int)");
+
+# Setup master node
+my $node_slave = get_new_node('slave');
+$node_slave->init_from_backup($node_master, 'master_backup', has_streaming => 1);
+$node_slave->start;
+
+# Switch to synchronous replication
+$node_master->append_conf('postgresql.conf', qq(
+	synchronous_standby_names = '*'
+));
+$node_master->psql('postgres', "select pg_reload_conf()");
+
+my $psql_out = '';
+my $psql_rc = '';
+
+###############################################################################
+# Check that we can commit and abort tx after soft restart.
+# Here checkpoint happens before shutdown and no WAL replay will occur at next
+# startup. In this case postgres re-create shared-memory state from twophase
+# files.
+###############################################################################
+
+$node_master->psql('postgres', "
+	begin;
+	insert into t values (42);
+	savepoint s1;
+	insert into t values (43);
+	prepare transaction 'x';
+	begin;
+	insert into t values (142);
+	savepoint s1;
+	insert into t values (143);
+	prepare transaction 'y';");
+$node_master->stop;
+$node_master->start;
+
+$psql_rc = $node_master->psql('postgres', "commit prepared 'x'");
+is($psql_rc, '0', 'Commit prepared transaction after restart.');
+
+$psql_rc = $node_master->psql('postgres', "rollback prepared 'y'");
+is($psql_rc, '0', 'Rollback prepared transaction after restart.');
+
+###############################################################################
+# Check that we can commit and abort after hard restart.
+# At next startup, WAL replay will re-create shared memory state for prepared
+# transaction using dedicated WAL records.
+###############################################################################
+
+$node_master->psql('postgres', "
+	checkpoint;
+	begin;
+	insert into t values (42);
+	savepoint s1;
+	insert into t values (43);
+	prepare transaction 'x';
+	begin;
+	insert into t values (142);
+	savepoint s1;
+	insert into t values (143);
+	prepare transaction 'y';");
+$node_master->teardown_node;
+$node_master->start;
+
+$psql_rc = $node_master->psql('postgres', "commit prepared 'x'");
+is($psql_rc, '0', 'Commit prepared tx after teardown.');
+
+$psql_rc = $node_master->psql('postgres', "rollback prepared 'y'");
+is($psql_rc, '0', 'Rollback prepared transaction after teardown.');
+
+###############################################################################
+# Check that WAL replay can handle several transactions with same name GID.
+###############################################################################
+
+$node_master->psql('postgres', "
+	checkpoint;
+	begin;
+	insert into t values (42);
+	savepoint s1;
+	insert into t values (43);
+	prepare transaction 'x';
+	commit prepared 'x';
+	begin;
+	insert into t values (42);
+	savepoint s1;
+	insert into t values (43);
+	prepare transaction 'x';");
+$node_master->teardown_node;
+$node_master->start;
+
+$psql_rc = $node_master->psql('postgres', "commit prepared 'x'");
+is($psql_rc, '0', 'Replay several transactions with same GID.');
+
+###############################################################################
+# Check that WAL replay cleans up its shared memory state and releases locks
+# while replaying transaction commits.
+###############################################################################
+
+$node_master->psql('postgres', "
+	begin;
+	insert into t values (42);
+	savepoint s1;
+	insert into t values (43);
+	prepare transaction 'x';
+	commit prepared 'x';");
+$node_master->teardown_node;
+$node_master->start;
+$psql_rc = $node_master->psql('postgres', "begin;
+	insert into t values (42);
+	savepoint s1;
+	insert into t values (43);
+	-- This prepare can fail due to conflicting GID or locks conflicts if
+	-- replay did not fully cleanup its state on previous commit.
+	prepare transaction 'x';");
+is($psql_rc, '0', "Cleanup of shared memory state for 2PC commit");
+
+$node_master->psql('postgres', "commit prepared 'x'");
+
+###############################################################################
+# Check that WAL replay will cleanup its shared memory state on running slave.
+###############################################################################
+
+$node_master->psql('postgres', "
+	begin;
+	insert into t values (42);
+	savepoint s1;
+	insert into t values (43);
+	prepare transaction 'x';
+	commit prepared 'x';");
+$node_slave->psql('postgres', "select count(*) from pg_prepared_xacts;",
+	  stdout => \$psql_out);
+is($psql_out, '0',
+   "Cleanup of shared memory state on running standby without checkpoint.");
+
+###############################################################################
+# Same as in previous case, but let's force checkpoint on slave between
+# prepare and commit to use on-disk twophase files.
+###############################################################################
+
+$node_master->psql('postgres', "
+	begin;
+	insert into t values (42);
+	savepoint s1;
+	insert into t values (43);
+	prepare transaction 'x';");
+$node_slave->psql('postgres', "checkpoint;");
+$node_master->psql('postgres', "commit prepared 'x';");
+$node_slave->psql('postgres', "select count(*) from pg_prepared_xacts;",
+	  stdout => \$psql_out);
+is($psql_out, '0',
+   "Cleanup of shared memory state on running standby after checkpoint.");
+
+###############################################################################
+# Check that prepared transactions can be committed on promoted slave.
+###############################################################################
+
+$node_master->psql('postgres', "
+	begin;
+	insert into t values (42);
+	savepoint s1;
+	insert into t values (43);
+	prepare transaction 'x';");
+$node_master->teardown_node;
+$node_slave->promote;
+$node_slave->poll_query_until('postgres', "SELECT pg_is_in_recovery() <> true");
+
+$psql_rc = $node_slave->psql('postgres', "commit prepared 'x';");
+is($psql_rc, '0', "Restore of prepared transaction on promoted slave.");
+
+# change roles
+($node_master, $node_slave) = ($node_slave, $node_master);
+$node_slave->enable_streaming($node_master);
+$node_slave->append_conf('recovery.conf', qq(
+recovery_target_timeline='latest'
+));
+$node_slave->start;
+
+###############################################################################
+# Check that prepared transactions are replayed after soft restart of standby
+# while master is down. Since standby knows that master is down it uses
+# different code path on start to be sure that the status of transactions is
+# consistent.
+###############################################################################
+
+$node_master->psql('postgres', "
+	begin;
+	insert into t values (42);
+	savepoint s1;
+	insert into t values (43);
+	prepare transaction 'x';");
+$node_master->stop;
+$node_slave->restart;
+$node_slave->promote;
+$node_slave->poll_query_until('postgres', "SELECT pg_is_in_recovery() <> true");
+
+$node_slave->psql('postgres', "select count(*) from pg_prepared_xacts",
+	  stdout => \$psql_out);
+is($psql_out, '1',
+   "Restore prepared transactions from files with master down.");
+
+# restore state
+($node_master, $node_slave) = ($node_slave, $node_master);
+$node_slave->enable_streaming($node_master);
+$node_slave->append_conf('recovery.conf', qq(
+recovery_target_timeline='latest'
+));
+$node_slave->start;
+$node_master->psql('postgres', "commit prepared 'x'");
+
+###############################################################################
+# Check that prepared transactions are correctly replayed after slave hard
+# restart while master is down.
+###############################################################################
+
+$node_master->psql('postgres', "
+	begin;
+	insert into t values (242);
+	savepoint s1;
+	insert into t values (243);
+	prepare transaction 'x';
+	");
+$node_master->stop;
+$node_slave->teardown_node;
+$node_slave->start;
+$node_slave->promote;
+$node_slave->poll_query_until('postgres',
+	  "SELECT pg_is_in_recovery() <> true");
+
+$node_slave->psql('postgres', "select count(*) from pg_prepared_xacts",
+	  stdout => \$psql_out);
+is($psql_out, '1',
+   "Restore prepared transactions from records with master down.");
+
+# restore state
+($node_master, $node_slave) = ($node_slave, $node_master);
+$node_slave->enable_streaming($node_master);
+$node_slave->append_conf('recovery.conf', qq(
+recovery_target_timeline='latest'
+));
+$node_slave->start;
+$node_master->psql('postgres', "commit prepared 'x'");
+
+
+###############################################################################
+# Check for a lock confcict between prepared tx with DDL inside and replay of
+# XLOG_STANDBY_LOCK wal record.
+###############################################################################
+
+$node_master->psql('postgres', "
+	begin;
+	create table t2(id int);
+	savepoint s1;
+	insert into t2 values (42);
+	prepare transaction 'x';
+	-- checkpoint will issue XLOG_STANDBY_LOCK that can conflict with lock
+	-- held by 'create table' statement
+	checkpoint;
+	commit prepared 'x';");
+
+$node_slave->psql('postgres', "select count(*) from pg_prepared_xacts",
+	  stdout => \$psql_out);
+is($psql_out, '0', "Replay prepared transaction with DDL.");
+
+
+###############################################################################
+# Check that replay will correctly set SUBTRANS and properly andvance nextXid
+# so it won't conflict with savepoint xids.
+###############################################################################
+
+$node_master->psql('postgres', "
+	begin;
+	delete from t;
+	insert into t values (43);
+	savepoint s1;
+	insert into t values (43);
+	savepoint s2;
+	insert into t values (43);
+	savepoint s3;
+	insert into t values (43);
+	savepoint s4;
+	insert into t values (43);
+	savepoint s5;
+	insert into t values (43);
+	prepare transaction 'x';
+	checkpoint;");
+
+$node_master->stop;
+$node_master->start;
+$node_master->psql('postgres', "
+	-- here we can get xid of previous savepoint if nextXid
+	-- wasn't properly advanced
+	begin;
+	insert into t values (142);
+	abort;
+	commit prepared 'x';");
+
+$node_master->psql('postgres', "select count(*) from t",
+	  stdout => \$psql_out);
+is($psql_out, '6', "Check nextXid handling for prepared subtransactions");
\ No newline at end of file
