From 7fac0f2d35591155493fec637ec1df3c2b7239e0 Mon Sep 17 00:00:00 2001
From: "houzj.fnst" <houzj.fnst@cn.fujitsu.com>
Date: Wed, 20 Apr 2022 16:45:07 +0800
Subject: [PATCH] Perform streaming logical transactions by background workers

Currently, for large transactions, the publisher sends the data in multiple
streams (changes divided into chunks depending upon logical_decoding_work_mem),
and then on the subscriber-side, the apply worker writes the changes into
temporary files and once it receives the commit, it read from the file and
apply the entire transaction. To improve the performance of such transactions,
we can instead allow them to be applied via background workers.

In this approach, we assign a new bgworker (if available) as soon as the xact's
first stream came and the main apply worker will send changes to this new
worker via shared memory. The bgworker will directly apply the change instead
of writing it to temporary files.  We keep this worker assigned till the
transaction commit came and also wait for the worker to finish at commit. This
preserves commit ordering and avoid writing to and reading from file in most
cases. We still need to spill if there is no worker available. We also need to
allow stream_stop to complete by the background worker to finish it to avoid
deadlocks because T-1's current stream of changes can update rows in
conflicting order with T-2's next stream of changes.

Also extend the subscription streaming option so that user can control whether
apply the streaming transaction in a bgworker or spill the change to disk. User
can set the streaming option to 'on/off', 'apply', 'spool'. For now, 'on' and
'apply' means the streaming will be applied via a bgworker if available.
'spool' means the streaming transaction will be spilled to disk.
---
 doc/src/sgml/ref/create_subscription.sgml   |   24 +-
 src/backend/commands/subscriptioncmds.c     |   70 +-
 src/backend/postmaster/bgworker.c           |    3 +
 src/backend/replication/logical/launcher.c  |   72 +-
 src/backend/replication/logical/origin.c    |   10 +-
 src/backend/replication/logical/proto.c     |    7 +-
 src/backend/replication/logical/tablesync.c |   10 +-
 src/backend/replication/logical/worker.c    | 1532 ++++++++++++++++++++++++---
 src/backend/utils/activity/wait_event.c     |    3 +
 src/bin/pg_dump/pg_dump.c                   |    8 +-
 src/include/catalog/pg_subscription.h       |   16 +-
 src/include/replication/logicalproto.h      |    4 +-
 src/include/replication/logicalworker.h     |    1 +
 src/include/replication/origin.h            |    2 +-
 src/include/replication/worker_internal.h   |    7 +-
 src/include/utils/wait_event.h              |    1 +
 src/test/subscription/t/029_on_error.pl     |    2 +-
 17 files changed, 1564 insertions(+), 208 deletions(-)

diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 203bb41..0b4eb64 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -217,13 +217,29 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
        </varlistentry>
 
        <varlistentry>
-        <term><literal>streaming</literal> (<type>boolean</type>)</term>
+        <term><literal>streaming</literal> (<type>enum</type>)</term>
         <listitem>
          <para>
           Specifies whether to enable streaming of in-progress transactions
-          for this subscription.  By default, all transactions
-          are fully decoded on the publisher and only then sent to the
-          subscriber as a whole.
+          for this subscription.  The default value is <literal>off</literal>,
+          all transactions are fully decoded on the publisher and only then
+          sent to the subscriber as a whole.
+         </para>
+
+         <para>
+          If set to <literal>spool</literal> or <literal>on</literal>, the changes
+          of transaction are written to temporary files and then applied at
+          once after the transaction is committed on the publisher.
+         </para>
+
+         <para>
+          If set to <literal>apply</literal>, incoming changes are directly
+          applied via one of the background worker, if available. If no
+          background worker is free to handle streaming transaction then the
+          changes are written to a file and applied after the transaction is
+          committed. Note that if error happen when applying changes in
+          background worker, it might not report the finish LSN of the remote
+          transaction in server log.
          </para>
         </listitem>
        </varlistentry>
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index b94236f..ab7a668 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -83,7 +83,7 @@ typedef struct SubOpts
 	bool		copy_data;
 	bool		refresh;
 	bool		binary;
-	bool		streaming;
+	char		streaming;
 	bool		twophase;
 	bool		disableonerr;
 	XLogRecPtr	lsn;
@@ -96,6 +96,66 @@ static void ReportSlotConnectionError(List *rstates, Oid subid, char *slotname,
 
 
 /*
+ * Extract the streaming mode value from a DefElem.  This is like
+ * defGetBoolean() but also accepts the special value "spool" and "apply".
+ */
+static char
+defGetStreamingMode(DefElem *def)
+{
+	/*
+	 * If no parameter given, assume "true" is meant.
+	 */
+	if (def->arg == NULL)
+		return SUBSTREAM_APPLY;
+
+	/*
+	 * Allow 0, 1, "true", "false", "on", "off", "spool" or "apply".
+	 */
+	switch (nodeTag(def->arg))
+	{
+		case T_Integer:
+			switch (intVal(def->arg))
+			{
+				case 0:
+					return SUBSTREAM_OFF;
+				case 1:
+					return SUBSTREAM_APPLY;
+				default:
+					/* otherwise, error out below */
+					break;
+			}
+			break;
+		default:
+			{
+				char	*sval = defGetString(def);
+
+				/*
+				 * The set of strings accepted here should match up with the
+				 * grammar's opt_boolean_or_string production.
+				 */
+				if (pg_strcasecmp(sval, "true") == 0)
+					return SUBSTREAM_APPLY;
+				if (pg_strcasecmp(sval, "false") == 0)
+					return SUBSTREAM_OFF;
+				if (pg_strcasecmp(sval, "on") == 0)
+					return SUBSTREAM_APPLY;
+				if (pg_strcasecmp(sval, "off") == 0)
+					return SUBSTREAM_OFF;
+				if (pg_strcasecmp(sval, "spool") == 0)
+					return SUBSTREAM_SPOOL;
+				if (pg_strcasecmp(sval, "apply") == 0)
+					return SUBSTREAM_APPLY;
+			}
+			break;
+	}
+	ereport(ERROR,
+				(errcode(ERRCODE_SYNTAX_ERROR),
+				 errmsg("%s requires a Boolean value or \"spool\" or \"apply\"",
+					def->defname)));
+	return SUBSTREAM_OFF;	/* keep compiler quiet */
+}
+
+/*
  * Common option parsing function for CREATE and ALTER SUBSCRIPTION commands.
  *
  * Since not all options can be specified in both commands, this function
@@ -132,7 +192,7 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
 	if (IsSet(supported_opts, SUBOPT_BINARY))
 		opts->binary = false;
 	if (IsSet(supported_opts, SUBOPT_STREAMING))
-		opts->streaming = false;
+		opts->streaming = SUBSTREAM_OFF;
 	if (IsSet(supported_opts, SUBOPT_TWOPHASE_COMMIT))
 		opts->twophase = false;
 	if (IsSet(supported_opts, SUBOPT_DISABLE_ON_ERR))
@@ -233,7 +293,7 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
 				errorConflictingDefElem(defel, pstate);
 
 			opts->specified_opts |= SUBOPT_STREAMING;
-			opts->streaming = defGetBoolean(defel);
+			opts->streaming = defGetStreamingMode(defel);
 		}
 		else if (strcmp(defel->defname, "two_phase") == 0)
 		{
@@ -601,7 +661,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 	values[Anum_pg_subscription_subowner - 1] = ObjectIdGetDatum(owner);
 	values[Anum_pg_subscription_subenabled - 1] = BoolGetDatum(opts.enabled);
 	values[Anum_pg_subscription_subbinary - 1] = BoolGetDatum(opts.binary);
-	values[Anum_pg_subscription_substream - 1] = BoolGetDatum(opts.streaming);
+	values[Anum_pg_subscription_substream - 1] = CharGetDatum(opts.streaming);
 	values[Anum_pg_subscription_subtwophasestate - 1] =
 		CharGetDatum(opts.twophase ?
 					 LOGICALREP_TWOPHASE_STATE_PENDING :
@@ -1060,7 +1120,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
 				if (IsSet(opts.specified_opts, SUBOPT_STREAMING))
 				{
 					values[Anum_pg_subscription_substream - 1] =
-						BoolGetDatum(opts.streaming);
+						CharGetDatum(opts.streaming);
 					replaces[Anum_pg_subscription_substream - 1] = true;
 				}
 
diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 30682b6..194a36b 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -128,6 +128,9 @@ static const struct
 	},
 	{
 		"ApplyWorkerMain", ApplyWorkerMain
+	},
+	{
+		"LogicalApplyBgwMain", LogicalApplyBgwMain
 	}
 };
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 0adb2d1..bac737c 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -72,6 +72,7 @@ static void logicalrep_launcher_onexit(int code, Datum arg);
 static void logicalrep_worker_onexit(int code, Datum arg);
 static void logicalrep_worker_detach(void);
 static void logicalrep_worker_cleanup(LogicalRepWorker *worker);
+static void stop_worker(LogicalRepWorker *worker);
 
 static bool on_commit_launcher_wakeup = false;
 
@@ -225,7 +226,7 @@ logicalrep_worker_find(Oid subid, Oid relid, bool only_running)
 		LogicalRepWorker *w = &LogicalRepCtx->workers[i];
 
 		if (w->in_use && w->subid == subid && w->relid == relid &&
-			(!only_running || w->proc))
+			(!only_running || w->proc) && !w->subworker)
 		{
 			res = w;
 			break;
@@ -262,9 +263,9 @@ logicalrep_workers_find(Oid subid, bool only_running)
 /*
  * Start new apply background worker, if possible.
  */
-void
+bool
 logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
-						 Oid relid)
+						 Oid relid, dsm_handle subworker_dsm)
 {
 	BackgroundWorker bgw;
 	BackgroundWorkerHandle *bgw_handle;
@@ -351,7 +352,7 @@ retry:
 	if (OidIsValid(relid) && nsyncworkers >= max_sync_workers_per_subscription)
 	{
 		LWLockRelease(LogicalRepWorkerLock);
-		return;
+		return false;
 	}
 
 	/*
@@ -365,7 +366,7 @@ retry:
 				(errcode(ERRCODE_CONFIGURATION_LIMIT_EXCEEDED),
 				 errmsg("out of logical replication worker slots"),
 				 errhint("You might need to increase max_logical_replication_workers.")));
-		return;
+		return false;
 	}
 
 	/* Prepare the worker slot. */
@@ -380,6 +381,7 @@ retry:
 	worker->relstate = SUBREL_STATE_UNKNOWN;
 	worker->relstate_lsn = InvalidXLogRecPtr;
 	worker->stream_fileset = NULL;
+	worker->subworker = (subworker_dsm != DSM_HANDLE_INVALID);
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -397,19 +399,31 @@ retry:
 		BGWORKER_BACKEND_DATABASE_CONNECTION;
 	bgw.bgw_start_time = BgWorkerStart_RecoveryFinished;
 	snprintf(bgw.bgw_library_name, BGW_MAXLEN, "postgres");
-	snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
+
+	if (subworker_dsm == DSM_HANDLE_INVALID)
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
+	else
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "LogicalApplyBgwMain");
+
+
 	if (OidIsValid(relid))
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
 				 "logical replication worker for subscription %u sync %u", subid, relid);
-	else
+	else if (subworker_dsm == DSM_HANDLE_INVALID)
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
 				 "logical replication worker for subscription %u", subid);
+	else
+		snprintf(bgw.bgw_name, BGW_MAXLEN,
+				 "logical replication apply worker for subscription %u", subid);
 	snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication worker");
 
 	bgw.bgw_restart_time = BGW_NEVER_RESTART;
 	bgw.bgw_notify_pid = MyProcPid;
 	bgw.bgw_main_arg = Int32GetDatum(slot);
 
+	if (subworker_dsm != DSM_HANDLE_INVALID)
+		memcpy(bgw.bgw_extra, &subworker_dsm, sizeof(dsm_handle));
+
 	if (!RegisterDynamicBackgroundWorker(&bgw, &bgw_handle))
 	{
 		/* Failed to start worker, so clean up the worker slot. */
@@ -422,11 +436,13 @@ retry:
 				(errcode(ERRCODE_CONFIGURATION_LIMIT_EXCEEDED),
 				 errmsg("out of background worker slots"),
 				 errhint("You might need to increase max_worker_processes.")));
-		return;
+		return false;
 	}
 
 	/* Now wait until it attaches. */
 	WaitForReplicationWorkerAttach(worker, generation, bgw_handle);
+
+	return true;
 }
 
 /*
@@ -437,7 +453,6 @@ void
 logicalrep_worker_stop(Oid subid, Oid relid)
 {
 	LogicalRepWorker *worker;
-	uint16		generation;
 
 	LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
 
@@ -450,6 +465,18 @@ logicalrep_worker_stop(Oid subid, Oid relid)
 		return;
 	}
 
+	stop_worker(worker);
+
+	LWLockRelease(LogicalRepWorkerLock);
+}
+
+static void
+stop_worker(LogicalRepWorker *worker)
+{
+	uint16		generation;
+
+	Assert(LWLockHeldByMe(LogicalRepWorkerLock));
+
 	/*
 	 * Remember which generation was our worker so we can check if what we see
 	 * is still the same one.
@@ -523,8 +550,6 @@ logicalrep_worker_stop(Oid subid, Oid relid)
 
 		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
 	}
-
-	LWLockRelease(LogicalRepWorkerLock);
 }
 
 /*
@@ -600,6 +625,28 @@ logicalrep_worker_attach(int slot)
 static void
 logicalrep_worker_detach(void)
 {
+	/*
+	 * If we are the main apply worker, stop all the sub apply workers we
+	 * started before.
+	 */
+	if (!MyLogicalRepWorker->subworker)
+	{
+		List	 *workers;
+		ListCell *lc;
+
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+
+		workers = logicalrep_workers_find(MyLogicalRepWorker->subid, true);
+		foreach(lc, workers)
+		{
+			LogicalRepWorker *w = (LogicalRepWorker *) lfirst(lc);
+			if (w->subworker)
+				stop_worker(w);
+		}
+
+		LWLockRelease(LogicalRepWorkerLock);
+	}
+
 	/* Block concurrent access. */
 	LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
 
@@ -622,6 +669,7 @@ logicalrep_worker_cleanup(LogicalRepWorker *worker)
 	worker->userid = InvalidOid;
 	worker->subid = InvalidOid;
 	worker->relid = InvalidOid;
+	worker->subworker = false;
 }
 
 /*
@@ -869,7 +917,7 @@ ApplyLauncherMain(Datum main_arg)
 					wait_time = wal_retrieve_retry_interval;
 
 					logicalrep_worker_launch(sub->dbid, sub->oid, sub->name,
-											 sub->owner, InvalidOid);
+											 sub->owner, InvalidOid, DSM_HANDLE_INVALID);
 				}
 			}
 
diff --git a/src/backend/replication/logical/origin.c b/src/backend/replication/logical/origin.c
index b0c8b6c..c6ea68a 100644
--- a/src/backend/replication/logical/origin.c
+++ b/src/backend/replication/logical/origin.c
@@ -1068,7 +1068,7 @@ ReplicationOriginExitCleanup(int code, Datum arg)
  * with replorigin_session_reset().
  */
 void
-replorigin_session_setup(RepOriginId node)
+replorigin_session_setup(RepOriginId node, bool acquire)
 {
 	static bool registered_cleanup;
 	int			i;
@@ -1110,7 +1110,11 @@ replorigin_session_setup(RepOriginId node)
 		if (curstate->roident != node)
 			continue;
 
-		else if (curstate->acquired_by != 0)
+		/*
+		 * We allow the apply worker to get the slot which is acquired by its
+		 * leader process.
+		 */
+		else if (curstate->acquired_by != 0 && acquire)
 		{
 			ereport(ERROR,
 					(errcode(ERRCODE_OBJECT_IN_USE),
@@ -1321,7 +1325,7 @@ pg_replication_origin_session_setup(PG_FUNCTION_ARGS)
 
 	name = text_to_cstring((text *) DatumGetPointer(PG_GETARG_DATUM(0)));
 	origin = replorigin_by_name(name, false);
-	replorigin_session_setup(origin);
+	replorigin_session_setup(origin, true);
 
 	replorigin_session_origin = origin;
 
diff --git a/src/backend/replication/logical/proto.c b/src/backend/replication/logical/proto.c
index ff8513e..0409fde 100644
--- a/src/backend/replication/logical/proto.c
+++ b/src/backend/replication/logical/proto.c
@@ -1138,14 +1138,11 @@ logicalrep_write_stream_commit(StringInfo out, ReorderBufferTXN *txn,
 /*
  * Read STREAM COMMIT from the output stream.
  */
-TransactionId
+void
 logicalrep_read_stream_commit(StringInfo in, LogicalRepCommitData *commit_data)
 {
-	TransactionId xid;
 	uint8		flags;
 
-	xid = pq_getmsgint(in, 4);
-
 	/* read flags (unused for now) */
 	flags = pq_getmsgbyte(in);
 
@@ -1156,8 +1153,6 @@ logicalrep_read_stream_commit(StringInfo in, LogicalRepCommitData *commit_data)
 	commit_data->commit_lsn = pq_getmsgint64(in);
 	commit_data->end_lsn = pq_getmsgint64(in);
 	commit_data->committime = pq_getmsgint64(in);
-
-	return xid;
 }
 
 /*
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 49ceec3..04c9f96 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -568,7 +568,8 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 												 MySubscription->oid,
 												 MySubscription->name,
 												 MyLogicalRepWorker->userid,
-												 rstate->relid);
+												 rstate->relid,
+												 DSM_HANDLE_INVALID);
 						hentry->last_start_time = now;
 					}
 				}
@@ -589,6 +590,9 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 void
 process_syncing_tables(XLogRecPtr current_lsn)
 {
+	if (MyLogicalRepWorker->subworker)
+		return;
+
 	if (am_tablesync_worker())
 		process_syncing_tables_for_sync(current_lsn);
 	else
@@ -1275,7 +1279,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		 * time this tablesync was launched.
 		 */
 		originid = replorigin_by_name(originname, false);
-		replorigin_session_setup(originid);
+		replorigin_session_setup(originid, true);
 		replorigin_session_origin = originid;
 		*origin_startpos = replorigin_session_get_progress(false);
 
@@ -1386,7 +1390,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						   true /* go backward */ , true /* WAL log */ );
 		UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
 
-		replorigin_session_setup(originid);
+		replorigin_session_setup(originid, true);
 		replorigin_session_origin = originid;
 	}
 	else
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 4171371..dd8e4af 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -22,7 +22,23 @@
  * STREAMED TRANSACTIONS
  * ---------------------
  * Streamed transactions (large transactions exceeding a memory limit on the
- * upstream) are not applied immediately, but instead, the data is written
+ * upstream) are applied via one of two approaches.
+ *
+ * 1) Separate background workers
+ *
+ * Assign a new bgworker (if available) as soon as the xact's first stream came
+ * and the main apply worker will send changes to this new worker via shared
+ * memory. We keep this worker assigned till the transaction commit came and
+ * also wait for the worker to finish at commit. This preserves commit ordering
+ * and avoid writing to and reading from file in most cases. We still need to
+ * spill if there is no worker available. We also need to allow stream_stop to
+ * complete by the background worker to finish it to avoid deadlocks because
+ * T-1's current stream of changes can update rows in conflicting order with
+ * T-2's next stream of changes.
+ *
+ * 2) Write to temporary files and apply when the final commit arrives
+ *
+ * If no worker is available to handle streamed transaction, we write the data
  * to temporary files and then applied at once when the final commit arrives.
  *
  * Unlike the regular (non-streamed) case, handling streamed transactions has
@@ -174,11 +190,15 @@
 #include "rewrite/rewriteHandler.h"
 #include "storage/buffile.h"
 #include "storage/bufmgr.h"
+#include "storage/dsm.h"
 #include "storage/fd.h"
 #include "storage/ipc.h"
 #include "storage/lmgr.h"
 #include "storage/proc.h"
 #include "storage/procarray.h"
+#include "storage/shm_mq.h"
+#include "storage/shm_toc.h"
+#include "storage/spin.h"
 #include "tcop/tcopprot.h"
 #include "utils/acl.h"
 #include "utils/builtins.h"
@@ -198,6 +218,59 @@
 
 #define NAPTIME_PER_CYCLE 1000	/* max sleep time between cycles (1s) */
 
+#define PG_LOGICAL_APPLY_SHM_MAGIC 0x79fb2447 // TODO Consider change
+
+typedef struct ParallelState
+{
+	slock_t	mutex;
+	bool	attached;
+	bool	ready;
+	bool	finished;
+	bool	failed;
+	Oid		subid;
+	TransactionId	stream_xid;
+	uint32	n;
+} ParallelState;
+
+typedef struct WorkerState
+{
+	shm_mq_handle			*mq_handle;
+	dsm_segment				*dsm_seg;
+	ParallelState volatile	*pstate;
+} WorkerState;
+
+typedef struct WorkerEntry
+{
+	TransactionId	xid;
+	WorkerState	   *wstate;
+} WorkerEntry;
+
+/* Apply workers hash table (initialized on first use) */
+static HTAB *ApplyWorkersHash = NULL;
+static List *ApplyWorkersIdleList = NIL;
+static List *ApplyWorkersList = NIL;
+static uint32 nworkers = 0;
+static uint32 nfreeworkers = 0;
+
+/* Fields valid only for apply background workers */
+bool isLogicalApplyWorker = false;
+volatile ParallelState *MyParallelState = NULL;
+static List *subxactlist = NIL;
+
+/* Worker setup and interactions */
+static void setup_dsm(WorkerState *wstate);
+static WorkerState *setup_background_worker(void);
+static void wait_for_worker_ready(WorkerState *wstate, bool notify);
+static void wait_for_transaction_finish(WorkerState *wstate);
+static void send_data_to_worker(WorkerState *wstate, Size nbytes,
+								const void *data);
+static WorkerState *find_or_start_worker(TransactionId xid, bool start);
+static void free_stream_apply_worker(void);
+static bool transaction_applied_in_bgworker(TransactionId xid);
+static void check_workers_status(void);
+
+static uint32 nchanges = 0;
+
 typedef struct FlushPosition
 {
 	dlist_node	node;
@@ -260,18 +333,22 @@ static XLogRecPtr remote_final_lsn = InvalidXLogRecPtr;
 static bool in_streamed_transaction = false;
 
 static TransactionId stream_xid = InvalidTransactionId;
+static WorkerState *stream_apply_worker = NULL;
+
+#define applying_changes_in_bgworker() (in_streamed_transaction && stream_apply_worker != NULL)
 
 /*
  * We enable skipping all data modification changes (INSERT, UPDATE, etc.) for
  * the subscription if the remote transaction's finish LSN matches the subskiplsn.
  * Once we start skipping changes, we don't stop it until we skip all changes of
  * the transaction even if pg_subscription is updated and MySubscription->skiplsn
- * gets changed or reset during that. Also, in streaming transaction cases, we
- * don't skip receiving and spooling the changes since we decide whether or not
+ * gets changed or reset during that. Also, in streaming transaction cases (spool mode),
+ * we don't skip receiving and spooling the changes since we decide whether or not
  * to skip applying the changes when starting to apply changes. The subskiplsn is
  * cleared after successfully skipping the transaction or applying non-empty
  * transaction. The latter prevents the mistakenly specified subskiplsn from
- * being left.
+ * being left. Note that we cannot skip the streaming transaction in apply mode,
+ * because we cannot get the finish LSN before applying the changes.
  */
 static XLogRecPtr skip_xact_finish_lsn = InvalidXLogRecPtr;
 #define is_skipping_changes() (unlikely(!XLogRecPtrIsInvalid(skip_xact_finish_lsn)))
@@ -428,41 +505,96 @@ end_replication_step(void)
 /*
  * Handle streamed transactions.
  *
- * If in streaming mode (receiving a block of streamed transaction), we
- * simply redirect it to a file for the proper toplevel transaction.
+ * For the main apply worker, if in streaming mode (receiving a block of
+ * streamed transaction), we send the data to the background worker.
+ *
+ * For the background worker, define a savepoint if new subtransaction was
+ * started.
  *
  * Returns true for streamed transactions, false otherwise (regular mode).
  */
 static bool
 handle_streamed_transaction(LogicalRepMsgType action, StringInfo s)
 {
-	TransactionId xid;
+	TransactionId current_xid = InvalidTransactionId;
 
 	/* not in streaming mode */
-	if (!in_streamed_transaction)
+	if (!in_streamed_transaction && !isLogicalApplyWorker)
 		return false;
 
-	Assert(stream_fd != NULL);
 	Assert(TransactionIdIsValid(stream_xid));
 
 	/*
 	 * We should have received XID of the subxact as the first part of the
 	 * message, so extract it.
 	 */
-	xid = pq_getmsgint(s, 4);
+	current_xid = pq_getmsgint(s, 4);
 
-	if (!TransactionIdIsValid(xid))
+	if (!TransactionIdIsValid(current_xid))
 		ereport(ERROR,
 				(errcode(ERRCODE_PROTOCOL_VIOLATION),
 				 errmsg_internal("invalid transaction ID in streamed replication transaction")));
 
-	/* Add the new subxact to the array (unless already there). */
-	subxact_info_add(xid);
+	if (isLogicalApplyWorker)
+	{
+		/*
+		 * Inside logical apply worker we can figure out that new
+		 * subtransaction was started if new change arrived with different xid.
+		 * In that case we can define named savepoint, so that we were able to
+		 * commit/rollback it separately later.
+		 *
+		 * Special case is if the first change comes from subtransaction, then
+		 * we check that current_xid differs from stream_xid.
+		 */
+		if (current_xid != stream_xid &&
+			!list_member_int(subxactlist, (int) current_xid))
+		{
+			MemoryContext oldctx;
+			char *spname = (char *) palloc(64 * sizeof(char));
+			sprintf(spname, "savepoint_for_xid_%u", current_xid);
+
+			elog(LOG, "[Apply BGW #%u] defining savepoint %s", MyParallelState->n, spname);
+
+			DefineSavepoint(spname);
+
+			CommitTransactionCommand();
+
+			oldctx = MemoryContextSwitchTo(ApplyContext);
+			subxactlist = lappend_int(subxactlist, (int) current_xid);
+			MemoryContextSwitchTo(oldctx);
+		}
+	}
+	else if (applying_changes_in_bgworker())
+	{
+		/*
+		 * If we decided to apply the changes of this transaction in a
+		 * bgworker, pass the data to the worker.
+		 */
+		send_data_to_worker(stream_apply_worker, s->len, s->data);
+		nchanges += 1;
+
+		/*
+		 * XXX The publisher side don't always send relation update message
+		 * after the streaming transaction, so update the relation in main
+		 * worker here.
+		 */
+		if (action == LOGICAL_REP_MSG_RELATION)
+		{
+			LogicalRepRelation *rel = logicalrep_read_rel(s);
+			logicalrep_relmap_update(rel);
+		}
+
+	}
+	else
+	{
+		/* Add the new subxact to the array (unless already there). */
+		subxact_info_add(current_xid);
 
-	/* write the change to the current file */
-	stream_write_change(action, s);
+		/* write the change to the current file */
+		stream_write_change(action, s);
+	}
 
-	return true;
+	return !isLogicalApplyWorker;
 }
 
 /*
@@ -844,6 +976,9 @@ apply_handle_commit(StringInfo s)
 
 	apply_handle_commit_internal(&commit_data);
 
+	/* Check the status of bgworkers if any. */
+	check_workers_status();
+
 	/* Process any tables that are being synchronized in parallel. */
 	process_syncing_tables(commit_data.end_lsn);
 
@@ -898,7 +1033,9 @@ apply_handle_prepare_internal(LogicalRepPreparedTxnData *prepare_data)
 	 * BeginTransactionBlock is necessary to balance the EndTransactionBlock
 	 * called within the PrepareTransactionBlock below.
 	 */
-	BeginTransactionBlock();
+	if (!IsTransactionBlock())
+		BeginTransactionBlock();
+
 	CommitTransactionCommand(); /* Completes the preceding Begin command. */
 
 	/*
@@ -950,6 +1087,9 @@ apply_handle_prepare(StringInfo s)
 
 	in_remote_transaction = false;
 
+	/* Check the status of bgworkers if any. */
+	check_workers_status();
+
 	/* Process any tables that are being synchronized in parallel. */
 	process_syncing_tables(prepare_data.end_lsn);
 
@@ -976,30 +1116,51 @@ apply_handle_commit_prepared(StringInfo s)
 	char		gid[GIDSIZE];
 
 	logicalrep_read_commit_prepared(s, &prepare_data);
+
 	set_apply_error_context_xact(prepare_data.xid, prepare_data.commit_lsn);
 
-	/* Compute GID for two_phase transactions. */
-	TwoPhaseTransactionGid(MySubscription->oid, prepare_data.xid,
-						   gid, sizeof(gid));
+	/* Check if we have prepared transaction in another bgworker */
+	if (transaction_applied_in_bgworker(prepare_data.xid))
+	{
+		elog(DEBUG1, "received commit for streamed transaction %u", prepare_data.xid);
 
-	/* There is no transaction when COMMIT PREPARED is called */
-	begin_replication_step();
+		/* Send commit message */
+		send_data_to_worker(stream_apply_worker, s->len, s->data);
 
-	/*
-	 * Update origin state so we can restart streaming from correct position
-	 * in case of crash.
-	 */
-	replorigin_session_origin_lsn = prepare_data.end_lsn;
-	replorigin_session_origin_timestamp = prepare_data.commit_time;
+		/* Notify worker, that we are done with this xact */
+		wait_for_transaction_finish(stream_apply_worker);
+
+		free_stream_apply_worker();
+	}
+	else
+	{
+		/* Compute GID for two_phase transactions. */
+		TwoPhaseTransactionGid(MySubscription->oid, prepare_data.xid,
+							   gid, sizeof(gid));
+
+		/* There is no transaction when COMMIT PREPARED is called */
+		begin_replication_step();
+
+		/*
+		 * Update origin state so we can restart streaming from correct position
+		 * in case of crash.
+		 */
+		replorigin_session_origin_lsn = prepare_data.end_lsn;
+		replorigin_session_origin_timestamp = prepare_data.commit_time;
+
+		FinishPreparedTransaction(gid, true);
+		end_replication_step();
+		CommitTransactionCommand();
+	}
 
-	FinishPreparedTransaction(gid, true);
-	end_replication_step();
-	CommitTransactionCommand();
 	pgstat_report_stat(false);
 
 	store_flush_position(prepare_data.end_lsn);
 	in_remote_transaction = false;
 
+	/* Check the status of bgworkers if any. */
+	check_workers_status();
+
 	/* Process any tables that are being synchronized in parallel. */
 	process_syncing_tables(prepare_data.end_lsn);
 
@@ -1019,35 +1180,51 @@ apply_handle_rollback_prepared(StringInfo s)
 	char		gid[GIDSIZE];
 
 	logicalrep_read_rollback_prepared(s, &rollback_data);
+
 	set_apply_error_context_xact(rollback_data.xid, rollback_data.rollback_end_lsn);
 
-	/* Compute GID for two_phase transactions. */
-	TwoPhaseTransactionGid(MySubscription->oid, rollback_data.xid,
-						   gid, sizeof(gid));
+	/* Check if we are processing the prepared transaction in a bgworker */
+	if (transaction_applied_in_bgworker(rollback_data.xid))
+	{
+		send_data_to_worker(stream_apply_worker, s->len, s->data);
 
-	/*
-	 * It is possible that we haven't received prepare because it occurred
-	 * before walsender reached a consistent point or the two_phase was still
-	 * not enabled by that time, so in such cases, we need to skip rollback
-	 * prepared.
-	 */
-	if (LookupGXact(gid, rollback_data.prepare_end_lsn,
-					rollback_data.prepare_time))
+		/* Notify worker, that we are done with this xact */
+		wait_for_transaction_finish(stream_apply_worker);
+
+		elog(LOG, "rollback prepared streaming of xid %u", rollback_data.xid);
+
+		free_stream_apply_worker();
+	}
+	else
 	{
+		/* Compute GID for two_phase transactions. */
+		TwoPhaseTransactionGid(MySubscription->oid, rollback_data.xid,
+							   gid, sizeof(gid));
+
 		/*
-		 * Update origin state so we can restart streaming from correct
-		 * position in case of crash.
+		 * It is possible that we haven't received prepare because it occurred
+		 * before walsender reached a consistent point or the two_phase was still
+		 * not enabled by that time, so in such cases, we need to skip rollback
+		 * prepared.
 		 */
-		replorigin_session_origin_lsn = rollback_data.rollback_end_lsn;
-		replorigin_session_origin_timestamp = rollback_data.rollback_time;
+		if (LookupGXact(gid, rollback_data.prepare_end_lsn,
+						rollback_data.prepare_time))
+		{
+			/*
+			 * Update origin state so we can restart streaming from correct
+			 * position in case of crash.
+			 */
+			replorigin_session_origin_lsn = rollback_data.rollback_end_lsn;
+			replorigin_session_origin_timestamp = rollback_data.rollback_time;
 
-		/* There is no transaction when ABORT/ROLLBACK PREPARED is called */
-		begin_replication_step();
-		FinishPreparedTransaction(gid, false);
-		end_replication_step();
-		CommitTransactionCommand();
+			/* There is no transaction when ABORT/ROLLBACK PREPARED is called */
+			begin_replication_step();
+			FinishPreparedTransaction(gid, false);
+			end_replication_step();
+			CommitTransactionCommand();
 
-		clear_subscription_skip_lsn(rollback_data.rollback_end_lsn);
+			clear_subscription_skip_lsn(rollback_data.rollback_end_lsn);
+		}
 	}
 
 	pgstat_report_stat(false);
@@ -1055,6 +1232,9 @@ apply_handle_rollback_prepared(StringInfo s)
 	store_flush_position(rollback_data.rollback_end_lsn);
 	in_remote_transaction = false;
 
+	/* Check the status of bgworkers if any. */
+	check_workers_status();
+
 	/* Process any tables that are being synchronized in parallel. */
 	process_syncing_tables(rollback_data.rollback_end_lsn);
 
@@ -1063,11 +1243,148 @@ apply_handle_rollback_prepared(StringInfo s)
 }
 
 /*
- * Handle STREAM PREPARE.
+ * Look up worker inside ApplyWorkersHash for requested xid.
  *
- * Logic is in two parts:
- * 1. Replay all the spooled operations
- * 2. Mark the transaction as prepared
+ * If start flag is true, try to start a new worker if not found in hash table.
+ */
+static WorkerState *
+find_or_start_worker(TransactionId xid, bool start)
+{
+	bool found;
+	WorkerState *wstate;
+	WorkerEntry *entry = NULL;
+
+	/*
+	 * We don't start new background worker when stream option is off or spool.
+	 */
+	if (MySubscription->stream != SUBSTREAM_APPLY)
+		return NULL;
+
+	/*
+	 * We don't start new background worker if user has set skiplsn as it's
+	 * possible that user want to skip the streaming transaction. For streaming
+	 * transaction, we need to spill the transaction to disk so that we can get
+	 * the last LSN of the transaction to judge whether to skip before starting
+	 * to apply the change.
+	 */
+	else if (start && !XLogRecPtrIsInvalid(MySubscription->skiplsn))
+		return NULL;
+
+	/*
+	 * For streaming transactions that is being applied in bgworker, we cannot
+	 * decide whether to apply the change for a relation that is not in the
+	 * READY state (see should_apply_changes_for_rel) as we won't know
+	 * remote_final_lsn by that time. So, we don't start new bgworker in this
+	 * case.
+	 */
+	else if (start && !AllTablesyncsReady())
+		return NULL;
+	else if (!start && ApplyWorkersHash == NULL)
+		return NULL;
+
+	/* First time through, initialize apply workers hashtable */
+	if (ApplyWorkersHash == NULL)
+	{
+		HASHCTL		ctl;
+
+		MemSet(&ctl, 0, sizeof(ctl));
+		ctl.keysize = sizeof(TransactionId);
+		ctl.entrysize = sizeof(WorkerEntry);
+		ctl.hcxt = ApplyContext;
+
+		ApplyWorkersHash = hash_create("logical apply workers hash", 8, &ctl,
+									   HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
+	}
+
+	/*
+	 * Find entry for requested transaction.
+	 */
+	entry = hash_search(ApplyWorkersHash, &xid, start ? HASH_ENTER : HASH_FIND,
+						&found);
+	if (found)
+		return entry->wstate;
+	else if (!start)
+		return NULL;
+
+	/* If there is at least one worker in the idle list, then take one. */
+	if (list_length(ApplyWorkersIdleList) > 0)
+	{
+		wstate = (WorkerState *) llast(ApplyWorkersIdleList);
+		ApplyWorkersIdleList = list_delete_last(ApplyWorkersIdleList);
+	}
+	else
+	{
+		wstate = setup_background_worker();
+
+		if (wstate == NULL)
+		{
+			/*
+			 * If there is no more worker can be launched here, remove the
+			 * entry in hash table.
+			 */
+			hash_search(ApplyWorkersHash, &xid, HASH_REMOVE, &found);
+			return NULL;
+		}
+	}
+
+	/* Fill up the hash entry */
+	wstate->pstate->finished = false;
+	wstate->pstate->stream_xid = xid;
+	entry->wstate = wstate;
+	entry->xid = xid;
+
+	return wstate;
+}
+
+/*
+ * Add the worker to the freelist and remove the entry from hash table.
+ */
+static void
+free_stream_apply_worker(void)
+{
+	bool found;
+	MemoryContext oldctx;
+	TransactionId xid = stream_apply_worker->pstate->stream_xid;
+
+	Assert(stream_apply_worker);
+
+	oldctx = MemoryContextSwitchTo(ApplyContext);
+
+	hash_search(ApplyWorkersHash, &xid,
+				HASH_REMOVE, &found);
+
+	elog(LOG, "adding finished apply worker #%u for xid %u to the idle list",
+		 stream_apply_worker->pstate->n, stream_apply_worker->pstate->stream_xid);
+
+	ApplyWorkersIdleList = lappend(ApplyWorkersIdleList, stream_apply_worker);
+	stream_apply_worker = NULL;
+
+	MemoryContextSwitchTo(oldctx);
+}
+
+static void
+serialize_stream_prepare(LogicalRepPreparedTxnData *prepare_data)
+{
+	/* Replay all the spooled operations. */
+	apply_spooled_messages(prepare_data->xid, prepare_data->prepare_lsn);
+
+	/* Mark the transaction as prepared. */
+	apply_handle_prepare_internal(prepare_data);
+
+	CommitTransactionCommand();
+
+	pgstat_report_stat(false);
+
+	store_flush_position(prepare_data->end_lsn);
+
+	in_remote_transaction = false;
+
+	/* unlink the files with serialized changes and subxact info. */
+	stream_cleanup_files(MyLogicalRepWorker->subid, prepare_data->xid);
+}
+
+/*
+ * Handle STREAM PREPARE.
  */
 static void
 apply_handle_stream_prepare(StringInfo s)
@@ -1088,24 +1405,49 @@ apply_handle_stream_prepare(StringInfo s)
 	logicalrep_read_stream_prepare(s, &prepare_data);
 	set_apply_error_context_xact(prepare_data.xid, prepare_data.prepare_lsn);
 
-	elog(DEBUG1, "received prepare for streamed transaction %u", prepare_data.xid);
+	/*
+	 * If we are in a bgworker, just prepare the transaction.
+	 */
+	if (isLogicalApplyWorker)
+	{
+		elog(LOG, "received prepare for streamed transaction %u", prepare_data.xid);
 
-	/* Replay all the spooled operations. */
-	apply_spooled_messages(prepare_data.xid, prepare_data.prepare_lsn);
+		/* Mark the transaction as prepared. */
+		apply_handle_prepare_internal(&prepare_data);
 
-	/* Mark the transaction as prepared. */
-	apply_handle_prepare_internal(&prepare_data);
+		CommitTransactionCommand();
 
-	CommitTransactionCommand();
+		pgstat_report_stat(false);
 
-	pgstat_report_stat(false);
+		list_free(subxactlist);
+		subxactlist = NIL;
+	}
 
-	store_flush_position(prepare_data.end_lsn);
+	/*
+	 * If we are in main apply worker, check if we are processing this
+	 * transaction in a bgworker.
+	 */
+	else if (transaction_applied_in_bgworker(prepare_data.xid))
+	{
+		send_data_to_worker(stream_apply_worker, s->len, s->data);
+		wait_for_worker_ready(stream_apply_worker, true);
+
+		pgstat_report_stat(false);
+		store_flush_position(prepare_data.end_lsn);
+	}
+
+	/*
+	 * If we are in main apply worker and the transaction has been serialized
+	 * to file, replay all the spooled operations.
+	 */
+	else
+		serialize_stream_prepare(&prepare_data);
 
 	in_remote_transaction = false;
+	stream_apply_worker = NULL;
 
-	/* unlink the files with serialized changes and subxact info. */
-	stream_cleanup_files(MyLogicalRepWorker->subid, prepare_data.xid);
+	/* Check the status of bgworkers if any. */
+	check_workers_status();
 
 	/* Process any tables that are being synchronized in parallel. */
 	process_syncing_tables(prepare_data.end_lsn);
@@ -1142,19 +1484,9 @@ apply_handle_origin(StringInfo s)
 				 errmsg_internal("ORIGIN message sent out of order")));
 }
 
-/*
- * Handle STREAM START message.
- */
 static void
-apply_handle_stream_start(StringInfo s)
+serialize_stream_start(bool first_segment)
 {
-	bool		first_segment;
-
-	if (in_streamed_transaction)
-		ereport(ERROR,
-				(errcode(ERRCODE_PROTOCOL_VIOLATION),
-				 errmsg_internal("duplicate STREAM START message")));
-
 	/*
 	 * Start a transaction on stream start, this transaction will be committed
 	 * on the stream stop unless it is a tablesync worker in which case it
@@ -1164,19 +1496,6 @@ apply_handle_stream_start(StringInfo s)
 	 */
 	begin_replication_step();
 
-	/* notify handle methods we're processing a remote transaction */
-	in_streamed_transaction = true;
-
-	/* extract XID of the top-level transaction */
-	stream_xid = logicalrep_read_stream_start(s, &first_segment);
-
-	if (!TransactionIdIsValid(stream_xid))
-		ereport(ERROR,
-				(errcode(ERRCODE_PROTOCOL_VIOLATION),
-				 errmsg_internal("invalid transaction ID in streamed replication transaction")));
-
-	set_apply_error_context_xact(stream_xid, InvalidXLogRecPtr);
-
 	/*
 	 * Initialize the worker's stream_fileset if we haven't yet. This will be
 	 * used for the entire duration of the worker so create it in a permanent
@@ -1204,60 +1523,132 @@ apply_handle_stream_start(StringInfo s)
 	if (!first_segment)
 		subxact_info_read(MyLogicalRepWorker->subid, stream_xid);
 
-	pgstat_report_activity(STATE_RUNNING, NULL);
-
 	end_replication_step();
 }
 
 /*
- * Handle STREAM STOP message.
+ * Handle STREAM START message.
  */
 static void
-apply_handle_stream_stop(StringInfo s)
+apply_handle_stream_start(StringInfo s)
 {
-	if (!in_streamed_transaction)
+	bool		first_segment;
+
+	if (in_streamed_transaction)
 		ereport(ERROR,
 				(errcode(ERRCODE_PROTOCOL_VIOLATION),
-				 errmsg_internal("STREAM STOP message without STREAM START")));
-
-	/*
-	 * Close the file with serialized changes, and serialize information about
-	 * subxacts for the toplevel transaction.
-	 */
-	subxact_info_write(MyLogicalRepWorker->subid, stream_xid);
-	stream_close_file();
-
-	/* We must be in a valid transaction state */
-	Assert(IsTransactionState());
-
-	/* Commit the per-stream transaction */
-	CommitTransactionCommand();
-
-	in_streamed_transaction = false;
-
-	/* Reset per-stream context */
-	MemoryContextReset(LogicalStreamingContext);
+				 errmsg_internal("duplicate STREAM START message")));
 
-	pgstat_report_activity(STATE_IDLE, NULL);
-	reset_apply_error_context_info();
-}
+	/* notify handle methods we're processing a remote transaction */
+	in_streamed_transaction = true;
 
-/*
- * Handle STREAM abort message.
- */
-static void
-apply_handle_stream_abort(StringInfo s)
-{
-	TransactionId xid;
-	TransactionId subxid;
+	/* extract XID of the top-level transaction */
+	stream_xid = logicalrep_read_stream_start(s, &first_segment);
 
-	if (in_streamed_transaction)
+	if (!TransactionIdIsValid(stream_xid))
 		ereport(ERROR,
 				(errcode(ERRCODE_PROTOCOL_VIOLATION),
-				 errmsg_internal("STREAM ABORT message without STREAM STOP")));
+				 errmsg_internal("invalid transaction ID in streamed replication transaction")));
 
-	logicalrep_read_stream_abort(s, &xid, &subxid);
+	set_apply_error_context_xact(stream_xid, InvalidXLogRecPtr);
 
+	if (isLogicalApplyWorker)
+	{
+		/* If we are in a bgworker, begin the transaction */
+		AcceptInvalidationMessages();
+		maybe_reread_subscription();
+
+		StartTransactionCommand();
+		BeginTransactionBlock();
+		CommitTransactionCommand();
+
+		return;
+	}
+
+	/*
+	 * If we are in main apply worker, check if there is any free bgworker
+	 * we can use to process this transaction.
+	 */
+	stream_apply_worker = find_or_start_worker(stream_xid, first_segment);
+
+	if (applying_changes_in_bgworker())
+	{
+		/*
+		 * If we have free worker or we already started to apply this
+		 * transaction in bgworker, we pass the data to worker.
+		 */
+		if (first_segment)
+			send_data_to_worker(stream_apply_worker, s->len, s->data);
+
+		nchanges = 0;
+
+		SpinLockAcquire(&stream_apply_worker->pstate->mutex);
+		stream_apply_worker->pstate->ready = false;
+		SpinLockRelease(&stream_apply_worker->pstate->mutex);
+
+		elog(LOG, "starting streaming of xid %u", stream_xid);
+	}
+
+	/*
+	 * If no worker is available for the first stream start, we start to
+	 * serialize all the changes of the transaction.
+	 */
+	else
+		serialize_stream_start(first_segment);
+
+	pgstat_report_activity(STATE_RUNNING, NULL);
+}
+
+static void
+serialize_stream_stop()
+{
+	/*
+	 * Close the file with serialized changes, and serialize information about
+	 * subxacts for the toplevel transaction.
+	 */
+	subxact_info_write(MyLogicalRepWorker->subid, stream_xid);
+	stream_close_file();
+
+	/* We must be in a valid transaction state */
+	Assert(IsTransactionState());
+
+	/* Commit the per-stream transaction */
+	CommitTransactionCommand();
+
+	/* Reset per-stream context */
+	MemoryContextReset(LogicalStreamingContext);
+}
+
+/*
+ * Handle STREAM STOP message.
+ */
+static void
+apply_handle_stream_stop(StringInfo s)
+{
+	if (!in_streamed_transaction)
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg_internal("STREAM STOP message without STREAM START")));
+
+	if (applying_changes_in_bgworker())
+	{
+		wait_for_worker_ready(stream_apply_worker, true);
+
+		elog(LOG, "stopped streaming of xid %u, %u changes streamed", stream_xid, nchanges);
+	}
+	else
+		serialize_stream_stop();
+
+	in_streamed_transaction = false;
+	stream_apply_worker = NULL;
+
+	pgstat_report_activity(STATE_IDLE, NULL);
+	reset_apply_error_context_info();
+}
+
+static void
+serialize_stream_abort(TransactionId xid, TransactionId subxid)
+{
 	/*
 	 * If the two XIDs are the same, it's in fact abort of toplevel xact, so
 	 * just delete the files with serialized info.
@@ -1339,8 +1730,114 @@ apply_handle_stream_abort(StringInfo s)
 		end_replication_step();
 		CommitTransactionCommand();
 	}
+}
 
-	reset_apply_error_context_info();
+/*
+ * Handle STREAM ABORT message.
+ */
+static void
+apply_handle_stream_abort(StringInfo s)
+{
+	TransactionId xid;
+	TransactionId subxid;
+
+	if (in_streamed_transaction)
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg_internal("STREAM COMMIT message without STREAM STOP")));
+
+	logicalrep_read_stream_abort(s, &xid, &subxid);
+
+	if (isLogicalApplyWorker)
+	{
+		ereport(LOG,
+				(errcode_for_file_access(),
+				errmsg("[Apply BGW #%u] aborting current transaction xid=%u, subxid=%u",
+				MyParallelState->n, GetCurrentTransactionIdIfAny(), GetCurrentSubTransactionId())));
+
+		/*
+		 * If the two XIDs are the same, it's in fact abort of toplevel xact,
+		 * so just free the subxactlist.
+		 */
+		if (subxid == xid)
+		{
+			set_apply_error_context_xact(subxid, InvalidXLogRecPtr);
+
+			AbortCurrentTransaction();
+
+			EndTransactionBlock(false);
+			CommitTransactionCommand();
+
+			list_free(subxactlist);
+			subxactlist = NIL;
+		}
+		else
+		{
+			/*
+			 * OK, so it's a subxact. Rollback to the savepoint.
+			 *
+			 * We also need to read the subxactlist, determine the offset
+			 * tracked for the subxact, and truncate the list.
+			 */
+			int		i;
+			bool	found = false;
+			char *spname = (char *) palloc(64 * sizeof(char));
+
+			set_apply_error_context_xact(subxid, InvalidXLogRecPtr);
+
+			sprintf(spname, "savepoint_for_xid_%u", subxid);
+
+			ereport(LOG,
+					(errcode_for_file_access(),
+					errmsg("[Apply BGW #%u] rolling back to savepoint %s", MyParallelState->n, spname)));
+
+			for(i = list_length(subxactlist) - 1; i >= 0; i--)
+			{
+				xid = (TransactionId) list_nth_int(subxactlist, i);
+				if (xid == subxid)
+				{
+					found = true;
+					break;
+				}
+			}
+
+			if (found)
+			{
+				elog(LOG, "rolled back to savepoint %s", spname);
+				RollbackToSavepoint(spname);
+				CommitTransactionCommand();
+				subxactlist = list_truncate(subxactlist, i + 1);
+			}
+
+			pfree(spname);
+		}
+
+		reset_apply_error_context_info();
+	}
+
+	/*
+	 * If we are in main apply worker, check if we are processing this
+	 * transaction in a bgworker.
+	 */
+	else if (transaction_applied_in_bgworker(xid))
+	{
+		send_data_to_worker(stream_apply_worker, s->len, s->data);
+
+		if (subxid == xid)
+		{
+			wait_for_transaction_finish(stream_apply_worker);
+			free_stream_apply_worker();
+		}
+		else
+			wait_for_worker_ready(stream_apply_worker, true);
+	}
+
+	/*
+	 * We are in main apply worker and the transaction has been serialized
+	 * to file.
+	 */
+	else
+		serialize_stream_abort(xid, subxid);
 }
 
 /*
@@ -1463,40 +1960,6 @@ apply_spooled_messages(TransactionId xid, XLogRecPtr lsn)
 }
 
 /*
- * Handle STREAM COMMIT message.
- */
-static void
-apply_handle_stream_commit(StringInfo s)
-{
-	TransactionId xid;
-	LogicalRepCommitData commit_data;
-
-	if (in_streamed_transaction)
-		ereport(ERROR,
-				(errcode(ERRCODE_PROTOCOL_VIOLATION),
-				 errmsg_internal("STREAM COMMIT message without STREAM STOP")));
-
-	xid = logicalrep_read_stream_commit(s, &commit_data);
-	set_apply_error_context_xact(xid, commit_data.commit_lsn);
-
-	elog(DEBUG1, "received commit for streamed transaction %u", xid);
-
-	apply_spooled_messages(xid, commit_data.commit_lsn);
-
-	apply_handle_commit_internal(&commit_data);
-
-	/* unlink the files with serialized changes and subxact info */
-	stream_cleanup_files(MyLogicalRepWorker->subid, xid);
-
-	/* Process any tables that are being synchronized in parallel. */
-	process_syncing_tables(commit_data.end_lsn);
-
-	pgstat_report_activity(STATE_IDLE, NULL);
-
-	reset_apply_error_context_info();
-}
-
-/*
  * Helper function for apply_handle_commit and apply_handle_stream_commit.
  */
 static void
@@ -2445,6 +2908,93 @@ apply_handle_truncate(StringInfo s)
 	end_replication_step();
 }
 
+/*
+ * Handle STREAM COMMIT message.
+ */
+static void
+apply_handle_stream_commit(StringInfo s)
+{
+	LogicalRepCommitData commit_data;
+	TransactionId xid;
+
+	if (in_streamed_transaction)
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg_internal("STREAM COMMIT message without STREAM STOP")));
+
+	xid = pq_getmsgint(s, 4);
+	logicalrep_read_stream_commit(s, &commit_data);
+	set_apply_error_context_xact(xid, commit_data.commit_lsn);
+
+	elog(DEBUG1, "received commit for streamed transaction %u", xid);
+
+	if (isLogicalApplyWorker)
+	{
+		/*
+		 * Update origin state so we can restart streaming from correct
+		 * position in case of crash.
+		 */
+		replorigin_session_origin_lsn = commit_data.end_lsn;
+		replorigin_session_origin_timestamp = commit_data.committime;
+
+		CommitTransactionCommand();
+		EndTransactionBlock(false);
+		CommitTransactionCommand();
+
+		pgstat_report_stat(false);
+
+		list_free(subxactlist);
+		subxactlist = NIL;
+	}
+
+	/*
+	 * If we are in main apply worker, check if we are processing this
+	 * transaction in a bgworker.
+	 */
+	else if (transaction_applied_in_bgworker(xid))
+	{
+		/* Send commit message */
+		send_data_to_worker(stream_apply_worker, s->len, s->data);
+
+		/* Notify worker, that we are done with this xact */
+		wait_for_transaction_finish(stream_apply_worker);
+
+		pgstat_report_stat(false);
+		store_flush_position(commit_data.end_lsn);
+		stop_skipping_changes();
+
+		free_stream_apply_worker();
+
+		/*
+		 * The transaction is either non-empty or skipped, so we clear the
+		 * subskiplsn.
+		 */
+		clear_subscription_skip_lsn(commit_data.commit_lsn);
+	}
+	else
+	{
+		/*
+		 * If we are in main apply worker and the transaction has been
+		 * serialized to file, replay all the spooled operations.
+		 */
+		apply_spooled_messages(xid, commit_data.commit_lsn);
+
+		apply_handle_commit_internal(&commit_data);
+
+		/* unlink the files with serialized changes and subxact info */
+		stream_cleanup_files(MyLogicalRepWorker->subid, xid);
+	}
+
+	/* Check the status of bgworkers if any. */
+	check_workers_status();
+
+	/* Process any tables that are being synchronized in parallel. */
+	process_syncing_tables(commit_data.end_lsn);
+
+	pgstat_report_activity(STATE_IDLE, NULL);
+
+	reset_apply_error_context_info();
+}
 
 /*
  * Logical replication protocol message dispatcher.
@@ -2511,6 +3061,7 @@ apply_dispatch(StringInfo s)
 			break;
 
 		case LOGICAL_REP_MSG_STREAM_START:
+			elog(LOG, "LOGICAL_REP_MSG_STREAM_START");
 			apply_handle_stream_start(s);
 			break;
 
@@ -2618,6 +3169,10 @@ store_flush_position(XLogRecPtr remote_lsn)
 {
 	FlushPosition *flushpos;
 
+	/* We only need to collect the LSN in main apply worker */
+	if (isLogicalApplyWorker)
+		return;
+
 	/* Need to do this in permanent context */
 	MemoryContextSwitchTo(ApplyContext);
 
@@ -2794,6 +3349,9 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 			AcceptInvalidationMessages();
 			maybe_reread_subscription();
 
+			/* Check the status of bgworkers if any. */
+			check_workers_status();
+
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
 		}
@@ -3467,6 +4025,7 @@ TwoPhaseTransactionGid(Oid subid, TransactionId xid, char *gid, int szgid)
 	snprintf(gid, szgid, "pg_gid_%u_%u", subid, xid);
 }
 
+
 /*
  * Execute the initial sync with error handling. Disable the subscription,
  * if it's required.
@@ -3686,7 +4245,7 @@ ApplyWorkerMain(Datum main_arg)
 		originid = replorigin_by_name(originname, true);
 		if (!OidIsValid(originid))
 			originid = replorigin_create(originname);
-		replorigin_session_setup(originid);
+		replorigin_session_setup(originid, true);
 		replorigin_session_origin = originid;
 		origin_startpos = replorigin_session_get_progress(false);
 		CommitTransactionCommand();
@@ -3733,7 +4292,7 @@ ApplyWorkerMain(Datum main_arg)
 
 	options.proto.logical.publication_names = MySubscription->publications;
 	options.proto.logical.binary = MySubscription->binary;
-	options.proto.logical.streaming = MySubscription->stream;
+	options.proto.logical.streaming = (MySubscription->stream != SUBSTREAM_OFF);
 	options.proto.logical.twophase = false;
 
 	if (!am_tablesync_worker())
@@ -3891,7 +4450,8 @@ clear_subscription_skip_lsn(XLogRecPtr finish_lsn)
 	XLogRecPtr	myskiplsn = MySubscription->skiplsn;
 	bool		started_tx = false;
 
-	if (likely(XLogRecPtrIsInvalid(myskiplsn)))
+	if (likely(XLogRecPtrIsInvalid(myskiplsn)) ||
+		isLogicalApplyWorker)
 		return;
 
 	if (!IsTransactionState())
@@ -4027,3 +4587,643 @@ reset_apply_error_context_info(void)
 	apply_error_callback_arg.remote_attnum = -1;
 	set_apply_error_context_xact(InvalidTransactionId, InvalidXLogRecPtr);
 }
+
+/* Apply Background Worker main loop */
+static void
+LogicalApplyBgwLoop(shm_mq_handle *mqh, volatile ParallelState *pst)
+{
+	shm_mq_result		shmq_res;
+	PGPROC				*registrant;
+	ErrorContextCallback errcallback;
+	XLogRecPtr			last_received = InvalidXLogRecPtr;
+
+	registrant = BackendPidGetProc(MyBgworkerEntry->bgw_notify_pid);
+	SetLatch(&registrant->procLatch);
+
+	/*
+	 * Push apply error context callback. Fields will be filled during
+	 * applying a change.
+	 */
+	errcallback.callback = apply_error_callback;
+	errcallback.previous = error_context_stack;
+	error_context_stack = &errcallback;
+
+	for (;;)
+	{
+		void *data;
+		Size  len;
+		StringInfoData s;
+		MemoryContext	oldctx;
+
+		CHECK_FOR_INTERRUPTS();
+
+		/* Ensure we are reading the data into our memory context. */
+		oldctx = MemoryContextSwitchTo(ApplyMessageContext);
+
+		shmq_res = shm_mq_receive(mqh, &len, &data, false);
+
+		if (shmq_res != SHM_MQ_SUCCESS)
+			break;
+
+		if (len == 0)
+		{
+			elog(LOG, "[Apply BGW #%u] got zero-length message, stopping", pst->n);
+			break;
+		}
+		else
+		{
+			XLogRecPtr	start_lsn;
+			XLogRecPtr	end_lsn;
+			TimestampTz send_time;
+
+			s.cursor = 0;
+			s.maxlen = -1;
+			s.data = (char *) data;
+			s.len = len;
+
+			/*
+			 * We use first byte of message for additional communication between
+			 * main Logical replication worker and Apply BGWorkers, so if it
+			 * differs from 'w', then process it first.
+			 */
+			switch (pq_getmsgbyte(&s))
+			{
+				/* Stream stop */
+				case 'E':
+				{
+					elog(LOG, "[Apply BGW #%u] ended processing streaming chunk,"
+							  "waiting on shm_mq_receive", pst->n);
+
+					SpinLockAcquire(&pst->mutex);
+					pst->ready = true;
+					SpinLockRelease(&pst->mutex);
+
+					SetLatch(&registrant->procLatch);
+
+					in_streamed_transaction = false;
+
+					pgstat_report_activity(STATE_IDLE, NULL);
+
+					continue;
+				}
+				/* Finished processing xact */
+				case 'F':
+				{
+					elog(LOG, "[Apply BGW #%u] finished processing xact %u", pst->n, stream_xid);
+
+					SpinLockAcquire(&pst->mutex);
+					pst->finished = true;
+					SpinLockRelease(&pst->mutex);
+
+					in_remote_transaction = false;
+
+					continue;
+				}
+				default:
+					break;
+			}
+
+			start_lsn = pq_getmsgint64(&s);
+			end_lsn = pq_getmsgint64(&s);
+			send_time = pq_getmsgint64(&s);
+
+			if (last_received < start_lsn)
+				last_received = start_lsn;
+
+			if (last_received < end_lsn)
+				last_received = end_lsn;
+
+			/*
+			 * TO IMPROVE: Do we need to display the bgworker's information in
+			 * pg_stat_replication ?
+			 */
+			UpdateWorkerStats(last_received, send_time, false);
+
+			/*
+			 * Make sure the handle apply_dispatch methods are aware we're in a remote
+			 * transaction.
+			 */
+			in_remote_transaction = true;
+
+			apply_dispatch(&s);
+
+			if (ConfigReloadPending)
+			{
+				ConfigReloadPending = false;
+				ProcessConfigFile(PGC_SIGHUP);
+			}
+		}
+
+		MemoryContextSwitchTo(oldctx);
+		MemoryContextReset(ApplyMessageContext);
+	}
+
+	MemoryContextSwitchTo(TopMemoryContext);
+	MemoryContextReset(ApplyContext);
+
+	/* Pop the error context stack */
+	error_context_stack = errcallback.previous;
+
+	elog(LOG, "[Apply BGW #%u] exiting", pst->n);
+
+	/* Signal main process that we are done. */
+	SetLatch(&registrant->procLatch);
+}
+
+/*
+ * Set the failed flag so that the main apply worker can realize we have
+ * shutdown.
+ */
+static void
+ApplyBgwShutdown(int code, Datum arg)
+{
+	SpinLockAcquire(&MyParallelState->mutex);
+	MyParallelState->failed = true;
+	SpinLockRelease(&MyParallelState->mutex);
+
+	dsm_detach((dsm_segment *) DatumGetPointer(arg));
+}
+
+/*
+ * Apply Background Worker entry point
+ */
+void
+LogicalApplyBgwMain(Datum main_arg)
+{
+	volatile ParallelState *pst;
+
+	dsm_handle			handle;
+	dsm_segment			*seg;
+	shm_toc				*toc;
+	shm_mq				*mq;
+	shm_mq_handle		*mqh;
+	MemoryContext		 oldcontext;
+	RepOriginId			originid;
+	int			worker_slot = DatumGetInt32(main_arg);
+	char		originname[NAMEDATALEN];
+
+	/* Attach to slot */
+	logicalrep_worker_attach(worker_slot);
+
+	MemoryContextSwitchTo(TopMemoryContext);
+
+	/* Load the subscription into persistent memory context. */
+	ApplyContext = AllocSetContextCreate(TopMemoryContext,
+										 "ApplyContext",
+										 ALLOCSET_DEFAULT_SIZES);
+
+	/*
+	 * Init the ApplyMessageContext which we clean up after each replication
+	 * protocol message.
+	 */
+	ApplyMessageContext = AllocSetContextCreate(ApplyContext,
+												"ApplyMessageContext",
+												ALLOCSET_DEFAULT_SIZES);
+
+	isLogicalApplyWorker = true;
+
+	/* Setup signal handling */
+	pqsignal(SIGHUP, SignalHandlerForConfigReload);
+	pqsignal(SIGTERM, die);
+	BackgroundWorkerUnblockSignals();
+
+	/*
+	 * Connect to the dynamic shared memory segment.
+	 *
+	 * The backend that registered this worker passed us the ID of a shared
+	 * memory segment to which we must attach for further instructions.  In
+	 * order to attach to dynamic shared memory, we need a resource owner.
+	 * Once we've mapped the segment in our address space, attach to the table
+	 * of contents so we can locate the various data structures we'll need to
+	 * find within the segment.
+	 */
+	CurrentResourceOwner = ResourceOwnerCreate(NULL, "Logical apply worker");
+	memcpy(&handle, MyBgworkerEntry->bgw_extra, sizeof(dsm_handle));
+	seg = dsm_attach(handle);
+	if (seg == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+				 errmsg("unable to map dynamic shared memory segment")));
+	toc = shm_toc_attach(PG_LOGICAL_APPLY_SHM_MAGIC, dsm_segment_address(seg));
+	if (toc == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+				 errmsg("bad magic number in dynamic shared memory segment")));
+
+	before_shmem_exit(ApplyBgwShutdown, PointerGetDatum(seg));
+
+	/*
+	 * Acquire a worker number.
+	 *
+	 * By convention, the process registering this background worker should
+	 * have stored the control structure at key 0.  We look up that key to
+	 * find it.  Our worker number gives our identity: there may be just one
+	 * worker involved in this parallel operation, or there may be many.
+	 */
+	pst = shm_toc_lookup(toc, 0, false);
+	MyParallelState = pst;
+
+	SpinLockAcquire(&pst->mutex);
+	pst->attached = true;
+	SpinLockRelease(&pst->mutex);
+
+	/*
+	 * Attach to the message queue.
+	 */
+	mq = shm_toc_lookup(toc, 1, false);
+	shm_mq_set_receiver(mq, MyProc);
+	mqh = shm_mq_attach(mq, seg, NULL);
+
+	/* Connect to our database. */
+	BackgroundWorkerInitializeConnectionByOid(MyLogicalRepWorker->dbid,
+											  MyLogicalRepWorker->userid,
+											  0);
+
+	/*
+	 * Set the client encoding to the database encoding, since that is what
+	 * the leader will expect.
+	 */
+	SetClientEncoding(GetDatabaseEncoding());
+
+	stream_xid = pst->stream_xid;
+
+	StartTransactionCommand();
+	oldcontext = MemoryContextSwitchTo(ApplyContext);
+
+	MySubscription = GetSubscription(MyLogicalRepWorker->subid, true);
+	if (!MySubscription)
+	{
+		ereport(LOG,
+				(errmsg("logical replication apply worker for subscription %u will not "
+						"start because the subscription was removed during startup",
+						MyLogicalRepWorker->subid)));
+		proc_exit(0);
+	}
+
+	MySubscriptionValid = true;
+	MemoryContextSwitchTo(oldcontext);
+
+	/* Setup synchronous commit according to the user's wishes */
+	SetConfigOption("synchronous_commit", MySubscription->synccommit,
+					PGC_BACKEND, PGC_S_OVERRIDE);
+
+	/* Keep us informed about subscription changes. */
+	CacheRegisterSyscacheCallback(SUBSCRIPTIONOID,
+								  subscription_change_cb,
+								  (Datum) 0);
+
+	CommitTransactionCommand();
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	snprintf(originname, sizeof(originname), "pg_%u", MySubscription->oid);
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, false);
+	replorigin_session_origin = originid;
+	CommitTransactionCommand();
+
+	/*
+	 * Allocate the origin name in long-lived context for error context
+	 * message.
+	 */
+	apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
+															   originname);
+
+	/*
+	 * Indicate that we're fully initialized and ready to begin the main part
+	 * of the parallel operation.
+	 *
+	 * Once we signal that we're ready, the user backend is entitled to assume
+	 * that our on_dsm_detach callbacks will fire before we disconnect from
+	 * the shared memory segment and exit.  Generally, that means we must have
+	 * attached to all relevant dynamic shared memory data structures by now.
+	 */
+	SpinLockAcquire(&pst->mutex);
+	pst->ready = true;
+	SpinLockRelease(&pst->mutex);
+
+	elog(LOG, "[Apply BGW #%u] started", pst->n);
+
+	PG_TRY();
+	{
+		LogicalApplyBgwLoop(mqh, pst);
+	}
+	PG_CATCH();
+	{
+		/*
+		 * Report the worker failed while applying streaming transaction
+		 * changes. Abort the current transaction so that the stats message is
+		 * sent in an idle state.
+		 */
+		AbortOutOfAnyTransaction();
+		pgstat_report_subscription_error(MySubscription->oid, false);
+
+		PG_RE_THROW();
+	}
+	PG_END_TRY();
+
+	/*
+	 * We're done.  Explicitly detach the shared memory segment so that we
+	 * don't get a resource leak warning at commit time.  This will fire any
+	 * on_dsm_detach callbacks we've registered, as well.  Once that's done,
+	 * we can go ahead and exit.
+	 */
+	dsm_detach(seg);
+	proc_exit(0);
+}
+
+/*
+ * Set up a dynamic shared memory segment.
+ *
+ * We set up a control region that contains a ParallelState,
+ * plus one region per message queue. There are as many message queues as
+ * the number of workers.
+ */
+static void
+setup_dsm(WorkerState *wstate)
+{
+	shm_toc_estimator	 e;
+	int					 toc_key = 0;
+	Size				 segsize;
+	dsm_segment			*seg;
+	shm_toc				*toc;
+	ParallelState		*pst;
+	shm_mq				*mq;
+	int64				 queue_size = 160000000; /* 16 MB for now */
+
+	/*
+	 * Estimate how much shared memory we need.
+	 *
+	 * Because the TOC machinery may choose to insert padding of oddly-sized
+	 * requests, we must estimate each chunk separately.
+	 *
+	 * We need one key to register the location of the header, and we need
+	 * nworkers keys to track the locations of the message queues.
+	 */
+	shm_toc_initialize_estimator(&e);
+	shm_toc_estimate_chunk(&e, sizeof(ParallelState));
+	shm_toc_estimate_chunk(&e, (Size) queue_size);
+
+	shm_toc_estimate_keys(&e, 1 + 1);
+	segsize = shm_toc_estimate(&e);
+
+	/* Create the shared memory segment and establish a table of contents. */
+	seg = dsm_create(shm_toc_estimate(&e), 0);
+	toc = shm_toc_create(PG_LOGICAL_APPLY_SHM_MAGIC, dsm_segment_address(seg),
+						 segsize);
+
+	/* Set up the header region. */
+	pst = shm_toc_allocate(toc, sizeof(ParallelState));
+	SpinLockInit(&pst->mutex);
+	pst->attached = false;
+	pst->ready = false;
+	pst->finished = false;
+	pst->failed = false;
+	pst->subid = MyLogicalRepWorker->subid;
+	pst->stream_xid = stream_xid;
+	pst->n = nworkers + 1;
+
+	shm_toc_insert(toc, toc_key++, pst);
+
+	/* Set up one message queue per worker, plus one. */
+	mq = shm_mq_create(shm_toc_allocate(toc, (Size) queue_size),
+						(Size) queue_size);
+	shm_toc_insert(toc, toc_key++, mq);
+	shm_mq_set_sender(mq, MyProc);
+
+	/* Attach the queues. */
+	wstate->mq_handle = shm_mq_attach(mq, seg, NULL);
+
+	/* Return results to caller. */
+	wstate->dsm_seg = seg;
+	wstate->pstate = pst;
+}
+
+/*
+ * Register background workers.
+ */
+static WorkerState *
+setup_background_worker(void)
+{
+	MemoryContext		oldcontext;
+	bool				launched;
+	WorkerState		   *wstate;
+
+	elog(LOG, "setting up apply worker #%u", nworkers + 1);
+
+	oldcontext = MemoryContextSwitchTo(ApplyContext);
+
+	wstate = (WorkerState *) palloc0(sizeof(WorkerState));
+
+	/* Setup shared memory */
+	setup_dsm(wstate);
+
+	launched = logicalrep_worker_launch(MyLogicalRepWorker->dbid,
+										MySubscription->oid,
+										MySubscription->name,
+										MyLogicalRepWorker->userid,
+										InvalidOid,
+										dsm_segment_handle(wstate->dsm_seg));
+
+	if (launched)
+	{
+		/* Wait for worker to become ready. */
+		wait_for_worker_ready(wstate, false);
+
+		ApplyWorkersList = lappend(ApplyWorkersList, wstate);
+		nworkers += 1;
+	}
+	else
+	{
+		pfree(wstate);
+		wstate = NULL;
+	}
+
+	MemoryContextSwitchTo(oldcontext);
+
+	return wstate;
+}
+
+/*
+ * Send the data to worker via shared-memory queue.
+ */
+static void
+send_data_to_worker(WorkerState *wstate, Size nbytes, const void *data)
+{
+	shm_mq_result result;
+
+	result = shm_mq_send(wstate->mq_handle, nbytes, data, false, true);
+
+	if (result != SHM_MQ_SUCCESS)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+				 errmsg("could not send tuple to shared-memory queue")));
+}
+
+static void
+wait_for_worker_ready(WorkerState *wstate, bool notify)
+{
+	bool result = false;
+
+	if (notify)
+	{
+		char action = 'E';
+
+		/* Notify worker that we received STREAM STOP */
+		send_data_to_worker(wstate, 1, &action);
+	}
+
+	for (;;)
+	{
+		bool ready;
+		bool failed;
+
+		/* If the worker is ready, we have succeeded. */
+		SpinLockAcquire(&wstate->pstate->mutex);
+		ready = wstate->pstate->ready;
+		failed = wstate->pstate->failed;
+		SpinLockRelease(&wstate->pstate->mutex);
+
+		if (ready)
+		{
+			result = true;
+			break;
+		}
+
+		/* If any workers (or the postmaster) have died, we have failed. */
+		if (failed)
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("Background worker %u failed to apply transaction %u",
+							wstate->pstate->n, wstate->pstate->stream_xid)));
+
+		/* Wait to be signalled. */
+		WaitLatch(MyLatch, WL_LATCH_SET | WL_EXIT_ON_PM_DEATH, 0,
+							WAIT_EVENT_LOGICAL_APPLY_WORKER_READY);
+
+		/* Reset the latch so we don't spin. */
+		ResetLatch(MyLatch);
+
+		/* An interrupt may have occurred while we were waiting. */
+		CHECK_FOR_INTERRUPTS();
+	}
+
+	if (!result)
+		ereport(ERROR,
+				(errcode(ERRCODE_INSUFFICIENT_RESOURCES),
+				 errmsg("one or more background workers failed to start")));
+}
+
+static void
+wait_for_transaction_finish(WorkerState *wstate)
+{
+	char action = 'F';
+
+	/* Notify worker that the streaming transaction has finished */
+	send_data_to_worker(wstate, 1, &action);
+
+	elog(LOG, "waiting for apply worker #%u to finish processing xid %u",
+		 wstate->pstate->n, wstate->pstate->stream_xid);
+
+	for (;;)
+	{
+		bool finished;
+		bool failed;
+
+		/* If the worker is finished, we have succeeded. */
+		SpinLockAcquire(&wstate->pstate->mutex);
+		finished = wstate->pstate->finished;
+		failed = wstate->pstate->failed;
+		SpinLockRelease(&wstate->pstate->mutex);
+
+		if (failed)
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("Background worker %u failed to apply transaction %u",
+							wstate->pstate->n, wstate->pstate->stream_xid)));
+
+		if (finished)
+		{
+			break;
+		}
+
+		/*
+		 * TO IMPROVE. Would it be better to use ConditionVariable instead of
+		 * Wait/Reset Latch here?
+		 */
+
+		/* Wait to be signalled. */
+		WaitLatch(MyLatch, WL_LATCH_SET | WL_EXIT_ON_PM_DEATH, 0,
+							WAIT_EVENT_LOGICAL_APPLY_WORKER_READY);
+
+		/* Reset the latch so we don't spin. */
+		ResetLatch(MyLatch);
+
+		/* An interrupt may have occurred while we were waiting. */
+		CHECK_FOR_INTERRUPTS();
+	}
+}
+
+/*
+ * Check if the streamed transaction was being processed in a bgworker.
+ */
+static bool
+transaction_applied_in_bgworker(TransactionId xid)
+{
+	if (!TransactionIdIsValid(xid) || isLogicalApplyWorker ||
+		ApplyWorkersHash == NULL)
+		return false;
+
+	stream_apply_worker = find_or_start_worker(xid, false);
+
+	return stream_apply_worker != NULL;
+}
+
+/*
+ * Check the status of workers and report an error if any bgworker exit
+ * unexpectedly.
+ *
+ * Exit if any relation is not in the READY state and if any worker is handling
+ * the streaming transaction at the same time. Because for streaming
+ * transactions that is being applied in bgworker, we cannot decide whether to
+ * apply the change for a relation that is not in the READY state (see
+ * should_apply_changes_for_rel) as we won't know remote_final_lsn by that
+ * time.
+ */
+static void
+check_workers_status(void)
+{
+	ListCell *lc;
+
+	if (isLogicalApplyWorker || MySubscription->stream != SUBSTREAM_APPLY)
+		return;
+
+	foreach(lc, ApplyWorkersList)
+	{
+		WorkerState *wstate = (WorkerState *) lfirst(lc);
+
+		/*
+		 * We don't lock here as in the worst case we will just detect the
+		 * failure of worker a bit later.
+		 */
+		if (wstate->pstate->failed)
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("Background worker %u exited unexpectedly",
+							wstate->pstate->n)));
+	}
+
+	if (!AllTablesyncsReady() && nfreeworkers != list_length(ApplyWorkersList))
+	{
+		ereport(LOG,
+				(errmsg("logical replication apply workers for subscription \"%s\" will restart",
+						MySubscription->name),
+				 errdetail("Cannot start table synchronization while bgworkers are "
+						   "handling streamed replication transaction")));
+
+		proc_exit(0);
+	}
+
+}
diff --git a/src/backend/utils/activity/wait_event.c b/src/backend/utils/activity/wait_event.c
index 87c15b9..24a6fac 100644
--- a/src/backend/utils/activity/wait_event.c
+++ b/src/backend/utils/activity/wait_event.c
@@ -388,6 +388,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_HASH_GROW_BUCKETS_REINSERT:
 			event_name = "HashGrowBucketsReinsert";
 			break;
+		case WAIT_EVENT_LOGICAL_APPLY_WORKER_READY:
+			event_name = "LogicalApplyWorkerReady";
+			break;
 		case WAIT_EVENT_LOGICAL_SYNC_DATA:
 			event_name = "LogicalSyncData";
 			break;
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 969e2a7..8c39050 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4521,8 +4521,12 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
 	if (strcmp(subinfo->subbinary, "t") == 0)
 		appendPQExpBufferStr(query, ", binary = true");
 
-	if (strcmp(subinfo->substream, "f") != 0)
-		appendPQExpBufferStr(query, ", streaming = on");
+	if (strcmp(subinfo->substream, "f") == 0)
+		appendPQExpBufferStr(query, ", streaming = off");
+	else if (strcmp(subinfo->substream, "s") == 0)
+		appendPQExpBufferStr(query, ", streaming = spool");
+	else if (strcmp(subinfo->substream, "a") == 0)
+		appendPQExpBufferStr(query, ", streaming = apply");
 
 	if (strcmp(subinfo->subtwophasestate, two_phase_disabled) != 0)
 		appendPQExpBufferStr(query, ", two_phase = on");
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index f006a92..1cd176a 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -68,7 +68,7 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
 	bool		subbinary;		/* True if the subscription wants the
 								 * publisher to send data in binary */
 
-	bool		substream;		/* Stream in-progress transactions. */
+	char		substream;		/* Stream in-progress transactions. */
 
 	char		subtwophasestate;	/* Stream two-phase transactions */
 
@@ -111,7 +111,7 @@ typedef struct Subscription
 	bool		enabled;		/* Indicates if the subscription is enabled */
 	bool		binary;			/* Indicates if the subscription wants data in
 								 * binary format */
-	bool		stream;			/* Allow streaming in-progress transactions. */
+	char		stream;			/* Allow streaming in-progress transactions. */
 	char		twophasestate;	/* Allow streaming two-phase transactions */
 	bool		disableonerr;	/* Indicates if the subscription should be
 								 * automatically disabled if a worker error
@@ -122,6 +122,18 @@ typedef struct Subscription
 	List	   *publications;	/* List of publication names to subscribe to */
 } Subscription;
 
+/* Disallow streaming in-progress transactions */
+#define SUBSTREAM_OFF	'f'
+
+/*
+ * Streaming transactions are written to a temporary file and applied only
+ * after the transaction is committed on upstream.
+ */
+#define SUBSTREAM_SPOOL	's'
+
+/* Streaming transactions are appied immediately via a background worker */
+#define SUBSTREAM_APPLY	'a'
+
 extern Subscription *GetSubscription(Oid subid, bool missing_ok);
 extern void FreeSubscription(Subscription *sub);
 extern void DisableSubscription(Oid subid);
diff --git a/src/include/replication/logicalproto.h b/src/include/replication/logicalproto.h
index a771ab8..0116867 100644
--- a/src/include/replication/logicalproto.h
+++ b/src/include/replication/logicalproto.h
@@ -243,8 +243,10 @@ extern TransactionId logicalrep_read_stream_start(StringInfo in,
 extern void logicalrep_write_stream_stop(StringInfo out);
 extern void logicalrep_write_stream_commit(StringInfo out, ReorderBufferTXN *txn,
 										   XLogRecPtr commit_lsn);
-extern TransactionId logicalrep_read_stream_commit(StringInfo out,
+extern TransactionId logicalrep_read_stream_commit_old(StringInfo out,
 												   LogicalRepCommitData *commit_data);
+extern void logicalrep_read_stream_commit(StringInfo out,
+										  LogicalRepCommitData *commit_data);
 extern void logicalrep_write_stream_abort(StringInfo out, TransactionId xid,
 										  TransactionId subxid);
 extern void logicalrep_read_stream_abort(StringInfo in, TransactionId *xid,
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index cd1b6e8..3bb3511 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -13,6 +13,7 @@
 #define LOGICALWORKER_H
 
 extern void ApplyWorkerMain(Datum main_arg);
+extern void LogicalApplyBgwMain(Datum main_arg);
 
 extern bool IsLogicalWorker(void);
 
diff --git a/src/include/replication/origin.h b/src/include/replication/origin.h
index 14d5c49..1ed51bc 100644
--- a/src/include/replication/origin.h
+++ b/src/include/replication/origin.h
@@ -53,7 +53,7 @@ extern XLogRecPtr replorigin_get_progress(RepOriginId node, bool flush);
 
 extern void replorigin_session_advance(XLogRecPtr remote_commit,
 									   XLogRecPtr local_commit);
-extern void replorigin_session_setup(RepOriginId node);
+extern void replorigin_session_setup(RepOriginId node, bool acquire);
 extern void replorigin_session_reset(void);
 extern XLogRecPtr replorigin_session_get_progress(bool flush);
 
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 4485d4e..4d11a13 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -60,6 +60,8 @@ typedef struct LogicalRepWorker
 	 */
 	FileSet    *stream_fileset;
 
+	bool		subworker;
+
 	/* Stats. */
 	XLogRecPtr	last_lsn;
 	TimestampTz last_send_time;
@@ -84,8 +86,9 @@ extern void logicalrep_worker_attach(int slot);
 extern LogicalRepWorker *logicalrep_worker_find(Oid subid, Oid relid,
 												bool only_running);
 extern List *logicalrep_workers_find(Oid subid, bool only_running);
-extern void logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname,
-									 Oid userid, Oid relid);
+extern bool logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname,
+									 Oid userid, Oid relid,
+									 dsm_handle subworker_dsm);
 extern void logicalrep_worker_stop(Oid subid, Oid relid);
 extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
 extern void logicalrep_worker_wakeup_ptr(LogicalRepWorker *worker);
diff --git a/src/include/utils/wait_event.h b/src/include/utils/wait_event.h
index b578e2e..d5081d9 100644
--- a/src/include/utils/wait_event.h
+++ b/src/include/utils/wait_event.h
@@ -105,6 +105,7 @@ typedef enum
 	WAIT_EVENT_HASH_GROW_BUCKETS_ALLOCATE,
 	WAIT_EVENT_HASH_GROW_BUCKETS_ELECT,
 	WAIT_EVENT_HASH_GROW_BUCKETS_REINSERT,
+	WAIT_EVENT_LOGICAL_APPLY_WORKER_READY,
 	WAIT_EVENT_LOGICAL_SYNC_DATA,
 	WAIT_EVENT_LOGICAL_SYNC_STATE_CHANGE,
 	WAIT_EVENT_MQ_INTERNAL,
diff --git a/src/test/subscription/t/029_on_error.pl b/src/test/subscription/t/029_on_error.pl
index e8b904b..9659fb9 100644
--- a/src/test/subscription/t/029_on_error.pl
+++ b/src/test/subscription/t/029_on_error.pl
@@ -108,7 +108,7 @@ my $publisher_connstr = $node_publisher->connstr . ' dbname=postgres';
 $node_publisher->safe_psql('postgres',
 	"CREATE PUBLICATION pub FOR TABLE tbl");
 $node_subscriber->safe_psql('postgres',
-	"CREATE SUBSCRIPTION sub CONNECTION '$publisher_connstr' PUBLICATION pub WITH (disable_on_error = true, streaming = on, two_phase = on)"
+	"CREATE SUBSCRIPTION sub CONNECTION '$publisher_connstr' PUBLICATION pub WITH (disable_on_error = true, streaming = spool, two_phase = on)"
 );
 
 # Initial synchronization failure causes the subscription to be disabled.
-- 
2.7.2.windows.1

