From 06cb238bc2acd9e2beaf4d2b3229e8e528088638 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 reads from the file and
applies 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 apply background worker (if available) as
soon as the xact's first stream is received and the main apply worker will send
changes to this new worker via shared memory. The apply background worker will
directly apply the change instead of writing it to temporary files.  We keep
this worker assigned till the transaction commit is received and also wait for
the worker to finish at commit. This preserves commit ordering and avoids
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
apply 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.

This patch also extends the subscription streaming option so that the user can
control whether to apply the streaming transaction in an apply background
worker or spill the change to disk. User can set the streaming option to
'on/off', 'apply'. For now, 'apply' means the streaming will be applied via a
apply background worker if available. 'on' means the streaming transaction will
be spilled to disk.
---
 doc/src/sgml/catalogs.sgml                  |   10 +-
 doc/src/sgml/ref/create_subscription.sgml   |   24 +-
 src/backend/commands/subscriptioncmds.c     |   66 +-
 src/backend/postmaster/bgworker.c           |    3 +
 src/backend/replication/logical/launcher.c  |   87 +-
 src/backend/replication/logical/origin.c    |    6 +-
 src/backend/replication/logical/tablesync.c |   10 +-
 src/backend/replication/logical/worker.c    | 1396 ++++++++++++++++++++++++---
 src/backend/utils/activity/wait_event.c     |    3 +
 src/bin/pg_dump/pg_dump.c                   |    6 +-
 src/include/catalog/pg_subscription.h       |   17 +-
 src/include/replication/logicalworker.h     |    1 +
 src/include/replication/origin.h            |    2 +-
 src/include/replication/worker_internal.h   |   13 +-
 src/include/utils/wait_event.h              |    1 +
 src/test/regress/expected/subscription.out  |    2 +-
 src/tools/pgindent/typedefs.list            |    4 +
 17 files changed, 1455 insertions(+), 196 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index a533a21..5e46c4b 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -7863,11 +7863,15 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
 
      <row>
       <entry role="catalog_table_entry"><para role="column_definition">
-       <structfield>substream</structfield> <type>bool</type>
+       <structfield>substream</structfield> <type>char</type>
       </para>
       <para>
-       If true, the subscription will allow streaming of in-progress
-       transactions
+       Controls how to handle the streaming of in-progress transactions:
+       <literal>f</literal> = disallow streaming of in-progress transactions,
+       <literal>t</literal> = spill the changes of in-progress transactions to
+       disk and apply at once after the transaction is committed on the
+       publisher,
+       <literal>a</literal> = apply changes directly using a background worker
       </para></entry>
      </row>
 
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 203bb41..22bd818 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>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 workers, 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 an error happens when
+          applying changes in a background worker, it might not report the
+          finish LSN of the remote transaction in the server log.
          </para>
         </listitem>
        </varlistentry>
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 690cdaa..f154762 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,62 @@ 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 and "apply".
+ */
+static char
+defGetStreamingMode(DefElem *def)
+{
+	/*
+	 * If no parameter given, assume "true" is meant.
+	 */
+	if (def->arg == NULL)
+		return SUBSTREAM_ON;
+
+	/*
+	 * Allow 0, 1, "true", "false", "on", "off" or "apply".
+	 */
+	switch (nodeTag(def->arg))
+	{
+		case T_Integer:
+			switch (intVal(def->arg))
+			{
+				case 0:
+					return SUBSTREAM_OFF;
+				case 1:
+					return SUBSTREAM_ON;
+				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, "false") == 0 ||
+					pg_strcasecmp(sval, "off") == 0)
+					return SUBSTREAM_OFF;
+				if (pg_strcasecmp(sval, "true") == 0 ||
+					pg_strcasecmp(sval, "on") == 0)
+					return SUBSTREAM_ON;
+				if (pg_strcasecmp(sval, "apply") == 0)
+					return SUBSTREAM_APPLY;
+			}
+			break;
+	}
+	ereport(ERROR,
+				(errcode(ERRCODE_SYNTAX_ERROR),
+				 errmsg("%s requires a Boolean value 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 +188,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 +289,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 +657,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 +1116,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 40601ae..40ccb89 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -128,6 +128,9 @@ static const struct
 	},
 	{
 		"ApplyWorkerMain", ApplyWorkerMain
+	},
+	{
+		"ApplyBgworkerMain", ApplyBgworkerMain
 	}
 };
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index bd5f78c..f8f0490 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -73,6 +73,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 logicalrep_worker_stop_internal(LogicalRepWorker *worker);
 
 static bool on_commit_launcher_wakeup = false;
 
@@ -223,6 +224,10 @@ logicalrep_worker_find(Oid subid, Oid relid, bool only_running)
 	{
 		LogicalRepWorker *w = &LogicalRepCtx->workers[i];
 
+		/* We only need main apply worker or table sync worker here */
+		if (w->subworker)
+			continue;
+
 		if (w->in_use && w->subid == subid && w->relid == relid &&
 			(!only_running || w->proc))
 		{
@@ -259,11 +264,11 @@ logicalrep_workers_find(Oid subid, bool only_running)
 }
 
 /*
- * Start new apply background worker, if possible.
+ * Start new 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;
@@ -273,6 +278,10 @@ logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
 	LogicalRepWorker *worker = NULL;
 	int			nsyncworkers;
 	TimestampTz now;
+	bool		is_subworker = (subworker_dsm != DSM_HANDLE_INVALID);
+
+	/* We don't support table sync in subworker */
+	Assert(!(is_subworker && OidIsValid(relid)));
 
 	ereport(DEBUG1,
 			(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -350,7 +359,7 @@ retry:
 	if (OidIsValid(relid) && nsyncworkers >= max_sync_workers_per_subscription)
 	{
 		LWLockRelease(LogicalRepWorkerLock);
-		return;
+		return false;
 	}
 
 	/*
@@ -364,7 +373,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. */
@@ -379,6 +388,7 @@ retry:
 	worker->relstate = SUBREL_STATE_UNKNOWN;
 	worker->relstate_lsn = InvalidXLogRecPtr;
 	worker->stream_fileset = NULL;
+	worker->subworker = is_subworker;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -396,19 +406,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, "ApplyBgworkerMain");
+
+
 	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 (is_subworker)
+		memcpy(bgw.bgw_extra, &subworker_dsm, sizeof(dsm_handle));
+
 	if (!RegisterDynamicBackgroundWorker(&bgw, &bgw_handle))
 	{
 		/* Failed to start worker, so clean up the worker slot. */
@@ -421,11 +443,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;
 }
 
 /*
@@ -436,7 +460,6 @@ void
 logicalrep_worker_stop(Oid subid, Oid relid)
 {
 	LogicalRepWorker *worker;
-	uint16		generation;
 
 	LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
 
@@ -449,6 +472,22 @@ logicalrep_worker_stop(Oid subid, Oid relid)
 		return;
 	}
 
+	logicalrep_worker_stop_internal(worker);
+
+	LWLockRelease(LogicalRepWorkerLock);
+}
+
+/*
+ * Workhorse for logicalrep_worker_stop() and logicalrep_worker_detach(). Stop
+ * the worker and wait for it to die.
+ */
+static void
+logicalrep_worker_stop_internal(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.
@@ -485,10 +524,7 @@ logicalrep_worker_stop(Oid subid, Oid relid)
 		 * different, meaning that a different worker has taken the slot.
 		 */
 		if (!worker->in_use || worker->generation != generation)
-		{
-			LWLockRelease(LogicalRepWorkerLock);
 			return;
-		}
 
 		/* Worker has assigned proc, so it has started. */
 		if (worker->proc)
@@ -522,8 +558,6 @@ logicalrep_worker_stop(Oid subid, Oid relid)
 
 		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
 	}
-
-	LWLockRelease(LogicalRepWorkerLock);
 }
 
 /*
@@ -599,6 +633,28 @@ logicalrep_worker_attach(int slot)
 static void
 logicalrep_worker_detach(void)
 {
+	/*
+	 * If we are the main apply worker, stop all the apply background 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)
+				logicalrep_worker_stop_internal(w);
+		}
+
+		LWLockRelease(LogicalRepWorkerLock);
+	}
+
 	/* Block concurrent access. */
 	LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
 
@@ -621,6 +677,7 @@ logicalrep_worker_cleanup(LogicalRepWorker *worker)
 	worker->userid = InvalidOid;
 	worker->subid = InvalidOid;
 	worker->relid = InvalidOid;
+	worker->subworker = false;
 }
 
 /*
@@ -868,7 +925,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 21937ab..aaa6694 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,7 @@ replorigin_session_setup(RepOriginId node)
 		if (curstate->roident != node)
 			continue;
 
-		else if (curstate->acquired_by != 0)
+		else if (curstate->acquired_by != 0 && acquire)
 		{
 			ereport(ERROR,
 					(errcode(ERRCODE_OBJECT_IN_USE),
@@ -1321,7 +1321,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/tablesync.c b/src/backend/replication/logical/tablesync.c
index b03e0f5..b9be9d0 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 3b80ed9..862ad41 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -22,8 +22,25 @@
  * STREAMED TRANSACTIONS
  * ---------------------
  * Streamed transactions (large transactions exceeding a memory limit on the
- * upstream) are not applied immediately, but instead, the data is written
- * to temporary files and then applied at once when the final commit arrives.
+ * 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 is
+ * received and the main apply worker will send changes to this new worker via
+ * shared memory. We keep this worker assigned till the transaction commit is
+ * received and also wait for the worker to finish at commit. This preserves
+ * commit ordering and avoids 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, the data is
+ * written to temporary files and then applied at once when the final commit
+ * arrives.
  *
  * Unlike the regular (non-streamed) case, handling streamed transactions has
  * to handle aborts of both the toplevel transaction and subtransactions. This
@@ -174,11 +191,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 +219,75 @@
 
 #define NAPTIME_PER_CYCLE 1000	/* max sleep time between cycles (1s) */
 
+#define PG_LOGICAL_APPLY_SHM_MAGIC 0x79fb2447 // TODO Consider change
+
+/*
+ * States for apply background worker.
+ */
+typedef enum ApplyBgworkerState
+{
+	APPLY_BGWORKER_ATTACHED = 0,
+	APPLY_BGWORKER_READY,
+	APPLY_BGWORKER_BUSY,
+	APPLY_BGWORKER_FINISHED,
+	APPLY_BGWORKER_EXIT
+} ApplyBgworkerState;
+
+/*
+ * Shared information among apply workers.
+ */
+typedef struct ApplyBgworkerShared
+{
+	slock_t	mutex;
+
+	/* State for apply background worker. */
+	ApplyBgworkerState	state;
+
+	TransactionId	stream_xid;
+	uint32	n;	/* id of apply background worker */
+} ApplyBgworkerShared;
+
+/*
+ * Struct for maintaining an apply background worker.
+ */
+typedef struct WorkerState
+{
+	shm_mq_handle			*mq_handle;
+	dsm_segment				*dsm_seg;
+	ApplyBgworkerShared volatile	*pstate;
+} WorkerState;
+
+typedef struct WorkerEntry
+{
+	TransactionId	xid;
+	WorkerState	   *wstate;
+} WorkerEntry;
+
+/* Apply background workers hash table (initialized on first use) */
+static HTAB *ApplyWorkersHash = NULL;
+static List *ApplyWorkersIdleList = NIL;
+static List *ApplyWorkersList = NIL;
+
+/* Fields valid only for apply background workers */
+volatile ApplyBgworkerShared *MyParallelState = NULL;
+static List *subxactlist = NIL;
+
+/* The number of changes during one streaming block */
+static uint32 nchanges = 0;
+
+/* Worker setup and interactions */
+static WorkerState *apply_bgworker_setup(void);
+static WorkerState *apply_bgworker_find_or_start(TransactionId xid,
+												 bool start);
+static void apply_bgworker_setup_dsm(WorkerState *wstate);
+static void apply_bgworker_wait_for(WorkerState *wstate,
+									char wait_for_state);
+static void apply_bgworker_send_data(WorkerState *wstate, Size nbytes,
+									 const void *data);
+static void apply_bgworker_free(WorkerState *wstate);
+static void apply_bgworker_check_status(void);
+static void apply_bgworker_set_state(char state);
+
 typedef struct FlushPosition
 {
 	dlist_node	node;
@@ -260,18 +350,23 @@ static XLogRecPtr remote_final_lsn = InvalidXLogRecPtr;
 static bool in_streamed_transaction = false;
 
 static TransactionId stream_xid = InvalidTransactionId;
+static WorkerState *stream_apply_worker = NULL;
+
+/* check if we apply transaction in apply bgworker */
+#define apply_bgworker_active() (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 (streaming = on),
+ * 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)))
@@ -426,43 +521,103 @@ end_replication_step(void)
 }
 
 /*
- * Handle streamed transactions.
+ * Handle streamed transactions for both main apply worker and apply background
+ * worker.
  *
- * 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 apply background worker.
+ *
+ * For the apply 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)
+	/*
+	 * Return if we are not in streaming mode and are not in an apply
+	 * background worker.
+	 */
+	if (!in_streamed_transaction && !am_apply_bgworker())
 		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 (am_apply_bgworker())
+	{
+		/*
+		 * Inside apply background 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[MAXPGPATH];
+
+			snprintf(spname, MAXPGPATH, "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 (apply_bgworker_active())
+	{
+		/*
+		 * If we decided to apply the changes of this transaction in an apply
+		 * background worker, pass the data to the worker.
+		 */
+		apply_bgworker_send_data(stream_apply_worker, s->len, s->data);
+		nchanges += 1;
+
+		/*
+		 * XXX The publisher side doesn't always send relation update messages
+		 * after the streaming transaction, so update the relation in main
+		 * apply 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 !am_apply_bgworker();
 }
 
 /*
@@ -844,6 +999,9 @@ apply_handle_commit(StringInfo s)
 
 	apply_handle_commit_internal(&commit_data);
 
+	/* Check the status of apply background worker if any. */
+	apply_bgworker_check_status();
+
 	/* Process any tables that are being synchronized in parallel. */
 	process_syncing_tables(commit_data.end_lsn);
 
@@ -898,7 +1056,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 +1110,9 @@ apply_handle_prepare(StringInfo s)
 
 	in_remote_transaction = false;
 
+	/* Check the status of apply background worker if any. */
+	apply_bgworker_check_status();
+
 	/* Process any tables that are being synchronized in parallel. */
 	process_syncing_tables(prepare_data.end_lsn);
 
@@ -1063,11 +1226,130 @@ 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 *
+apply_bgworker_find_or_start(TransactionId xid, bool start)
+{
+	bool found;
+	WorkerState *wstate;
+	WorkerEntry *entry = NULL;
+
+	if (!TransactionIdIsValid(xid))
+		return NULL;
+
+	/*
+	 * We don't start new background worker if we are not in streaming apply
+	 * mode.
+	 */
+	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.
+	 */
+	if (start && !XLogRecPtrIsInvalid(MySubscription->skiplsn))
+		return NULL;
+
+	/*
+	 * For streaming transactions that are 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.
+	 */
+	if (start && !AllTablesyncsReady())
+		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)
+	{
+		entry->wstate->pstate->state = APPLY_BGWORKER_BUSY;
+		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 = apply_bgworker_setup();
+
+		if (wstate == NULL)
+		{
+			/*
+			 * If the bgworker cannot be launched, remove entry in hash table.
+			 */
+			hash_search(ApplyWorkersHash, &xid, HASH_REMOVE, &found);
+			return NULL;
+		}
+	}
+
+	/* Fill up the hash entry */
+	wstate->pstate->state = APPLY_BGWORKER_BUSY;
+	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
+apply_bgworker_free(WorkerState *wstate)
+{
+	bool found;
+	MemoryContext oldctx;
+	TransactionId xid = wstate->pstate->stream_xid;
+
+	Assert(wstate->pstate->state == APPLY_BGWORKER_FINISHED);
+
+	oldctx = MemoryContextSwitchTo(ApplyContext);
+
+	hash_search(ApplyWorkersHash, &xid,
+				HASH_REMOVE, &found);
+
+	elog(LOG, "adding finished apply worker #%u for xid %u to the idle list",
+		 wstate->pstate->n, wstate->pstate->stream_xid);
+
+	ApplyWorkersIdleList = lappend(ApplyWorkersIdleList, wstate);
+
+	MemoryContextSwitchTo(oldctx);
+}
+
+/*
+ * Handle STREAM PREPARE.
  */
 static void
 apply_handle_stream_prepare(StringInfo s)
@@ -1088,24 +1370,68 @@ 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 (am_apply_bgworker())
+	{
+		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);
+		apply_bgworker_set_state(APPLY_BGWORKER_READY);
+	}
+	else
+	{
+		WorkerState *wstate = apply_bgworker_find_or_start(prepare_data.xid, false);
+
+		/*
+		 * If we are in main apply worker, check if we are processing this
+		 * transaction in a bgworker.
+		 */
+		if (wstate)
+		{
+			apply_bgworker_send_data(wstate, s->len, s->data);
+
+			apply_bgworker_wait_for(wstate, APPLY_BGWORKER_READY);
+
+			pgstat_report_stat(false);
+			store_flush_position(prepare_data.end_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(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);
+		}
+	}
 
 	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 apply background worker if any. */
+	apply_bgworker_check_status();
 
 	/* Process any tables that are being synchronized in parallel. */
 	process_syncing_tables(prepare_data.end_lsn);
@@ -1155,15 +1481,6 @@ apply_handle_stream_start(StringInfo s)
 				(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
-	 * will be committed after processing all the messages. We need the
-	 * transaction for handling the buffile, used for serializing the
-	 * streaming data and subxact info.
-	 */
-	begin_replication_step();
-
 	/* notify handle methods we're processing a remote transaction */
 	in_streamed_transaction = true;
 
@@ -1177,36 +1494,90 @@ apply_handle_stream_start(StringInfo s)
 
 	set_apply_error_context_xact(stream_xid, InvalidXLogRecPtr);
 
+	if (am_apply_bgworker())
+	{
+		/*
+		 * Make sure the handle apply_dispatch methods are aware we're in a
+		 * remote transaction.
+		 */
+		in_remote_transaction = true;
+
+		/* If we are in a bgworker, begin the transaction */
+		AcceptInvalidationMessages();
+		maybe_reread_subscription();
+
+		StartTransactionCommand();
+		BeginTransactionBlock();
+		CommitTransactionCommand();
+
+		return;
+	}
+
 	/*
-	 * 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
-	 * context. We create this on the very first streaming message from any
-	 * transaction and then use it for this and other streaming transactions.
-	 * Now, we could create a fileset at the start of the worker as well but
-	 * then we won't be sure that it will ever be used.
+	 * If we are in main apply worker, check if there is any free bgworker
+	 * we can use to process this transaction.
 	 */
-	if (MyLogicalRepWorker->stream_fileset == NULL)
+	stream_apply_worker = apply_bgworker_find_or_start(stream_xid, first_segment);
+
+	if (stream_apply_worker)
 	{
-		MemoryContext oldctx;
+		/*
+		 * If we have free worker or we already started to apply this
+		 * transaction in bgworker, we pass the data to worker.
+		 */
+		if (first_segment)
+			apply_bgworker_send_data(stream_apply_worker, s->len, s->data);
 
-		oldctx = MemoryContextSwitchTo(ApplyContext);
+		nchanges = 0;
+		elog(LOG, "starting streaming of xid %u", stream_xid);
+	}
 
-		MyLogicalRepWorker->stream_fileset = palloc(sizeof(FileSet));
-		FileSetInit(MyLogicalRepWorker->stream_fileset);
+	/*
+	 * If no worker is available for the first stream start, we start to
+	 * serialize all the changes of the transaction.
+	 */
+	else
+	{
+		/*
+		 * 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
+		 * will be committed after processing all the messages. We need the
+		 * transaction for handling the buffile, used for serializing the
+		 * streaming data and subxact info.
+		 */
+		begin_replication_step();
 
-		MemoryContextSwitchTo(oldctx);
-	}
+		/*
+		 * 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
+		 * context. We create this on the very first streaming message from any
+		 * transaction and then use it for this and other streaming transactions.
+		 * Now, we could create a fileset at the start of the worker as well but
+		 * then we won't be sure that it will ever be used.
+		 */
+		if (MyLogicalRepWorker->stream_fileset == NULL)
+		{
+			MemoryContext oldctx;
 
-	/* open the spool file for this transaction */
-	stream_open_file(MyLogicalRepWorker->subid, stream_xid, first_segment);
+			oldctx = MemoryContextSwitchTo(ApplyContext);
 
-	/* if this is not the first segment, open existing subxact file */
-	if (!first_segment)
-		subxact_info_read(MyLogicalRepWorker->subid, stream_xid);
+			MyLogicalRepWorker->stream_fileset = palloc(sizeof(FileSet));
+			FileSetInit(MyLogicalRepWorker->stream_fileset);
 
-	pgstat_report_activity(STATE_RUNNING, NULL);
+			MemoryContextSwitchTo(oldctx);
+		}
 
-	end_replication_step();
+		/* open the spool file for this transaction */
+		stream_open_file(MyLogicalRepWorker->subid, stream_xid, first_segment);
+
+		/* if this is not the first segment, open existing subxact file */
+		if (!first_segment)
+			subxact_info_read(MyLogicalRepWorker->subid, stream_xid);
+
+		end_replication_step();
+	}
+
+	pgstat_report_activity(STATE_RUNNING, NULL);
 }
 
 /*
@@ -1220,44 +1591,47 @@ apply_handle_stream_stop(StringInfo s)
 				(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();
+	if (apply_bgworker_active())
+	{
+		char	action = LOGICAL_REP_MSG_STREAM_STOP;
 
-	/* We must be in a valid transaction state */
-	Assert(IsTransactionState());
+		apply_bgworker_send_data(stream_apply_worker, 1, &action);
+		apply_bgworker_wait_for(stream_apply_worker, APPLY_BGWORKER_READY);
 
-	/* Commit the per-stream transaction */
-	CommitTransactionCommand();
+		elog(LOG, "stopped streaming of xid %u, %u changes streamed", stream_xid, nchanges);
+	}
+	else
+	{
+		/*
+		 * 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();
 
-	in_streamed_transaction = false;
+		/* We must be in a valid transaction state */
+		Assert(IsTransactionState());
+
+		/* Commit the per-stream transaction */
+		CommitTransactionCommand();
+
+		/* Reset per-stream context */
+		MemoryContextReset(LogicalStreamingContext);
+	}
 
-	/* Reset per-stream context */
-	MemoryContextReset(LogicalStreamingContext);
+	in_streamed_transaction = false;
+	stream_apply_worker = NULL;
 
 	pgstat_report_activity(STATE_IDLE, NULL);
 	reset_apply_error_context_info();
 }
 
 /*
- * Handle STREAM abort message.
+ * Handle STREAM ABORT message when the transaction was spilled to disk.
  */
 static void
-apply_handle_stream_abort(StringInfo s)
+serialize_stream_abort(TransactionId xid, TransactionId subxid)
 {
-	TransactionId xid;
-	TransactionId subxid;
-
-	if (in_streamed_transaction)
-		ereport(ERROR,
-				(errcode(ERRCODE_PROTOCOL_VIOLATION),
-				 errmsg_internal("STREAM ABORT message without STREAM STOP")));
-
-	logicalrep_read_stream_abort(s, &xid, &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,51 +1713,163 @@ apply_handle_stream_abort(StringInfo s)
 		end_replication_step();
 		CommitTransactionCommand();
 	}
-
-	reset_apply_error_context_info();
 }
 
 /*
- * Common spoolfile processing.
+ * Handle STREAM ABORT message.
  */
 static void
-apply_spooled_messages(TransactionId xid, XLogRecPtr lsn)
+apply_handle_stream_abort(StringInfo s)
 {
-	StringInfoData s2;
-	int			nchanges;
-	char		path[MAXPGPATH];
-	char	   *buffer = NULL;
-	MemoryContext oldcxt;
-	BufFile    *fd;
+	TransactionId xid;
+	TransactionId subxid;
 
-	maybe_start_skipping_changes(lsn);
+	if (in_streamed_transaction)
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg_internal("STREAM COMMIT message without STREAM STOP")));
 
-	/* Make sure we have an open transaction */
-	begin_replication_step();
+	logicalrep_read_stream_abort(s, &xid, &subxid);
 
-	/*
-	 * Allocate file handle and memory required to process all the messages in
-	 * TopTransactionContext to avoid them getting reset after each message is
-	 * processed.
-	 */
-	oldcxt = MemoryContextSwitchTo(TopTransactionContext);
+	if (am_apply_bgworker())
+	{
+		ereport(LOG,
+				(errmsg("[Apply BGW #%u] aborting current transaction xid=%u, subxid=%u",
+				MyParallelState->n, GetCurrentTransactionIdIfAny(), GetCurrentSubTransactionId())));
 
-	/* Open the spool file for the committed/prepared transaction */
-	changes_filename(path, MyLogicalRepWorker->subid, xid);
-	elog(DEBUG1, "replaying changes from file \"%s\"", path);
+		/*
+		 * 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);
 
-	fd = BufFileOpenFileSet(MyLogicalRepWorker->stream_fileset, path, O_RDONLY,
-							false);
+			AbortCurrentTransaction();
 
-	buffer = palloc(BLCKSZ);
-	initStringInfo(&s2);
+			EndTransactionBlock(false);
+			CommitTransactionCommand();
 
-	MemoryContextSwitchTo(oldcxt);
+			in_remote_transaction = false;
 
-	remote_final_lsn = lsn;
+			list_free(subxactlist);
+			subxactlist = NIL;
 
-	/*
-	 * Make sure the handle apply_dispatch methods are aware we're in a remote
+			apply_bgworker_set_state(APPLY_BGWORKER_FINISHED);
+		}
+		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[MAXPGPATH];
+
+			set_apply_error_context_xact(subxid, InvalidXLogRecPtr);
+			snprintf(spname, MAXPGPATH, "savepoint_for_xid_%u", subxid);
+
+			ereport(LOG,
+					(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);
+			}
+
+			apply_bgworker_set_state(APPLY_BGWORKER_READY);
+		}
+
+		reset_apply_error_context_info();
+	}
+	else
+	{
+		WorkerState *wstate = apply_bgworker_find_or_start(xid, false);
+
+		/*
+		 * If we are in main apply worker, check if we are processing this
+		 * transaction in a bgworker.
+		 */
+		if (wstate)
+		{
+			apply_bgworker_send_data(wstate, s->len, s->data);
+
+			if (subxid == xid)
+			{
+				apply_bgworker_wait_for(wstate, APPLY_BGWORKER_FINISHED);
+				apply_bgworker_free(wstate);
+			}
+			else
+				apply_bgworker_wait_for(wstate, APPLY_BGWORKER_READY);
+		}
+
+		/*
+		 * We are in main apply worker and the transaction has been serialized
+		 * to file.
+		 */
+		else
+			serialize_stream_abort(xid, subxid);
+	}
+}
+
+/*
+ * Common spoolfile processing.
+ */
+static void
+apply_spooled_messages(TransactionId xid, XLogRecPtr lsn)
+{
+	StringInfoData s2;
+	int			nchanges;
+	char		path[MAXPGPATH];
+	char	   *buffer = NULL;
+	MemoryContext oldcxt;
+	BufFile    *fd;
+
+	maybe_start_skipping_changes(lsn);
+
+	/* Make sure we have an open transaction */
+	begin_replication_step();
+
+	/*
+	 * Allocate file handle and memory required to process all the messages in
+	 * TopTransactionContext to avoid them getting reset after each message is
+	 * processed.
+	 */
+	oldcxt = MemoryContextSwitchTo(TopTransactionContext);
+
+	/* Open the spool file for the committed/prepared transaction */
+	changes_filename(path, MyLogicalRepWorker->subid, xid);
+	elog(DEBUG1, "replaying changes from file \"%s\"", path);
+
+	fd = BufFileOpenFileSet(MyLogicalRepWorker->stream_fileset, path, O_RDONLY,
+							false);
+
+	buffer = palloc(BLCKSZ);
+	initStringInfo(&s2);
+
+	MemoryContextSwitchTo(oldcxt);
+
+	remote_final_lsn = lsn;
+
+	/*
+	 * Make sure the handle apply_dispatch methods are aware we're in a remote
 	 * transaction.
 	 */
 	in_remote_transaction = true;
@@ -1463,40 +1949,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 +2897,100 @@ 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 = 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 (am_apply_bgworker())
+	{
+		/*
+		 * 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();
+
+		in_remote_transaction = false;
+
+		pgstat_report_stat(false);
+
+		list_free(subxactlist);
+		subxactlist = NIL;
+
+		apply_bgworker_set_state(APPLY_BGWORKER_FINISHED);
+	}
+	else
+	{
+		/*
+		 * If we are in main apply worker, check if we are processing this
+		 * transaction in an apply background worker.
+		 */
+		WorkerState *wstate = apply_bgworker_find_or_start(xid, false);
+
+		if (wstate)
+		{
+			/* Send commit message */
+			apply_bgworker_send_data(wstate, s->len, s->data);
+
+			/* Wait for apply background worker to finish */
+			apply_bgworker_wait_for(wstate, APPLY_BGWORKER_FINISHED);
+
+			pgstat_report_stat(false);
+			store_flush_position(commit_data.end_lsn);
+			stop_skipping_changes();
+
+			apply_bgworker_free(wstate);
+
+			/*
+			 * 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 apply background worker if any. */
+	apply_bgworker_check_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 +3057,8 @@ apply_dispatch(StringInfo s)
 			break;
 
 		case LOGICAL_REP_MSG_STREAM_START:
+			/* FIXME : log for debugging here. */
+			elog(LOG, "LOGICAL_REP_MSG_STREAM_START");
 			apply_handle_stream_start(s);
 			break;
 
@@ -2618,6 +3166,10 @@ store_flush_position(XLogRecPtr remote_lsn)
 {
 	FlushPosition *flushpos;
 
+	/* We only need to collect the LSN in main apply worker */
+	if (am_apply_bgworker())
+		return;
+
 	/* Need to do this in permanent context */
 	MemoryContextSwitchTo(ApplyContext);
 
@@ -2794,6 +3346,9 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 			AcceptInvalidationMessages();
 			maybe_reread_subscription();
 
+			/* Check the status of apply background worker if any. */
+			apply_bgworker_check_status();
+
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
 		}
@@ -3686,7 +4241,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 +4288,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 +4446,8 @@ clear_subscription_skip_lsn(XLogRecPtr finish_lsn)
 	XLogRecPtr	myskiplsn = MySubscription->skiplsn;
 	bool		started_tx = false;
 
-	if (likely(XLogRecPtrIsInvalid(myskiplsn)))
+	if (likely(XLogRecPtrIsInvalid(myskiplsn)) ||
+		am_apply_bgworker())
 		return;
 
 	if (!IsTransactionState())
@@ -4027,3 +4583,533 @@ 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 ApplyBgworkerShared *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;
+		XLogRecPtr	start_lsn;
+		XLogRecPtr	end_lsn;
+		TimestampTz send_time;
+
+		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)
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("lost connection to the main apply worker")));
+
+		if (len == 0)
+		{
+			elog(LOG, "[Apply BGW #%u] got zero-length message, stopping", pst->n);
+			break;
+		}
+
+		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))
+		{
+			/* End message of streaming chunk */
+			case LOGICAL_REP_MSG_STREAM_STOP:
+				elog(LOG, "[Apply BGW #%u] ended processing streaming chunk,"
+						  "waiting on shm_mq_receive", pst->n);
+
+				apply_bgworker_set_state(APPLY_BGWORKER_READY);
+
+				SetLatch(&registrant->procLatch);
+
+				in_streamed_transaction = false;
+				pgstat_report_activity(STATE_IDLE, NULL);
+				continue;
+			case 'w':
+				break;
+			default:
+				elog(ERROR, "unexpected message");
+				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);
+
+		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 exit state so that the main apply worker can realize we have
+ * shutdown.
+ */
+static void
+ApplyBgwShutdown(int code, Datum arg)
+{
+	SpinLockAcquire(&MyParallelState->mutex);
+	MyParallelState->state = APPLY_BGWORKER_EXIT;
+	SpinLockRelease(&MyParallelState->mutex);
+
+	dsm_detach((dsm_segment *) DatumGetPointer(arg));
+}
+
+/*
+ * Apply Background Worker entry point
+ */
+void
+ApplyBgworkerMain(Datum main_arg)
+{
+	volatile ApplyBgworkerShared *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);
+
+	/* 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));
+
+	/* Look up the parallel state. */
+	pst = shm_toc_lookup(toc, 0, false);
+	MyParallelState = pst;
+
+	/*
+	 * 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);
+
+	/*
+	 * The apply bgworker don't need to monopolize this replication origin
+	 * which was already acquired by its leader process.
+	 */
+	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 apply operation.
+	 */
+	apply_bgworker_set_state(APPLY_BGWORKER_ATTACHED);
+
+	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 ApplyBgworkerShared,
+ * plus one region per message queue. There are as many message queues as
+ * the number of workers.
+ */
+static void
+apply_bgworker_setup_dsm(WorkerState *wstate)
+{
+	shm_toc_estimator	 e;
+	int					 toc_key = 0;
+	Size				 segsize;
+	dsm_segment			*seg;
+	shm_toc				*toc;
+	ApplyBgworkerShared		*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
+	 * another key to track the location of the message queue.
+	 */
+	shm_toc_initialize_estimator(&e);
+	shm_toc_estimate_chunk(&e, sizeof(ApplyBgworkerShared));
+	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(ApplyBgworkerShared));
+	SpinLockInit(&pst->mutex);
+	pst->state = APPLY_BGWORKER_BUSY;
+	pst->stream_xid = stream_xid;
+	pst->n = list_length(ApplyWorkersList) + 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;
+}
+
+/*
+ * Start apply worker background worker process and allocate shared memory for
+ * it.
+ */
+static WorkerState *
+apply_bgworker_setup(void)
+{
+	MemoryContext		oldcontext;
+	bool				launched;
+	WorkerState		   *wstate;
+
+	elog(LOG, "setting up apply worker #%u", list_length(ApplyWorkersList) + 1);
+
+	oldcontext = MemoryContextSwitchTo(ApplyContext);
+
+	wstate = (WorkerState *) palloc0(sizeof(WorkerState));
+
+	/* Setup shared memory */
+	apply_bgworker_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 attach. */
+		apply_bgworker_wait_for(wstate, APPLY_BGWORKER_ATTACHED);
+
+		ApplyWorkersList = lappend(ApplyWorkersList, wstate);
+	}
+	else
+	{
+		pfree(wstate);
+		wstate = NULL;
+	}
+
+	MemoryContextSwitchTo(oldcontext);
+
+	return wstate;
+}
+
+/*
+ * Send the data to the specified apply bgworker via shared-memory queue.
+ */
+static void
+apply_bgworker_send_data(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 tuples to shared-memory queue")));
+}
+
+/*
+ * Wait until the state of apply background worker reaches the 'wait_for_state'
+ */
+static void
+apply_bgworker_wait_for(WorkerState *wstate, char wait_for_state)
+{
+	for (;;)
+	{
+		char status;
+
+		SpinLockAcquire(&wstate->pstate->mutex);
+		status = wstate->pstate->state;
+		SpinLockRelease(&wstate->pstate->mutex);
+
+		/* Done if already in correct state. */
+		if (status == wait_for_state)
+			break;
+
+		/* If any workers (or the postmaster) have died, we have failed. */
+		if (status == APPLY_BGWORKER_EXIT)
+			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_STATE_CHANGE);
+
+		/* Reset the latch so we don't spin. */
+		ResetLatch(MyLatch);
+
+		/* An interrupt may have occurred while we were waiting. */
+		CHECK_FOR_INTERRUPTS();
+	}
+}
+
+/*
+ * Check the status of workers and report an error if any apply background
+ * worker has exited 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 apply 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
+apply_bgworker_check_status(void)
+{
+	ListCell *lc;
+
+	if (am_apply_bgworker() || 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->state == APPLY_BGWORKER_EXIT)
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("Background worker %u exited unexpectedly",
+							wstate->pstate->n)));
+	}
+
+	if (list_length(ApplyWorkersIdleList) != list_length(ApplyWorkersList) &&
+		!AllTablesyncsReady())
+	{
+		ereport(LOG,
+				(errmsg("logical replication apply workers for subscription \"%s\" will restart",
+						MySubscription->name),
+				 errdetail("Cannot handle streamed replication transaction by apply "
+						   "bgworkers until all tables are synchronized")));
+
+		proc_exit(0);
+	}
+}
+
+/* Set the state of apply background worker */
+static void
+apply_bgworker_set_state(char state)
+{
+	if (!am_apply_bgworker())
+		return;
+
+	elog(LOG, "[Apply BGW #%u] set state to %c", MyParallelState->n, state);
+
+	SpinLockAcquire(&MyParallelState->mutex);
+	MyParallelState->state = state;
+	SpinLockRelease(&MyParallelState->mutex);
+}
diff --git a/src/backend/utils/activity/wait_event.c b/src/backend/utils/activity/wait_event.c
index 87c15b9..ba781e6 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_STATE_CHANGE:
+			event_name = "LogicalApplyWorkerStateChange";
+			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 7cc9c72..6f8b30a 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4450,7 +4450,7 @@ getSubscriptions(Archive *fout)
 	if (fout->remoteVersion >= 140000)
 		appendPQExpBufferStr(query, " s.substream,\n");
 	else
-		appendPQExpBufferStr(query, " false AS substream,\n");
+		appendPQExpBufferStr(query, " 'f' AS substream,\n");
 
 	if (fout->remoteVersion >= 150000)
 		appendPQExpBufferStr(query,
@@ -4580,8 +4580,10 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
 	if (strcmp(subinfo->subbinary, "t") == 0)
 		appendPQExpBufferStr(query, ", binary = true");
 
-	if (strcmp(subinfo->substream, "f") != 0)
+	if (strcmp(subinfo->substream, "t") == 0)
 		appendPQExpBufferStr(query, ", streaming = on");
+	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 d1260f5..9b394a4 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -68,7 +68,8 @@ 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.
+								 * See SUBSTREAM_xxx constants. */
 
 	char		subtwophasestate;	/* Stream two-phase transactions */
 
@@ -109,7 +110,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
@@ -120,6 +121,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_ON	't'
+
+/* Streaming transactions are applied 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/logicalworker.h b/src/include/replication/logicalworker.h
index cd1b6e8..6a1af7f 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 ApplyBgworkerMain(Datum main_arg);
 
 extern bool IsLogicalWorker(void);
 
diff --git a/src/include/replication/origin.h b/src/include/replication/origin.h
index 2d1b5e5..b270434 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 901845a..6613e32 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);
@@ -109,4 +112,10 @@ am_tablesync_worker(void)
 	return OidIsValid(MyLogicalRepWorker->relid);
 }
 
+static inline bool
+am_apply_bgworker(void)
+{
+	return MyLogicalRepWorker->subworker;
+}
+
 #endif							/* WORKER_INTERNAL_H */
diff --git a/src/include/utils/wait_event.h b/src/include/utils/wait_event.h
index b578e2e..c2d2a11 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_STATE_CHANGE,
 	WAIT_EVENT_LOGICAL_SYNC_DATA,
 	WAIT_EVENT_LOGICAL_SYNC_STATE_CHANGE,
 	WAIT_EVENT_MQ_INTERNAL,
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index 7fcfad1..f769835 100644
--- a/src/test/regress/expected/subscription.out
+++ b/src/test/regress/expected/subscription.out
@@ -197,7 +197,7 @@ ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
 DROP SUBSCRIPTION regress_testsub;
 -- fail - streaming must be boolean
 CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, streaming = foo);
-ERROR:  streaming requires a Boolean value
+ERROR:  streaming requires a Boolean value or "apply"
 -- now it works
 CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, streaming = true);
 WARNING:  tables were not subscribed, you will have to run ALTER SUBSCRIPTION ... REFRESH PUBLICATION to subscribe the tables
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index dd12149..9d89912 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -116,6 +116,8 @@ Append
 AppendPath
 AppendRelInfo
 AppendState
+ApplyBgworkerShared
+ApplyBgworkerState
 ApplyErrorCallbackArg
 ApplyExecutionData
 ApplySubXactData
@@ -2994,11 +2996,13 @@ WordEntryPosVector
 WordEntryPosVector1
 WorkTableScan
 WorkTableScanState
+WorkerEntry
 WorkerInfo
 WorkerInfoData
 WorkerInstrumentation
 WorkerJobDumpPtrType
 WorkerJobRestorePtrType
+WorkerState
 Working_State
 WriteBufPtrType
 WriteBytePtrType
-- 
2.7.2.windows.1

