[PATCH] Reuse Workers and Replication Slots during Logical Replication

Started by Melih Mutluover 3 years ago166 messages
#1Melih Mutlu
m.melihmutlu@gmail.com
1 attachment(s)

Hi hackers,

I created a patch to reuse tablesync workers and their replication slots
for more tables that are not synced yet. So that overhead of creating and
dropping workers/replication slots can be reduced.

Current version of logical replication has two steps: tablesync and apply.
In tablesync step, apply worker creates a tablesync worker for each table
and those tablesync workers are killed when they're done with their
associated table. (the number of tablesync workers running at the same time
is limited by "max_sync_workers_per_subscription")
Each tablesync worker also creates a replication slot on publisher during
its lifetime and drops the slot before exiting.

The purpose of this patch is getting rid of the overhead of
creating/killing a new worker (and replication slot) for each table.
It aims to reuse tablesync workers and their replication slots so that
tablesync workers can copy multiple tables from publisher to subscriber
during their lifetime.

The benefits of reusing tablesync workers can be significant if tables are
empty or close to empty.
In an empty table case, spawning tablesync workers and handling replication
slots are where the most time is spent since the actual copy phase takes
too little time.

The changes in the behaviour of tablesync workers with this patch as
follows:
1- After tablesync worker is done with syncing the current table, it takes
a lock and fetches tables in init state
2- it looks for a table that is not already being synced by another worker
from the tables with init state
3- If it founds one, updates its state for the new table and loops back to
beginning to start syncing
4- If no table found, it drops the replication slot and exits

With those changes, I did some benchmarking to see if it improves anything.
This results compares this patch with the latest version of master branch.
"max_sync_workers_per_subscription" is set to 2 as default.
Got some results simply averaging timings from 5 consecutive runs for each
branch.

First, tested logical replication with empty tables.
10 tables
----------------
- master: 286.964 ms
- the patch: 116.852 ms

100 tables
----------------
- master: 2785.328 ms
- the patch: 706.817 ms

10K tables
----------------
- master: 39612.349 ms
- the patch: 12526.981 ms

Also tried replication tables with some data
10 tables loaded with 10MB data
----------------
- master: 1517.714 ms
- the patch: 1399.965 ms

100 tables loaded with 10MB data
----------------
- master: 16327.229 ms
- the patch: 11963.696 ms

Then loaded more data
10 tables loaded with 100MB data
----------------
- master: 13910.189 ms
- the patch: 14770.982 ms

100 tables loaded with 100MB data
----------------
- master: 146281.457 ms
- the patch: 156957.512

If tables are mostly empty, the improvement can be significant - up to 3x
faster logical replication.
With some data loaded, it can still be faster to some extent.
When the table size increases more, the advantage of reusing workers
becomes insignificant.

I would appreciate your comments and suggestions.Thanks in advance for
reviewing.

Best,
Melih

Attachments:

0001-Reuse-Logical-Replication-Background-worker.patchapplication/octet-stream; name=0001-Reuse-Logical-Replication-Background-worker.patchDownload
From 09a9b666ee3df60b2662769e200643527dfc9435 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH] Reuse Logical Replication Background worker

---
 src/backend/catalog/pg_subscription.c       |  59 ++++
 src/backend/commands/subscriptioncmds.c     | 164 ++++++----
 src/backend/replication/logical/launcher.c  |   3 +
 src/backend/replication/logical/tablesync.c | 118 +++++--
 src/backend/replication/logical/worker.c    | 338 ++++++++++++--------
 src/include/catalog/pg_subscription_rel.h   |   1 +
 src/include/replication/slot.h              |   3 +-
 src/include/replication/worker_internal.h   |  14 +
 8 files changed, 488 insertions(+), 212 deletions(-)

diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 8856ce3b50..81f8ab6cbf 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -635,3 +635,62 @@ GetSubscriptionNotReadyRelations(Oid subid)
 
 	return res;
 }
+
+/*
+ * Get all relations for subscription that are in init state.
+ *
+ * Returned list is palloc'ed in current memory context.
+ */
+List *
+GetSubscriptionInitStateRelations(Oid subid)
+{
+	List	   *res = NIL;
+	Relation	rel;
+	HeapTuple	tup;
+	int			nkeys = 0;
+	ScanKeyData skey[2];
+	SysScanDesc scan;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	ScanKeyInit(&skey[nkeys++],
+				Anum_pg_subscription_rel_srsubid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(subid));
+
+	ScanKeyInit(&skey[nkeys++],
+				Anum_pg_subscription_rel_srsubstate,
+				BTEqualStrategyNumber, F_CHAREQ,
+				CharGetDatum(SUBREL_STATE_INIT));
+
+	scan = systable_beginscan(rel, InvalidOid, false,
+							  NULL, nkeys, skey);
+
+	while (HeapTupleIsValid(tup = systable_getnext(scan)))
+	{
+		Form_pg_subscription_rel subrel;
+		SubscriptionRelState *relstate;
+		Datum		d;
+		bool		isnull;
+
+		subrel = (Form_pg_subscription_rel) GETSTRUCT(tup);
+
+		relstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
+		relstate->relid = subrel->srrelid;
+		relstate->state = subrel->srsubstate;
+		d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+							Anum_pg_subscription_rel_srsublsn, &isnull);
+		if (isnull)
+			relstate->lsn = InvalidXLogRecPtr;
+		else
+			relstate->lsn = DatumGetLSN(d);
+
+		res = lappend(res, relstate);
+	}
+
+	/* Cleanup */
+	systable_endscan(scan);
+	table_close(rel, AccessShareLock);
+
+	return res;
+}
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index e2852286a7..34f4c0cb06 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -765,6 +765,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	} SubRemoveRels;
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -887,7 +889,18 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/* 
+				 * Find the logical replication sync worker if exists 
+				 * Store the slot number for dropping associated replication slot later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
@@ -921,31 +934,27 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
+		 * Drop the replication slots associated with tablesync workers for removed tables.
+		 * This has to be at the end because otherwise if there is an error while doing
 		 * the database operations we won't be able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			int *slot_to_drop = (int *) palloc(sizeof(int));
+			memcpy(slot_to_drop, lfirst(lc), sizeof(int));
+
+			/*
+			 * There is no certainty, maybe the slot
+			 * does not exist yet. Also, if we fail after removing some of
+			 * the slots, next time, it will again try to drop already
+			 * dropped slots and fail. For these reasons, we allow
+			 * missing_ok = true for the drop.
+			 */
+			ReplicationSlotNameForTablesync(sub->oid, *slot_to_drop,
+											syncslotname, sizeof(syncslotname));
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1530,39 +1539,16 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
-
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
-
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		List *slots = NULL;
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+		
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char *syncslotname = (char *) lfirst(lc);
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1591,6 +1577,69 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * WRITE COMMENT HERE
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok){
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+						" FROM pg_replication_slots"
+						" WHERE slot_name LIKE 'pg_%i_sync_%%';",
+						 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+					 errmsg("not tuple returned."));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();\
+	return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -1832,6 +1881,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -1842,15 +1892,21 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+		
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
 			char		syncslotname[NAMEDATALEN] = {0};
 
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
+			ReplicationSlotNameForTablesync(subid, worker->slot, syncslotname,
 											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
 				 syncslotname);
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 2bdab53e19..918d8137c0 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -370,7 +370,9 @@ retry:
 	/* Prepare the worker slot. */
 	worker->launch_time = now;
 	worker->in_use = true;
+	worker->is_first_run = true;
 	worker->generation++;
+	worker->slot = slot;
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -378,6 +380,7 @@ retry:
 	worker->relid = relid;
 	worker->relstate = SUBREL_STATE_UNKNOWN;
 	worker->relstate_lsn = InvalidXLogRecPtr;
+	worker->move_to_next_rel = false;
 	worker->stream_fileset = NULL;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 670c6fcada..c82a203fc5 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -126,12 +126,8 @@ static bool FetchTableStates(bool *started_tx);
 
 static StringInfo copybuf = NULL;
 
-/*
- * Exit routine for synchronization worker.
- */
 static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+clean_sync_worker(void)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -143,18 +139,27 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
-	/* And flush all writes. */
-	XLogFlush(GetXLogWriteRecPtr());
-
-	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
-	CommitTransactionCommand();
+	/* Disconnect from publisher.
+	 * Otherwise reused sync workers causes exceeding max_wal_senders 
+	 */
+	walrcv_disconnect(LogRepWorkerWalRcvConn);
+	LogRepWorkerWalRcvConn = NULL;
 
 	/* Find the main apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
+}
+
+/*
+ * Exit routine for synchronization worker.
+ */
+static void
+pg_attribute_noreturn()
+finish_sync_worker(void)
+{
+	clean_sync_worker();
+	
+	/* And flush all writes. */
+	XLogFlush(GetXLogWriteRecPtr());
 
 	/* Stop gracefully */
 	proc_exit(0);
@@ -180,7 +185,7 @@ wait_for_relation_state_change(Oid relid, char expected_state)
 		LogicalRepWorker *worker;
 		XLogRecPtr	statelsn;
 
-		CHECK_FOR_INTERRUPTS();
+		CHECK_FOR_INTERRUPTS();		
 
 		InvalidateCatalogSnapshot();
 		state = GetSubscriptionRelState(MyLogicalRepWorker->subid,
@@ -284,6 +289,10 @@ invalidate_syncing_table_states(Datum arg, int cacheid, uint32 hashvalue)
 static void
 process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 {
+	List	   *rstates;
+	SubscriptionRelState *rstate;
+	ListCell   *lc;
+
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_CATCHUP &&
@@ -323,18 +332,64 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
+										MyLogicalRepWorker->slot,
 										syncslotname,
 										sizeof(syncslotname));
 
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
+		 * Check if any table whose relation state is still INIT. 
+		 * If a table in INIT state is found, the worker will not be finished,
+		 * it will be reused instead.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
+		rstates = GetSubscriptionInitStateRelations(MySubscription->oid);
+		
+		foreach (lc, rstates)
+		{
+			rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
+			memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
 
-		finish_sync_worker();
+			/* 
+			 * Pick the table for the next run
+			 * if there is not another worker already picked that table.
+			 */
+			LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+			if (!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+			{
+				ereport(LOG,
+						(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
+								MySubscription->name,
+								get_rel_name(MyLogicalRepWorker->relid))));
+
+				/* Update worker state for the next table */
+				SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+				MyLogicalRepWorker->is_first_run = false;
+				MyLogicalRepWorker->relid = rstate->relid;
+				MyLogicalRepWorker->relstate = rstate->state;
+				MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+				MyLogicalRepWorker->move_to_next_rel = true;
+				SpinLockRelease(&MyLogicalRepWorker->relmutex);
+				LWLockRelease(LogicalRepWorkerLock);
+				break;
+			}
+			LWLockRelease(LogicalRepWorkerLock);
+		}
+
+		/* Cleanup before next run or ending the worker. */
+		if(!MyLogicalRepWorker->move_to_next_rel)
+		{
+		   /*
+			* It is important to give an error if we are unable to drop the slot,
+			* otherwise, it won't be dropped till the corresponding subscription
+			* is dropped. So passing missing_ok = false.
+			*/
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
+
+			finish_sync_worker();
+		}
+		else
+		{
+			clean_sync_worker();
+		}
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -1152,11 +1207,11 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
+ReplicationSlotNameForTablesync(Oid suboid, int slot,
 								char *syncslotname, int szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%i_" UINT64_FORMAT, suboid,
+			 slot, GetSystemIdentifier());
 }
 
 /*
@@ -1219,7 +1274,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
+									MyLogicalRepWorker->slot,
 									slotname,
 									NAMEDATALEN);
 
@@ -1356,11 +1411,14 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	 * drop subscription happens which would complete without removing this
 	 * slot leading to a dangling slot on the server.
 	 */
-	HOLD_INTERRUPTS();
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-	RESUME_INTERRUPTS();
+	if (MyLogicalRepWorker->is_first_run)
+	{
+		HOLD_INTERRUPTS();
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						slotname, false /* permanent */ , false /* two_phase */ ,
+						CRS_USE_SNAPSHOT, origin_startpos);
+		RESUME_INTERRUPTS();
+	}
 
 	/*
 	 * Setup replication origin tracking. The purpose of doing this before the
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 38e3b1c1b3..31ed8ed3d0 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -315,6 +315,7 @@ static void stream_cleanup_files(Oid subid, TransactionId xid);
 static void stream_open_file(Oid subid, TransactionId xid, bool first);
 static void stream_write_change(char action, StringInfo s);
 static void stream_close_file(void);
+static void stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
@@ -2814,6 +2815,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+			if (MyLogicalRepWorker->move_to_next_rel)
+			{
+				endofstream = true;
+			}
 		}
 
 		/* Cleanup the memory. */
@@ -2915,8 +2920,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	/* Pop the error context stack */
 	error_context_stack = errcallback.previous;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/* 
+	 * If it's moving to next relation, this is a sync worker.
+	 * Sync workers end the streaming during process_syncing_tables_for_sync.
+	 * Calling endstreaming twice causes "no COPY in progress" errors.
+	 */
+	if (!MyLogicalRepWorker->move_to_next_rel)
+	{
+		/* All done */
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	}
 }
 
 /*
@@ -3457,6 +3470,34 @@ stream_write_change(char action, StringInfo s)
 	BufFileWrite(stream_fd, &s->data[s->cursor], len);
 }
 
+/*
+ * stream_build_options_replication
+ * 		Build logical replication streaming options.
+ *
+ * This function sets streaming options including replication slot name
+ * and origin start position. Workers need these options for logical replication.
+ */
+static void
+stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos)
+{
+	int server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+	options->proto.logical.streaming = MySubscription->stream;
+	options->proto.logical.twophase = false;
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -3568,6 +3609,136 @@ start_apply(XLogRecPtr origin_startpos)
 	PG_END_TRY();
 }
 
+/*
+ * Runs the tablesync worker.
+ * It starts table sync. After successful sync, 
+ * builds streaming options and starts streaming. 
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options, 
+					 char *slotname,
+					 char *originname,
+					 int originame_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Set this to false for safety, in case we're already reusing the worker */
+    MyLogicalRepWorker->move_to_next_rel = false;
+
+    start_table_sync(origin_startpos, &slotname);
+
+    /*
+        * Allocate the origin name in long-lived context for error context
+        * message.
+        */
+    ReplicationOriginNameForTablesync(MySubscription->oid,
+                                        MyLogicalRepWorker->relid,
+                                        originname,
+                                        originame_size);
+    apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
+                                                                originname);
+    
+    stream_build_options(options, slotname, origin_startpos);
+
+    /* Start normal logical streaming replication. */
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+}
+
+/*
+ * Runs the apply worker.
+ * It sets up replication origin, the streaming options 
+ * and then starts streaming. 
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
+{
+    RepOriginId originid;
+    TimeLineID	startpointTLI;
+    char	   *err;
+
+    slotname = MySubscription->slotname;
+
+    /*
+	 * This shouldn't happen if the subscription is enabled, but guard
+	 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+	 * crash if slot is NULL.)
+	 */
+    if (!slotname)
+        ereport(ERROR,
+                (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+                    errmsg("subscription has no replication slot set")));
+
+    /* Setup replication origin tracking. */
+    StartTransactionCommand();
+    snprintf(originname, originname_size, "pg_%u", MySubscription->oid);
+    originid = replorigin_by_name(originname, true);
+    if (!OidIsValid(originid))
+        originid = replorigin_create(originname);
+    replorigin_session_setup(originid);
+    replorigin_session_origin = originid;
+    *origin_startpos = replorigin_session_get_progress(false);
+    CommitTransactionCommand();
+
+    LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+                                            MySubscription->name, &err);
+    if (LogRepWorkerWalRcvConn == NULL)
+        ereport(ERROR,
+                (errcode(ERRCODE_CONNECTION_FAILURE),
+                    errmsg("could not connect to the publisher: %s", err)));
+
+    /*
+	 * We don't really use the output identify_system for anything but it
+	 * does some initializations on the upstream so let's still call it.
+	 */
+    (void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+    /*
+	 * Allocate the origin name in long-lived context for error context
+	 * message.
+	 */
+    apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
+                                                                originname);
+
+    stream_build_options(options, slotname, origin_startpos);
+
+    /*
+     * Even when the two_phase mode is requested by the user, it remains
+     * as the tri-state PENDING until all tablesyncs have reached READY
+     * state. Only then, can it become ENABLED.
+     *
+     * Note: If the subscription has no tables then leave the state as
+     * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+     * work.
+     */
+    if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+        AllTablesyncsReady())
+    {
+        /* Start streaming with two_phase enabled */
+        options->proto.logical.twophase = true;
+        walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+        StartTransactionCommand();
+        UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+        MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+        CommitTransactionCommand();
+    }
+    else
+    {
+        walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+    }
+
+    ereport(DEBUG1,
+            (errmsg("logical replication apply worker for subscription \"%s\" two_phase is %s",
+                    MySubscription->name,
+                    MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+                    MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+                    MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+                    "?")));
+}
+
 /* Logical Replication Apply worker entry point */
 void
 ApplyWorkerMain(Datum main_arg)
@@ -3578,7 +3749,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	/* Attach to slot */
 	logicalrep_worker_attach(worker_slot);
@@ -3669,141 +3839,55 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		/*
-		 * Allocate the origin name in long-lived context for error context
-		 * message.
-		 */
-		ReplicationOriginNameForTablesync(MySubscription->oid,
-										  MyLogicalRepWorker->relid,
-										  originname,
-										  sizeof(originname));
-		apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
-																   originname);
-	}
-	else
-	{
-		/* This is main apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* 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);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		/*
-		 * Allocate the origin name in long-lived context for error context
-		 * message.
-		 */
-		apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
-																   originname);
-	}
-
 	/*
-	 * Setup callback for syscache so that we know when something changes in
-	 * the subscription relation state.
-	 */
+	* Setup callback for syscache so that we know when something changes in
+	* the subscription relation state.
+	* Do this outside the loop to avoid exceeding MAX_SYSCACHE_CALLBACKS
+	*/
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
-								  invalidate_syncing_table_states,
-								  (Datum) 0);
-
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
+								invalidate_syncing_table_states,
+								(Datum) 0);
 
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
+	/*
+	 * The loop where worker does its job.
+	 * It loops until the worker is not reused. 
+	 */
+	while (MyLogicalRepWorker->is_first_run || 
+			MyLogicalRepWorker->move_to_next_rel)
+	{
+		if (am_tablesync_worker())
+			{
+				/* 
+				* This is a tablesync worker. 
+				* Start syncing tables before starting the apply loop.  
+				*/
+				run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
+			}
+			else
+			{
+				/* This is main apply worker */
+				run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
+			}
 
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-	options.proto.logical.streaming = MySubscription->stream;
-	options.proto.logical.twophase = false;
+		/* Run the main loop. */
+		start_apply(origin_startpos);
 
-	if (!am_tablesync_worker())
-	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
+		if (MyLogicalRepWorker->move_to_next_rel)
 		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+			/* Reset the currenct replication origin session.
+			* Since we'll use the same process for another relation, it needs to be reset 
+			* and will be created again later while syncing the new relation.
+			*/
+			replorigin_session_origin = InvalidRepOriginId;
+			replorigin_session_reset();
 
 			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+			ereport(LOG,
+					(errmsg("logical replication table synchronization worker for subscription \"%s\" has moved to sync table \"%s\".",
+							MySubscription->name, get_rel_name(MyLogicalRepWorker->relid))));
 			CommitTransactionCommand();
 		}
-		else
-		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-		}
-
-		ereport(DEBUG1,
-				(errmsg("logical replication apply worker for subscription \"%s\" two_phase is %s",
-						MySubscription->name,
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-						"?")));
 	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
-
-	/* Run the main loop. */
-	start_apply(origin_startpos);
-
 	proc_exit(0);
 }
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 9df99c3418..21a773ad56 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -90,5 +90,6 @@ extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 extern bool HasSubscriptionRelations(Oid subid);
 extern List *GetSubscriptionRelations(Oid subid);
 extern List *GetSubscriptionNotReadyRelations(Oid subid);
+extern List *GetSubscriptionInitStateRelations(Oid subid);
 
 #endif							/* PG_SUBSCRIPTION_REL_H */
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 8c9f3321d5..57c4215cfe 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -217,8 +217,9 @@ extern void ReplicationSlotsDropDBSlots(Oid dboid);
 extern bool InvalidateObsoleteReplicationSlots(XLogSegNo oldestSegno);
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, int szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, int slot, char *syncslotname, int szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List* GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 901845abc2..db4e96be80 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -30,9 +30,18 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/* 
+	 * Indicates if worker is running for the first time
+	 * or in reuse
+	 */
+	bool		is_first_run;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
+	/* The slot that this worker is using */
+	int		slot;
+
 	/* Pointer to proc array. NULL if not running. */
 	PGPROC	   *proc;
 
@@ -51,6 +60,11 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/* 
+	 * Used to indicate whether sync worker will be reused for another relation
+	 */
+	bool		move_to_next_rel;
+	
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction, the
-- 
2.25.1

#2Amit Kapila
amit.kapila16@gmail.com
In reply to: Melih Mutlu (#1)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Tue, Jul 5, 2022 at 7:20 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

I created a patch to reuse tablesync workers and their replication slots for more tables that are not synced yet. So that overhead of creating and dropping workers/replication slots can be reduced.

Current version of logical replication has two steps: tablesync and apply.
In tablesync step, apply worker creates a tablesync worker for each table and those tablesync workers are killed when they're done with their associated table. (the number of tablesync workers running at the same time is limited by "max_sync_workers_per_subscription")
Each tablesync worker also creates a replication slot on publisher during its lifetime and drops the slot before exiting.

The purpose of this patch is getting rid of the overhead of creating/killing a new worker (and replication slot) for each table.
It aims to reuse tablesync workers and their replication slots so that tablesync workers can copy multiple tables from publisher to subscriber during their lifetime.

The benefits of reusing tablesync workers can be significant if tables are empty or close to empty.
In an empty table case, spawning tablesync workers and handling replication slots are where the most time is spent since the actual copy phase takes too little time.

The changes in the behaviour of tablesync workers with this patch as follows:
1- After tablesync worker is done with syncing the current table, it takes a lock and fetches tables in init state
2- it looks for a table that is not already being synced by another worker from the tables with init state
3- If it founds one, updates its state for the new table and loops back to beginning to start syncing
4- If no table found, it drops the replication slot and exits

How would you choose the slot name for the table sync, right now it
contains the relid of the table for which it needs to perform sync?
Say, if we ignore to include the appropriate identifier in the slot
name, we won't be able to resue/drop the slot after restart of table
sync worker due to an error.

With those changes, I did some benchmarking to see if it improves anything.
This results compares this patch with the latest version of master branch. "max_sync_workers_per_subscription" is set to 2 as default.
Got some results simply averaging timings from 5 consecutive runs for each branch.

First, tested logical replication with empty tables.
10 tables
----------------
- master: 286.964 ms
- the patch: 116.852 ms

100 tables
----------------
- master: 2785.328 ms
- the patch: 706.817 ms

10K tables
----------------
- master: 39612.349 ms
- the patch: 12526.981 ms

Also tried replication tables with some data
10 tables loaded with 10MB data
----------------
- master: 1517.714 ms
- the patch: 1399.965 ms

100 tables loaded with 10MB data
----------------
- master: 16327.229 ms
- the patch: 11963.696 ms

Then loaded more data
10 tables loaded with 100MB data
----------------
- master: 13910.189 ms
- the patch: 14770.982 ms

100 tables loaded with 100MB data
----------------
- master: 146281.457 ms
- the patch: 156957.512

If tables are mostly empty, the improvement can be significant - up to 3x faster logical replication.
With some data loaded, it can still be faster to some extent.

These results indicate that it is a good idea, especially for very small tables.

When the table size increases more, the advantage of reusing workers becomes insignificant.

It seems from your results that performance degrades for large
relations. Did you try to investigate the reasons for the same?

--
With Regards,
Amit Kapila.

#3Dilip Kumar
dilipbalaut@gmail.com
In reply to: Amit Kapila (#2)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Wed, Jul 6, 2022 at 9:06 AM Amit Kapila <amit.kapila16@gmail.com> wrote:

How would you choose the slot name for the table sync, right now it
contains the relid of the table for which it needs to perform sync?
Say, if we ignore to include the appropriate identifier in the slot
name, we won't be able to resue/drop the slot after restart of table
sync worker due to an error.

I had a quick look into the patch and it seems it is using the worker
array index instead of relid while forming the slot name, and I think
that make sense, because now whichever worker is using that worker
index can reuse the slot created w.r.t that index.

With those changes, I did some benchmarking to see if it improves anything.
This results compares this patch with the latest version of master branch. "max_sync_workers_per_subscription" is set to 2 as default.
Got some results simply averaging timings from 5 consecutive runs for each branch.

First, tested logical replication with empty tables.
10 tables
----------------
- master: 286.964 ms
- the patch: 116.852 ms

100 tables
----------------
- master: 2785.328 ms
- the patch: 706.817 ms

10K tables
----------------
- master: 39612.349 ms
- the patch: 12526.981 ms

Also tried replication tables with some data
10 tables loaded with 10MB data
----------------
- master: 1517.714 ms
- the patch: 1399.965 ms

100 tables loaded with 10MB data
----------------
- master: 16327.229 ms
- the patch: 11963.696 ms

Then loaded more data
10 tables loaded with 100MB data
----------------
- master: 13910.189 ms
- the patch: 14770.982 ms

100 tables loaded with 100MB data
----------------
- master: 146281.457 ms
- the patch: 156957.512

If tables are mostly empty, the improvement can be significant - up to 3x faster logical replication.
With some data loaded, it can still be faster to some extent.

These results indicate that it is a good idea, especially for very small tables.

When the table size increases more, the advantage of reusing workers becomes insignificant.

It seems from your results that performance degrades for large
relations. Did you try to investigate the reasons for the same?

Yeah, that would be interesting to know that why there is a drop in some cases.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

#4Amit Kapila
amit.kapila16@gmail.com
In reply to: Dilip Kumar (#3)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Wed, Jul 6, 2022 at 1:47 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Wed, Jul 6, 2022 at 9:06 AM Amit Kapila <amit.kapila16@gmail.com> wrote:

How would you choose the slot name for the table sync, right now it
contains the relid of the table for which it needs to perform sync?
Say, if we ignore to include the appropriate identifier in the slot
name, we won't be able to resue/drop the slot after restart of table
sync worker due to an error.

I had a quick look into the patch and it seems it is using the worker
array index instead of relid while forming the slot name, and I think
that make sense, because now whichever worker is using that worker
index can reuse the slot created w.r.t that index.

I think that won't work because each time on restart the slot won't be
fixed. Now, it is possible that we may drop the wrong slot if that
state of copying rel is SUBREL_STATE_DATASYNC. Also, it is possible
that while creating a slot, we fail because the same name slot already
exists due to some other worker which has created that slot has been
restarted. Also, what about origin_name, won't that have similar
problems? Also, if the state is already SUBREL_STATE_FINISHEDCOPY, if
the slot is not the same as we have used in the previous run of a
particular worker, it may start WAL streaming from a different point
based on the slot's confirmed_flush_location.

--
With Regards,
Amit Kapila.

#5Dilip Kumar
dilipbalaut@gmail.com
In reply to: Amit Kapila (#4)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Wed, Jul 6, 2022 at 2:48 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

On Wed, Jul 6, 2022 at 1:47 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Wed, Jul 6, 2022 at 9:06 AM Amit Kapila <amit.kapila16@gmail.com> wrote:

How would you choose the slot name for the table sync, right now it
contains the relid of the table for which it needs to perform sync?
Say, if we ignore to include the appropriate identifier in the slot
name, we won't be able to resue/drop the slot after restart of table
sync worker due to an error.

I had a quick look into the patch and it seems it is using the worker
array index instead of relid while forming the slot name, and I think
that make sense, because now whichever worker is using that worker
index can reuse the slot created w.r.t that index.

I think that won't work because each time on restart the slot won't be
fixed. Now, it is possible that we may drop the wrong slot if that
state of copying rel is SUBREL_STATE_DATASYNC.

So it will drop the previous slot the worker at that index was using,
so it is possible that on that slot some relation was at
SUBREL_STATE_FINISHEDCOPY or so and we will drop that slot. Because
now relid and replication slot association is not 1-1 so it would be
wrong to drop based on the relstate which is picked by this worker.
In short it makes sense what you have pointed out.

Also, it is possible

that while creating a slot, we fail because the same name slot already
exists due to some other worker which has created that slot has been
restarted. Also, what about origin_name, won't that have similar
problems? Also, if the state is already SUBREL_STATE_FINISHEDCOPY, if
the slot is not the same as we have used in the previous run of a
particular worker, it may start WAL streaming from a different point
based on the slot's confirmed_flush_location.

Yeah this is also true, when a tablesync worker has to do catch up
after completing the copy then it might stream from the wrong lsn.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

#6Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Amit Kapila (#4)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Amit and Dilip,

Thanks for the replies.

I had a quick look into the patch and it seems it is using the worker
array index instead of relid while forming the slot name

Yes, I changed the slot names so they include slot index instead of
relation id.
This was needed because I aimed to separate replication slots from
relations.

I think that won't work because each time on restart the slot won't be

fixed. Now, it is possible that we may drop the wrong slot if that
state of copying rel is SUBREL_STATE_DATASYNC. Also, it is possible
that while creating a slot, we fail because the same name slot already
exists due to some other worker which has created that slot has been
restarted. Also, what about origin_name, won't that have similar
problems? Also, if the state is already SUBREL_STATE_FINISHEDCOPY, if
the slot is not the same as we have used in the previous run of a
particular worker, it may start WAL streaming from a different point
based on the slot's confirmed_flush_location.

You're right Amit. In case of a failure, tablesync phase of a relation may
continue with different worker and replication slot due to this change in
naming.
Seems like the same replication slot should be used from start to end for a
relation during tablesync. However, creating/dropping replication slots can
be a major overhead in some cases.
It would be nice if these slots are somehow reused.

To overcome this issue, I've been thinking about making some changes in my
patch.
So far, my proposal would be as follows:

Slot naming can be like pg_<sub_id>_<worker_pid> instead of
pg_<sub_id>_<slot_index>. This way each worker can use the same replication
slot during their lifetime.
But if a worker is restarted, then it will switch to a new replication slot
since its pid has changed.

pg_subscription_rel catalog can store replication slot name for each
non-ready relation. Then we can find the slot needed for that particular
relation to complete tablesync.
If a worker syncs a relation without any error, everything works well and
this new replication slot column from the catalog will not be needed.
However if a worker is restarted due to a failure, the previous run of that
worker left its slot behind since it did not exit properly.
And the restarted worker (with a different pid) will see that the relation
is actually in SUBREL_STATE_FINISHEDCOPY and want to proceed for the
catchup step.
Then the worker can look for that particular relation's replication slot
from pg_subscription_rel catalog (slot name should be there since relation
state is not ready). And tablesync can proceed with that slot.

There might be some cases where some replication slots are left behind. An
example to such cases would be when the slot is removed from
pg_subscription_rel catalog and detached from any relation, but tha slot
actually couldn't be dropped for some reason. For such cases, a slot
cleanup logic is needed. This cleanup can also be done by tablesync workers.
Whenever a tablesync worker is created, it can look for existing
replication slots that do not belong to any relation and any worker (slot
name has pid for that), and drop those slots if it finds any.

What do you think about this new way of handling slots? Do you see any
points of concern?

I'm currently working on adding this change into the patch. And would
appreciate any comment.

Thanks,
Melih

#7Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Amit Kapila (#2)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

It seems from your results that performance degrades for large
relations. Did you try to investigate the reasons for the same?

I have not tried to investigate the performance degradation for large
relations yet.
Once I'm done with changes for the slot usage, I'll look into this and come
with more findings.

Thanks,
Melih

#8Amit Kapila
amit.kapila16@gmail.com
In reply to: Melih Mutlu (#6)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Fri, Jul 8, 2022 at 10:26 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

I think that won't work because each time on restart the slot won't be
fixed. Now, it is possible that we may drop the wrong slot if that
state of copying rel is SUBREL_STATE_DATASYNC. Also, it is possible
that while creating a slot, we fail because the same name slot already
exists due to some other worker which has created that slot has been
restarted. Also, what about origin_name, won't that have similar
problems? Also, if the state is already SUBREL_STATE_FINISHEDCOPY, if
the slot is not the same as we have used in the previous run of a
particular worker, it may start WAL streaming from a different point
based on the slot's confirmed_flush_location.

You're right Amit. In case of a failure, tablesync phase of a relation may continue with different worker and replication slot due to this change in naming.
Seems like the same replication slot should be used from start to end for a relation during tablesync. However, creating/dropping replication slots can be a major overhead in some cases.
It would be nice if these slots are somehow reused.

To overcome this issue, I've been thinking about making some changes in my patch.
So far, my proposal would be as follows:

Slot naming can be like pg_<sub_id>_<worker_pid> instead of pg_<sub_id>_<slot_index>. This way each worker can use the same replication slot during their lifetime.
But if a worker is restarted, then it will switch to a new replication slot since its pid has changed.

I think using worker_pid also has similar risks of mixing slots from
different workers because after restart same worker_pid could be
assigned to a totally different worker. Can we think of using a unique
64-bit number instead? This will be allocated when each workers
started for the very first time and after that we can refer catalog to
find it as suggested in the idea below.

pg_subscription_rel catalog can store replication slot name for each non-ready relation. Then we can find the slot needed for that particular relation to complete tablesync.

Yeah, this is worth investigating. However, instead of storing the
slot_name, we can store just the unique number (as suggested above).
We should use the same for the origin name as well.

If a worker syncs a relation without any error, everything works well and this new replication slot column from the catalog will not be needed.
However if a worker is restarted due to a failure, the previous run of that worker left its slot behind since it did not exit properly.
And the restarted worker (with a different pid) will see that the relation is actually in SUBREL_STATE_FINISHEDCOPY and want to proceed for the catchup step.
Then the worker can look for that particular relation's replication slot from pg_subscription_rel catalog (slot name should be there since relation state is not ready). And tablesync can proceed with that slot.

There might be some cases where some replication slots are left behind. An example to such cases would be when the slot is removed from pg_subscription_rel catalog and detached from any relation, but tha slot actually couldn't be dropped for some reason. For such cases, a slot cleanup logic is needed. This cleanup can also be done by tablesync workers.
Whenever a tablesync worker is created, it can look for existing replication slots that do not belong to any relation and any worker (slot name has pid for that), and drop those slots if it finds any.

This sounds tricky. Why not first drop slot/origin and then detach it
from pg_subscription_rel? On restarts, it is possible that we may
error out after dropping the slot or origin but before updating the
catalog entry but in such case we can ignore missing slot/origin and
detach them from pg_subscription_rel. Also, if we use the unique
number as suggested above, I think even if we don't remove it after
the relation state is ready, it should be okay.

What do you think about this new way of handling slots? Do you see any points of concern?

I'm currently working on adding this change into the patch. And would appreciate any comment.

Thanks for making progress!

--
With Regards,
Amit Kapila.

#9Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Amit Kapila (#8)
1 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Amit,

I updated the patch in order to prevent the problems that might be caused
by using different replication slots for syncing a table.
As suggested in previous emails, replication slot names are stored in the
catalog. So slot names can be reached later and it is ensured
that same replication slot is used during tablesync step of a table.

With the current version of the patch:
-. "srrelslotname" column is introduced into pg_subscibtion_rel catalog. It
stores the slot name for tablesync

-. Tablesync worker logic is now as follows:
1. Tablesync worker is launched by apply worker for a table.
2. Worker generates a default replication slot name for itself. Slot name
includes subid and worker pid for tracking purposes.
3. If table has a slot name value in the catalog:

i. If the table state is DATASYNC, drop the replication slot from the
catalog and proceed tablesync with a new slot.

ii. If the table state is FINISHEDCOPY, use the replicaton slot from the
catalog, do not create a new slot.

4. Before worker moves to new table, drop any replication slot that are
retrieved from the catalog and used.
5. In case of no table left to sync, drop the replication slot of that sync
worker with worker pid if it exists. (It's possible that a sync worker do
not create a replication slot for itself but uses slots read from the
catalog in each iteration)

I think using worker_pid also has similar risks of mixing slots from

different workers because after restart same worker_pid could be
assigned to a totally different worker. Can we think of using a unique
64-bit number instead? This will be allocated when each workers
started for the very first time and after that we can refer catalog to
find it as suggested in the idea below.

I'm not sure how likely to have colliding pid's for different tablesync
workers in the same subscription.
Though ,having pid in slot name makes it easier to track which slot belongs
to which worker. That's why I kept using pid in slot names.
But I think it should be simple to switch to using a unique 64-bit number.
So I can remove pid's from slot names, if you think that it would be
better.

We should use the same for the origin name as well.

I did not really change anything related to origin names. Origin names are
still the same and include relation id. What do you think would be an issue
with origin names in this patch?

This sounds tricky. Why not first drop slot/origin and then detach it
from pg_subscription_rel? On restarts, it is possible that we may
error out after dropping the slot or origin but before updating the
catalog entry but in such case we can ignore missing slot/origin and
detach them from pg_subscription_rel. Also, if we use the unique
number as suggested above, I think even if we don't remove it after
the relation state is ready, it should be okay.

Right, I did not add an additional slot cleanup step. The patch now drops
the slot when we're done with it and then removes it from the catalog.

Thanks,
Melih

Attachments:

v2-0001-Reuse-Logical-Replication-Background-worker.patchapplication/octet-stream; name=v2-0001-Reuse-Logical-Replication-Background-worker.patchDownload
From a4e9a10b695155111bcb32058dc35ef7f1aeb22f Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH] Reuse Logical Replication Background worker

This commit allows tablesync workers to move to another table that needs synchronization,
when they're done with the curren table in tablesync phase of Logical
Replication.

It reduces the overhead of launching/killing a new background worker for
each table. By reusing tablesync workers, replication slots created for
tablesync can be reused as well. Removing the burden of
creating/dropping replication slot improves tablesync speed.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/catalogs.sgml                  |   9 +
 src/backend/catalog/pg_subscription.c       | 241 +++++++++++++-
 src/backend/commands/subscriptioncmds.c     | 168 ++++++----
 src/backend/replication/logical/launcher.c  |   4 +
 src/backend/replication/logical/tablesync.c | 202 +++++++++---
 src/backend/replication/logical/worker.c    | 345 ++++++++++++--------
 src/include/catalog/pg_subscription_rel.h   |  11 +-
 src/include/replication/slot.h              |   3 +-
 src/include/replication/worker_internal.h   |  20 ++
 src/test/regress/expected/misc_sanity.out   |   3 +-
 10 files changed, 766 insertions(+), 240 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index a186e35f00..39b5da4710 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8041,6 +8041,15 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        otherwise null
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srrelslotname</structfield> <type>text</type>
+      </para>
+      <para>
+       Replication slot name that is used for synchronization of relation 
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 33ae3da8ae..e39cee30fe 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -284,7 +284,7 @@ textarray_to_stringlist(ArrayType *textarray)
  */
 void
 AddSubscriptionRelState(Oid subid, Oid relid, char state,
-						XLogRecPtr sublsn)
+						XLogRecPtr sublsn, char *relslotname)
 {
 	Relation	rel;
 	HeapTuple	tup;
@@ -313,9 +313,17 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
 	else
 		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (relslotname)
+	{
+		values[Anum_pg_subscription_rel_srrelslotname - 1] = CStringGetTextDatum(relslotname);
+	}
+	else
+	{
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	}
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
-
+ 
 	/* Insert tuple into catalog. */
 	CatalogTupleInsert(rel, tup);
 
@@ -325,6 +333,42 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Internal function to modify columns for relation state update
+ */
+static void
+UpdateSubscriptionRelState_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char state,
+						   			XLogRecPtr sublsn)
+{
+	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
+	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+
+	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (sublsn != InvalidXLogRecPtr)
+		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
+	else
+		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication slot update
+ */
+static void
+UpdateSubscriptionRelReplicationSlot_internal(Datum *values,
+											bool *nulls,
+											bool *replaces,
+											char *relslotname)
+{
+	replaces[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] = CStringGetTextDatum(relslotname);
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+}
+
 /*
  * Update the state of a subscription table.
  */
@@ -355,14 +399,94 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	memset(nulls, false, sizeof(nulls));
 	memset(replaces, false, sizeof(replaces));
 
-	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
-	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
 
-	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
-	if (sublsn != InvalidXLogRecPtr)
-		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
-	else
-		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update the replication slot name of a subscription table.
+ */
+void
+UpdateSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *relslotname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
+
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update replication slot name and state of a subscription table in one transaction.
+ */
+void
+UpdateSubscriptionRelStateAndSlot(Oid subid, 
+								Oid relid, 
+								char state,
+						   		XLogRecPtr sublsn,
+								char *relslotname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
 
 	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
 							replaces);
@@ -374,6 +498,46 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Get replication slot name of subscription table.
+ *
+ * Returns null if the subscription table does not have a replication slot.
+ */
+void
+GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*relrepslot;
+	bool		isnull;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srrelslotname, &isnull);
+	if (!isnull)
+	{
+		relrepslot = TextDatumGetCString(d);
+		memcpy(slotname, relrepslot, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
 /*
  * Get state of subscription table.
  *
@@ -643,3 +807,62 @@ GetSubscriptionNotReadyRelations(Oid subid)
 
 	return res;
 }
+
+/*
+ * Get all relations for subscription that are in init state.
+ *
+ * Returned list is palloc'ed in current memory context.
+ */
+List *
+GetSubscriptionInitStateRelations(Oid subid)
+{
+	List	   *res = NIL;
+	Relation	rel;
+	HeapTuple	tup;
+	int			nkeys = 0;
+	ScanKeyData skey[2];
+	SysScanDesc scan;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	ScanKeyInit(&skey[nkeys++],
+				Anum_pg_subscription_rel_srsubid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(subid));
+
+	ScanKeyInit(&skey[nkeys++],
+				Anum_pg_subscription_rel_srsubstate,
+				BTEqualStrategyNumber, F_CHAREQ,
+				CharGetDatum(SUBREL_STATE_INIT));
+
+	scan = systable_beginscan(rel, InvalidOid, false,
+							  NULL, nkeys, skey);
+
+	while (HeapTupleIsValid(tup = systable_getnext(scan)))
+	{
+		Form_pg_subscription_rel subrel;
+		SubscriptionRelState *relstate;
+		Datum		d;
+		bool		isnull;
+
+		subrel = (Form_pg_subscription_rel) GETSTRUCT(tup);
+
+		relstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
+		relstate->relid = subrel->srrelid;
+		relstate->state = subrel->srsubstate;
+		d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+							Anum_pg_subscription_rel_srsublsn, &isnull);
+		if (isnull)
+			relstate->lsn = InvalidXLogRecPtr;
+		else
+			relstate->lsn = DatumGetLSN(d);
+
+		res = lappend(res, relstate);
+	}
+
+	/* Cleanup */
+	systable_endscan(scan);
+	table_close(rel, AccessShareLock);
+
+	return res;
+}
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index bd0cc0848d..fd1cea16b9 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -704,7 +704,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 										 rv->schemaname, rv->relname);
 
 				AddSubscriptionRelState(subid, relid, table_state,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL);
 			}
 
 			/*
@@ -794,6 +794,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	} SubRemoveRels;
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -866,7 +868,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				AddSubscriptionRelState(sub->oid, relid,
 										copy_data ? SUBREL_STATE_INIT : SUBREL_STATE_READY,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL);
 				ereport(DEBUG1,
 						(errmsg_internal("table \"%s.%s\" added to subscription \"%s\"",
 										 rv->schemaname, rv->relname, sub->name)));
@@ -916,7 +918,18 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/* 
+				 * Find the logical replication sync worker if exists 
+				 * Store the slot number for dropping associated replication slot later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot_name);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
@@ -950,31 +963,23 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
+		 * Drop the replication slots associated with tablesync workers for removed tables.
+		 * This has to be at the end because otherwise if there is an error while doing
 		 * the database operations we won't be able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
+			memcpy(syncslotname, lfirst(lc), sizeof(NAMEDATALEN));
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			/*
+			 * There is no certainty, maybe the slot
+			 * does not exist yet. Also, if we fail after removing some of
+			 * the slots, next time, it will again try to drop already
+			 * dropped slots and fail. For these reasons, we allow
+			 * missing_ok = true for the drop.
+			 */
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1567,39 +1572,16 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
-
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
-
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		List *slots = NULL;
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+		
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char *syncslotname = (char *) lfirst(lc);
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1622,6 +1604,69 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * WRITE COMMENT HERE
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok){
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+						" FROM pg_replication_slots"
+						" WHERE slot_name LIKE 'pg_%i_sync_%%';",
+						 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+					 errmsg("not tuple returned."));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();\
+	return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -1863,6 +1908,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -1873,18 +1919,20 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+		
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
-			char		syncslotname[NAMEDATALEN] = {0};
-
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
-				 syncslotname);
+				 worker->slot_name);
 		}
 	}
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 3bbd522724..b673bc9d27 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -370,7 +370,10 @@ retry:
 	/* Prepare the worker slot. */
 	worker->launch_time = now;
 	worker->in_use = true;
+	worker->is_first_run = true;
 	worker->generation++;
+	worker->created_slot = false;
+	worker->slot_name = (char *) palloc(NAMEDATALEN);
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -378,6 +381,7 @@ retry:
 	worker->relid = relid;
 	worker->relstate = SUBREL_STATE_UNKNOWN;
 	worker->relstate_lsn = InvalidXLogRecPtr;
+	worker->move_to_next_rel = false;
 	worker->stream_fileset = NULL;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 670c6fcada..b62dcf661e 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -126,12 +126,8 @@ static bool FetchTableStates(bool *started_tx);
 
 static StringInfo copybuf = NULL;
 
-/*
- * Exit routine for synchronization worker.
- */
 static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+clean_sync_worker(void)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -143,18 +139,27 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
-	/* And flush all writes. */
-	XLogFlush(GetXLogWriteRecPtr());
-
-	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
-	CommitTransactionCommand();
+	/* Disconnect from publisher.
+	 * Otherwise reused sync workers causes exceeding max_wal_senders 
+	 */
+	walrcv_disconnect(LogRepWorkerWalRcvConn);
+	LogRepWorkerWalRcvConn = NULL;
 
 	/* Find the main apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
+}
+
+/*
+ * Exit routine for synchronization worker.
+ */
+static void
+pg_attribute_noreturn()
+finish_sync_worker(void)
+{
+	clean_sync_worker();
+	
+	/* And flush all writes. */
+	XLogFlush(GetXLogWriteRecPtr());
 
 	/* Stop gracefully */
 	proc_exit(0);
@@ -180,7 +185,7 @@ wait_for_relation_state_change(Oid relid, char expected_state)
 		LogicalRepWorker *worker;
 		XLogRecPtr	statelsn;
 
-		CHECK_FOR_INTERRUPTS();
+		CHECK_FOR_INTERRUPTS();		
 
 		InvalidateCatalogSnapshot();
 		state = GetSubscriptionRelState(MyLogicalRepWorker->subid,
@@ -284,6 +289,10 @@ invalidate_syncing_table_states(Datum arg, int cacheid, uint32 hashvalue)
 static void
 process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 {
+	List	   *rstates;
+	SubscriptionRelState *rstate;
+	ListCell   *lc;
+
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_CATCHUP &&
@@ -299,7 +308,6 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 
 		/*
 		 * UpdateSubscriptionRelState must be called within a transaction.
-		 * That transaction will be ended within the finish_sync_worker().
 		 */
 		if (!IsTransactionState())
 			StartTransactionCommand();
@@ -308,6 +316,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 								   MyLogicalRepWorker->relid,
 								   MyLogicalRepWorker->relstate,
 								   MyLogicalRepWorker->relstate_lsn);
+		CommitTransactionCommand();
 
 		/*
 		 * End streaming so that LogRepWorkerWalRcvConn can be used to drop
@@ -317,24 +326,86 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 
 		/*
 		 * Cleanup the tablesync slot.
+		 * If the slot name used by this worker is different from the default 
+		 * slot name for the worker, this means the current table had started to being 
+		 * synchronized by another worker and replication slot. And this worker
+		 * is reusing a replication slot from a previous attempt.
+		 * We do not need that replication slot anymore.
 		 *
 		 * This has to be done after updating the state because otherwise if
 		 * there is an error while doing the database operations we won't be
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
 										syncslotname,
 										sizeof(syncslotname));
 
+
+		/* This transaction will be ended within the clean_sync_worker(). */
+		StartTransactionCommand();
+		if (MyLogicalRepWorker->slot_name && strcmp(syncslotname, MyLogicalRepWorker->slot_name) != 0)
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
+			UpdateSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+												 MyLogicalRepWorker->relid,
+												 NULL);
+		}
+
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
+		 * Check if any table whose relation state is still INIT. 
+		 * If a table in INIT state is found, the worker will not be finished,
+		 * it will be reused instead.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
+		rstates = GetSubscriptionInitStateRelations(MySubscription->oid);
+		
+		foreach (lc, rstates)
+		{
+			rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
+			memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
 
-		finish_sync_worker();
+			/* 
+			 * Pick the table for the next run
+			 * if there is not another worker already picked that table.
+			 */
+			LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+			if (!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+			{
+				ereport(LOG,
+						(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
+								MySubscription->name,
+								get_rel_name(MyLogicalRepWorker->relid))));
+
+				/* Update worker state for the next table */
+				MyLogicalRepWorker->is_first_run = false;
+				MyLogicalRepWorker->relid = rstate->relid;
+				MyLogicalRepWorker->relstate = rstate->state;
+				MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+				MyLogicalRepWorker->move_to_next_rel = true;
+				LWLockRelease(LogicalRepWorkerLock);
+				break;
+			}
+			LWLockRelease(LogicalRepWorkerLock);
+		}
+
+		/* Cleanup before next run or ending the worker. */
+		if(!MyLogicalRepWorker->move_to_next_rel)
+		{
+		   /*
+			* It is important to give an error if we are unable to drop the slot,
+			* otherwise, it won't be dropped till the corresponding subscription
+			* is dropped. So passing missing_ok = false.
+			*/
+			if (MyLogicalRepWorker->created_slot)
+			{
+				ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
+			}
+
+			finish_sync_worker();
+		}
+		else
+		{
+			clean_sync_worker();
+		}
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -473,9 +544,9 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				/*
 				 * Update the state to READY only after the origin cleanup.
 				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
+				UpdateSubscriptionRelStateAndSlot(MyLogicalRepWorker->subid,
 										   rstate->relid, rstate->state,
-										   rstate->lsn);
+										   rstate->lsn, NULL);
 			}
 		}
 		else
@@ -1140,8 +1211,8 @@ copy_table(Relation rel)
  * The name must not exceed NAMEDATALEN - 1 because of remote node constraints
  * on slot name length. We append system_identifier to avoid slot_name
  * collision with subscriptions in other clusters. With the current scheme
- * pg_%u_sync_%u_UINT64_FORMAT (3 + 10 + 6 + 10 + 20 + '\0'), the maximum
- * length of slot_name will be 50.
+ * pg_%u_sync_%i_UINT64_FORMAT (3 + 10 + 6 + 5 + 20 + '\0'), the maximum
+ * length of slot_name will be 45.
  *
  * The returned slot name is stored in the supplied buffer (syncslotname) with
  * the given size.
@@ -1152,11 +1223,10 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
-								char *syncslotname, int szslot)
+ReplicationSlotNameForTablesync(Oid suboid, char *syncslotname, int szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%i_" UINT64_FORMAT, suboid,
+			 MyProcPid, GetSystemIdentifier());
 }
 
 /*
@@ -1191,6 +1261,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	WalRcvExecResult *res;
 	char		originname[NAMEDATALEN];
 	RepOriginId originid;
+	char  		*prev_slotname;
 
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
@@ -1219,7 +1290,6 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
 									slotname,
 									NAMEDATALEN);
 
@@ -1245,6 +1315,20 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 									  originname,
 									  sizeof(originname));
 
+
+	/*
+	 * See if tablesync of the current relation has been 
+	 * started with another replication slot. 
+	 * 
+	 * Read previous slot name from the catalog, if exists.
+	 */
+	prev_slotname = (char *) palloc0(NAMEDATALEN);
+	StartTransactionCommand();
+	GetSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+									  MyLogicalRepWorker->relid,
+									  prev_slotname);
+	CommitTransactionCommand();
+
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC)
 	{
 		/*
@@ -1258,10 +1342,34 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		 * breakdown then it wouldn't have succeeded so trying it next time
 		 * seems like a better bet.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, slotname, true);
+		if (strcmp(prev_slotname, ""))
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);
+		
+			StartTransactionCommand();
+			UpdateSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+												  MyLogicalRepWorker->relid,
+												  NULL);
+			CommitTransactionCommand();
+		}
 	}
 	else if (MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY)
 	{
+		/*
+		 * At this point, the table that is currently being synchronized should have
+		 * its replication slot name filled in the catalog. The tablesync process was started 
+		 * with another sync worker and replication slot.
+		 * We need to continue using the same replication slot in this worker too.
+		 */
+		if (!strcmp(prev_slotname, ""))
+		{
+			elog(ERROR, "Replication slot could not be found for relation %u",
+				 MyLogicalRepWorker->relid);
+		}
+
+		/* Proceed with the correct replication slot. */
+		slotname = prev_slotname;
+
 		/*
 		 * The COPY phase was previously done, but tablesync then crashed
 		 * before it was able to finish normally.
@@ -1289,10 +1397,11 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
+	UpdateSubscriptionRelStateAndSlot(MyLogicalRepWorker->subid,
 							   MyLogicalRepWorker->relid,
 							   MyLogicalRepWorker->relstate,
-							   MyLogicalRepWorker->relstate_lsn);
+							   MyLogicalRepWorker->relstate_lsn,
+							   slotname);
 	CommitTransactionCommand();
 	pgstat_report_stat(true);
 
@@ -1351,16 +1460,31 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	 * for the catchup phase after COPY is done, so tell it to use the
 	 * snapshot to make the final data consistent.
 	 *
+	 * Replication slot will only be created if either this is the first run 
+	 * of the worker or we're not using a previous replication slot. 
+	 * 
 	 * Prevent cancel/die interrupts while creating slot here because it is
 	 * possible that before the server finishes this command, a concurrent
 	 * drop subscription happens which would complete without removing this
 	 * slot leading to a dangling slot on the server.
 	 */
-	HOLD_INTERRUPTS();
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-	RESUME_INTERRUPTS();
+	if (MyLogicalRepWorker->is_first_run ||
+		(!strcmp(prev_slotname, "") && !MyLogicalRepWorker->created_slot))
+	{
+		HOLD_INTERRUPTS();
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						slotname, false /* permanent */ , false /* two_phase */ ,
+						CRS_USE_SNAPSHOT, origin_startpos);
+		RESUME_INTERRUPTS();
+
+		/*
+		 * Remember that we created the slot so that
+		 * we will not try to create it again.
+		 */
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->created_slot = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
+	}
 
 	/*
 	 * Setup replication origin tracking. The purpose of doing this before the
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 5f8c541763..163c0ea921 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -315,6 +315,7 @@ static void stream_cleanup_files(Oid subid, TransactionId xid);
 static void stream_open_file(Oid subid, TransactionId xid, bool first);
 static void stream_write_change(char action, StringInfo s);
 static void stream_close_file(void);
+static void stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
@@ -2814,6 +2815,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+			if (MyLogicalRepWorker->move_to_next_rel)
+			{
+				endofstream = true;
+			}
 		}
 
 		/* Cleanup the memory. */
@@ -2915,8 +2920,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	/* Pop the error context stack */
 	error_context_stack = errcallback.previous;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/* 
+	 * If it's moving to next relation, this is a sync worker.
+	 * Sync workers end the streaming during process_syncing_tables_for_sync.
+	 * Calling endstreaming twice causes "no COPY in progress" errors.
+	 */
+	if (!MyLogicalRepWorker->move_to_next_rel)
+	{
+		/* All done */
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	}
 }
 
 /*
@@ -3458,6 +3471,35 @@ stream_write_change(char action, StringInfo s)
 	BufFileWrite(stream_fd, &s->data[s->cursor], len);
 }
 
+/*
+ * stream_build_options_replication
+ * 		Build logical replication streaming options.
+ *
+ * This function sets streaming options including replication slot name
+ * and origin start position. Workers need these options for logical replication.
+ */
+static void
+stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos)
+{
+	int server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+	options->proto.logical.streaming = MySubscription->stream;
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -3532,6 +3574,9 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 
 	/* allocate slot name in long-lived context */
 	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+
+	/* Keep the replication slot name used for this sync. */
+	MyLogicalRepWorker->slot_name = *myslotname;
 	pfree(syncslotname);
 }
 
@@ -3569,6 +3614,136 @@ start_apply(XLogRecPtr origin_startpos)
 	PG_END_TRY();
 }
 
+/*
+ * Runs the tablesync worker.
+ * It starts table sync. After successful sync, 
+ * builds streaming options and starts streaming. 
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options, 
+					 char *slotname,
+					 char *originname,
+					 int originame_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Set this to false for safety, in case we're already reusing the worker */
+    MyLogicalRepWorker->move_to_next_rel = false;
+
+    start_table_sync(origin_startpos, &slotname);
+
+    /*
+        * Allocate the origin name in long-lived context for error context
+        * message.
+        */
+    ReplicationOriginNameForTablesync(MySubscription->oid,
+                                        MyLogicalRepWorker->relid,
+                                        originname,
+                                        originame_size);
+    apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
+                                                                originname);
+    
+    stream_build_options(options, slotname, origin_startpos);
+
+    /* Start normal logical streaming replication. */
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+}
+
+/*
+ * Runs the apply worker.
+ * It sets up replication origin, the streaming options 
+ * and then starts streaming. 
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
+{
+    RepOriginId originid;
+    TimeLineID	startpointTLI;
+    char	   *err;
+
+    slotname = MySubscription->slotname;
+
+    /*
+	 * This shouldn't happen if the subscription is enabled, but guard
+	 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+	 * crash if slot is NULL.)
+	 */
+    if (!slotname)
+        ereport(ERROR,
+                (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+                    errmsg("subscription has no replication slot set")));
+
+    /* Setup replication origin tracking. */
+    StartTransactionCommand();
+    snprintf(originname, originname_size, "pg_%u", MySubscription->oid);
+    originid = replorigin_by_name(originname, true);
+    if (!OidIsValid(originid))
+        originid = replorigin_create(originname);
+    replorigin_session_setup(originid);
+    replorigin_session_origin = originid;
+    *origin_startpos = replorigin_session_get_progress(false);
+    CommitTransactionCommand();
+
+    LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+                                            MySubscription->name, &err);
+    if (LogRepWorkerWalRcvConn == NULL)
+        ereport(ERROR,
+                (errcode(ERRCODE_CONNECTION_FAILURE),
+                    errmsg("could not connect to the publisher: %s", err)));
+
+    /*
+	 * We don't really use the output identify_system for anything but it
+	 * does some initializations on the upstream so let's still call it.
+	 */
+    (void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+    /*
+	 * Allocate the origin name in long-lived context for error context
+	 * message.
+	 */
+    apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
+                                                                originname);
+
+    stream_build_options(options, slotname, origin_startpos);
+
+    /*
+     * Even when the two_phase mode is requested by the user, it remains
+     * as the tri-state PENDING until all tablesyncs have reached READY
+     * state. Only then, can it become ENABLED.
+     *
+     * Note: If the subscription has no tables then leave the state as
+     * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+     * work.
+     */
+    if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+        AllTablesyncsReady())
+    {
+        /* Start streaming with two_phase enabled */
+        options->proto.logical.twophase = true;
+        walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+        StartTransactionCommand();
+        UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+        MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+        CommitTransactionCommand();
+    }
+    else
+    {
+        walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+    }
+
+    ereport(DEBUG1,
+            (errmsg("logical replication apply worker for subscription \"%s\" two_phase is %s",
+                    MySubscription->name,
+                    MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+                    MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+                    MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+                    "?")));
+}
+
 /* Logical Replication Apply worker entry point */
 void
 ApplyWorkerMain(Datum main_arg)
@@ -3579,7 +3754,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	/* Attach to slot */
 	logicalrep_worker_attach(worker_slot);
@@ -3670,142 +3844,55 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		/*
-		 * Allocate the origin name in long-lived context for error context
-		 * message.
-		 */
-		ReplicationOriginNameForTablesync(MySubscription->oid,
-										  MyLogicalRepWorker->relid,
-										  originname,
-										  sizeof(originname));
-		apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
-																   originname);
-	}
-	else
-	{
-		/* This is main apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* 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);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		/*
-		 * Allocate the origin name in long-lived context for error context
-		 * message.
-		 */
-		apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
-																   originname);
-	}
-
 	/*
-	 * Setup callback for syscache so that we know when something changes in
-	 * the subscription relation state.
-	 */
+	* Setup callback for syscache so that we know when something changes in
+	* the subscription relation state.
+	* Do this outside the loop to avoid exceeding MAX_SYSCACHE_CALLBACKS
+	*/
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
-								  invalidate_syncing_table_states,
-								  (Datum) 0);
-
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
-
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
-
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-	options.proto.logical.streaming = MySubscription->stream;
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
+								invalidate_syncing_table_states,
+								(Datum) 0);
 
-	if (!am_tablesync_worker())
+	/*
+	 * The loop where worker does its job.
+	 * It loops until the worker is not reused. 
+	 */
+	while (MyLogicalRepWorker->is_first_run || 
+			MyLogicalRepWorker->move_to_next_rel)
 	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
+		if (am_tablesync_worker())
+			{
+				/* 
+				* This is a tablesync worker. 
+				* Start syncing tables before starting the apply loop.  
+				*/
+				run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
+			}
+			else
+			{
+				/* This is main apply worker */
+				run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
+			}
+		
+		/* Run the main loop. */
+		start_apply(origin_startpos);
+
+		if (MyLogicalRepWorker->move_to_next_rel)
 		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+			/* Reset the currenct replication origin session.
+			* Since we'll use the same process for another relation, it needs to be reset 
+			* and will be created again later while syncing the new relation.
+			*/
+			replorigin_session_origin = InvalidRepOriginId;
+			replorigin_session_reset();
 
 			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+			ereport(LOG,
+					(errmsg("logical replication table synchronization worker for subscription \"%s\" has moved to sync table \"%s\".",
+							MySubscription->name, get_rel_name(MyLogicalRepWorker->relid))));
 			CommitTransactionCommand();
 		}
-		else
-		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-		}
-
-		ereport(DEBUG1,
-				(errmsg("logical replication apply worker for subscription \"%s\" two_phase is %s",
-						MySubscription->name,
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-						"?")));
 	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
-
-	/* Run the main loop. */
-	start_apply(origin_startpos);
-
 	proc_exit(0);
 }
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 9df99c3418..b127817b66 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -44,6 +44,8 @@ CATALOG(pg_subscription_rel,6102,SubscriptionRelRelationId)
 											 * used for synchronization
 											 * coordination, or NULL if not
 											 * valid */
+	text		srrelslotname BKI_FORCE_NULL;	/* name of the replication slot for relatio in subscription */
+
 #endif
 } FormData_pg_subscription_rel;
 
@@ -81,14 +83,21 @@ typedef struct SubscriptionRelState
 } SubscriptionRelState;
 
 extern void AddSubscriptionRelState(Oid subid, Oid relid, char state,
-									XLogRecPtr sublsn);
+									XLogRecPtr sublsn, char *relslotname);
 extern void UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 									   XLogRecPtr sublsn);
 extern char GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn);
 extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 
+extern void UpdateSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *relslotname);
+extern void GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname);
+
+extern void UpdateSubscriptionRelStateAndSlot(Oid subid, Oid relid, char state,
+									XLogRecPtr sublsn, char *relslotname);
+
 extern bool HasSubscriptionRelations(Oid subid);
 extern List *GetSubscriptionRelations(Oid subid);
 extern List *GetSubscriptionNotReadyRelations(Oid subid);
+extern List *GetSubscriptionInitStateRelations(Oid subid);
 
 #endif							/* PG_SUBSCRIPTION_REL_H */
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 8c9f3321d5..6bdf1d1238 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -217,8 +217,9 @@ extern void ReplicationSlotsDropDBSlots(Oid dboid);
 extern bool InvalidateObsoleteReplicationSlots(XLogSegNo oldestSegno);
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, int szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, char *syncslotname, int szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List* GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 901845abc2..2180ffca11 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -30,6 +30,21 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/* 
+	 * Indicates if worker is running for the first time
+	 * or in reuse
+	 */
+	bool		is_first_run;
+
+	/* 
+	 * Indicates if the sync worker created a replication slot
+	 * or it reuses an existing one created by another worker.
+	 */
+	bool		created_slot;
+
+	/* Replication slot name used by the worker. */
+	char		*slot_name;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
@@ -51,6 +66,11 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/* 
+	 * Used to indicate whether sync worker will be reused for another relation
+	 */
+	bool		move_to_next_rel;
+	
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction, the
diff --git a/src/test/regress/expected/misc_sanity.out b/src/test/regress/expected/misc_sanity.out
index a57fd142a9..a2a6f14944 100644
--- a/src/test/regress/expected/misc_sanity.out
+++ b/src/test/regress/expected/misc_sanity.out
@@ -60,7 +60,8 @@ ORDER BY 1, 2;
  pg_index                | indpred       | pg_node_tree
  pg_largeobject          | data          | bytea
  pg_largeobject_metadata | lomacl        | aclitem[]
-(11 rows)
+ pg_subscription_rel     | srrelslotname | text
+(12 rows)
 
 -- system catalogs without primary keys
 --
-- 
2.25.1

#10Amit Kapila
amit.kapila16@gmail.com
In reply to: Melih Mutlu (#9)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Wed, Jul 27, 2022 at 3:56 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi Amit,

I updated the patch in order to prevent the problems that might be caused by using different replication slots for syncing a table.
As suggested in previous emails, replication slot names are stored in the catalog. So slot names can be reached later and it is ensured
that same replication slot is used during tablesync step of a table.

With the current version of the patch:
-. "srrelslotname" column is introduced into pg_subscibtion_rel catalog. It stores the slot name for tablesync

-. Tablesync worker logic is now as follows:
1. Tablesync worker is launched by apply worker for a table.
2. Worker generates a default replication slot name for itself. Slot name includes subid and worker pid for tracking purposes.
3. If table has a slot name value in the catalog:

i. If the table state is DATASYNC, drop the replication slot from the catalog and proceed tablesync with a new slot.

ii. If the table state is FINISHEDCOPY, use the replicaton slot from the catalog, do not create a new slot.

4. Before worker moves to new table, drop any replication slot that are retrieved from the catalog and used.

Why after step 4, do you need to drop the replication slot? Won't just
clearing the required info from the catalog be sufficient?

5. In case of no table left to sync, drop the replication slot of that sync worker with worker pid if it exists. (It's possible that a sync worker do not create a replication slot for itself but uses slots read from the catalog in each iteration)

I think using worker_pid also has similar risks of mixing slots from
different workers because after restart same worker_pid could be
assigned to a totally different worker. Can we think of using a unique
64-bit number instead? This will be allocated when each workers
started for the very first time and after that we can refer catalog to
find it as suggested in the idea below.

I'm not sure how likely to have colliding pid's for different tablesync workers in the same subscription.

Hmm, I think even if there is an iota of a chance which I think is
there, we can't use worker_pid. Assume, that if the same worker_pid is
assigned to another worker once the worker using it got an error out,
the new worker will fail as soon as it will try to create a
replication slot.

Though ,having pid in slot name makes it easier to track which slot belongs to which worker. That's why I kept using pid in slot names.
But I think it should be simple to switch to using a unique 64-bit number. So I can remove pid's from slot names, if you think that it would be better.

I feel it would be better or maybe we need to think of some other
identifier but one thing we need to think about before using a 64-bit
unique identifier here is how will we retrieve its last used value
after restart of server. We may need to store it in a persistent way
somewhere.

We should use the same for the origin name as well.

I did not really change anything related to origin names. Origin names are still the same and include relation id. What do you think would be an issue with origin names in this patch?

The problems will be similar to the slot name. The origin is used to
track the progress of replication, so, if we use the wrong origin name
after the restart, it can send the wrong start_streaming position to
the publisher.

--
With Regards,
Amit Kapila.

#11Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Amit Kapila (#10)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Why after step 4, do you need to drop the replication slot? Won't just
clearing the required info from the catalog be sufficient?

The replication slots that we read from the catalog will not be used for
anything else after we're done with syncing the table which the rep slot
belongs to.
It's removed from the catalog when the sync is completed and it basically
becomes a slot that is not linked to any table or worker. That's why I
think it should be dropped rather than left behind.

Note that if a worker dies and its replication slot continues to exist,
that slot will only be used to complete the sync process of the one table
that the dead worker was syncing but couldn't finish.
When that particular table is synced and becomes ready, the replication
slot has no use anymore.

Hmm, I think even if there is an iota of a chance which I think is
there, we can't use worker_pid. Assume, that if the same worker_pid is
assigned to another worker once the worker using it got an error out,
the new worker will fail as soon as it will try to create a
replication slot.

Right. If something like that happens, worker will fail without doing
anything. Then a new one will be launched and that one will continue to
do the work.
The worst case might be having conflicting pid over and over again while
also having replication slots whose name includes one of those pids still
exist.
It seems unlikely but possible, yes.

I feel it would be better or maybe we need to think of some other
identifier but one thing we need to think about before using a 64-bit
unique identifier here is how will we retrieve its last used value
after restart of server. We may need to store it in a persistent way
somewhere.

We might consider storing this info in a catalog again. Since this last
used value will be different for each subscription, pg_subscription can be
a good place to keep that.

The problems will be similar to the slot name. The origin is used to
track the progress of replication, so, if we use the wrong origin name
after the restart, it can send the wrong start_streaming position to
the publisher.

I understand. But origin naming logic is still the same. Its format is like
pg_<subid>_<relid> .
I did not need to change this since it seems to me origins should belong to
only one table. The patch does not reuse origins.
So I don't think this change introduces an issue with origin. What do you
think?

Thanks,
Melih

#12Amit Kapila
amit.kapila16@gmail.com
In reply to: Melih Mutlu (#11)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Thu, Jul 28, 2022 at 9:32 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Why after step 4, do you need to drop the replication slot? Won't just
clearing the required info from the catalog be sufficient?

The replication slots that we read from the catalog will not be used for anything else after we're done with syncing the table which the rep slot belongs to.
It's removed from the catalog when the sync is completed and it basically becomes a slot that is not linked to any table or worker. That's why I think it should be dropped rather than left behind.

Note that if a worker dies and its replication slot continues to exist, that slot will only be used to complete the sync process of the one table that the dead worker was syncing but couldn't finish.
When that particular table is synced and becomes ready, the replication slot has no use anymore.

Why can't it be used to sync the other tables if any?

Hmm, I think even if there is an iota of a chance which I think is
there, we can't use worker_pid. Assume, that if the same worker_pid is
assigned to another worker once the worker using it got an error out,
the new worker will fail as soon as it will try to create a
replication slot.

Right. If something like that happens, worker will fail without doing anything. Then a new one will be launched and that one will continue to do the work.
The worst case might be having conflicting pid over and over again while also having replication slots whose name includes one of those pids still exist.
It seems unlikely but possible, yes.

I feel it would be better or maybe we need to think of some other
identifier but one thing we need to think about before using a 64-bit
unique identifier here is how will we retrieve its last used value
after restart of server. We may need to store it in a persistent way
somewhere.

We might consider storing this info in a catalog again. Since this last used value will be different for each subscription, pg_subscription can be a good place to keep that.

This sounds reasonable. Let's do this unless we get some better idea.

The problems will be similar to the slot name. The origin is used to
track the progress of replication, so, if we use the wrong origin name
after the restart, it can send the wrong start_streaming position to
the publisher.

I understand. But origin naming logic is still the same. Its format is like pg_<subid>_<relid> .
I did not need to change this since it seems to me origins should belong to only one table. The patch does not reuse origins.
So I don't think this change introduces an issue with origin. What do you think?

There is no such restriction that origins should belong to only one
table. What makes you think like that?

--
With Regards,
Amit Kapila.

#13Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Amit Kapila (#12)
1 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Amit,

Why after step 4, do you need to drop the replication slot? Won't just

clearing the required info from the catalog be sufficient?

The replication slots that we read from the catalog will not be used for

anything else after we're done with syncing the table which the rep slot
belongs to.

It's removed from the catalog when the sync is completed and it

basically becomes a slot that is not linked to any table or worker. That's
why I think it should be dropped rather than left behind.

Note that if a worker dies and its replication slot continues to exist,

that slot will only be used to complete the sync process of the one table
that the dead worker was syncing but couldn't finish.

When that particular table is synced and becomes ready, the replication

slot has no use anymore.

Why can't it be used to sync the other tables if any?

It can be used. But I thought it would be better not to, for example in the
following case:
Let's say a sync worker starts with a table in INIT state. The worker
creates a new replication slot to sync that table.
When sync of the table is completed, it will move to the next one. This
time the new table may be in FINISHEDCOPY state, so the worker may need to
use the new table's existing replication slot.
Before the worker will move to the next table again, there will be two
replication slots used by the worker. We might want to keep one and drop
the other.
At this point, I thought it would be better to keep the replication slot
created by this worker in the first place. I think it's easier to track
slots this way since we know how to generate the rep slots name.
Otherwise we would need to store the replication slot name somewhere too.

This sounds reasonable. Let's do this unless we get some better idea.

I updated the patch to use an unique id for replication slot names and
store the last used id in the catalog.
Can you look into it again please?

There is no such restriction that origins should belong to only one

table. What makes you think like that?

I did not reuse origins since I didn't think it would significantly improve
the performance as reusing replication slots does.
So I just kept the origins as they were, even if it was possible to reuse
them. Does that make sense?

Best,
Melih

Attachments:

v3-0001-Reuse-Logical-Replication-Background-worker.patchapplication/octet-stream; name=v3-0001-Reuse-Logical-Replication-Background-worker.patchDownload
From e3d9ab01146c78f8c8e2954c11c0f0d72bf204f4 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH] Reuse Logical Replication Background worker

This commit allows tablesync workers to move to another table that needs synchronization,
when they're done with the curren table in tablesync phase of Logical Replication.

It reduces the overhead of launching/killing a new background worker for each table.
By reusing tablesync workers, replication slots created for tablesync can be reused as well.
Removing the burden of creating/dropping replication slot improves tablesync speed.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/catalogs.sgml                  |  20 ++
 src/backend/catalog/pg_subscription.c       | 231 ++++++++++++-
 src/backend/commands/subscriptioncmds.c     | 169 ++++++----
 src/backend/replication/logical/launcher.c  |   9 +-
 src/backend/replication/logical/tablesync.c | 217 +++++++++---
 src/backend/replication/logical/worker.c    | 345 ++++++++++++--------
 src/include/catalog/pg_subscription.h       |   5 +
 src/include/catalog/pg_subscription_rel.h   |  10 +-
 src/include/replication/slot.h              |   3 +-
 src/include/replication/worker_internal.h   |  28 +-
 src/test/regress/expected/misc_sanity.out   |   3 +-
 11 files changed, 794 insertions(+), 246 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index cd2cc37aeb..8048710244 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -7957,6 +7957,17 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        origin.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>sublastusedid</structfield> <type>int8</type>
+      </para>
+      <para>
+       The last used ID for tablesync workers. This ID is used to 
+       create replication slots. The last used ID needs to be stored 
+       to make logical replication can safely proceed after an interruption.  
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
@@ -8041,6 +8052,15 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        otherwise null
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srrelslotname</structfield> <type>text</type>
+      </para>
+      <para>
+       Replication slot name that is used for synchronization of relation 
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index a506fc3ec8..fc7973859d 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -71,7 +71,6 @@ GetSubscription(Oid subid, bool missing_ok)
 	sub->stream = subform->substream;
 	sub->twophasestate = subform->subtwophasestate;
 	sub->disableonerr = subform->subdisableonerr;
-
 	/* Get conninfo */
 	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
 							tup,
@@ -114,6 +113,14 @@ GetSubscription(Oid subid, bool missing_ok)
 	Assert(!isnull);
 	sub->origin = TextDatumGetCString(datum);
 
+	/* Get last used id */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
+							tup,
+							Anum_pg_subscription_sublastusedid,
+							&isnull);
+	Assert(!isnull);
+	sub->lastusedid = DatumGetInt64(datum);
+
 	ReleaseSysCache(tup);
 
 	return sub;
@@ -205,6 +212,44 @@ DisableSubscription(Oid subid)
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update the last used replication slot ID for the given subscription.
+ */
+void
+UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid)
+{
+	Relation	rel;
+	bool		nulls[Natts_pg_subscription];
+	bool		replaces[Natts_pg_subscription];
+	Datum		values[Natts_pg_subscription];
+	HeapTuple	tup;
+
+	/* Look up the subscription in the catalog */
+	rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+	tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	/* Form a new tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	replaces[Anum_pg_subscription_sublastusedid - 1] = true;
+	values[Anum_pg_subscription_sublastusedid- 1] = Int64GetDatum(lastusedid);
+
+	/* Update the catalog */
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+	heap_freetuple(tup);
+
+	table_close(rel, NoLock);
+}
+
 /*
  * Convert text array to list of strings.
  *
@@ -234,7 +279,7 @@ textarray_to_stringlist(ArrayType *textarray)
  */
 void
 AddSubscriptionRelState(Oid subid, Oid relid, char state,
-						XLogRecPtr sublsn)
+						XLogRecPtr sublsn, char *relslotname)
 {
 	Relation	rel;
 	HeapTuple	tup;
@@ -263,9 +308,17 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
 	else
 		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (relslotname)
+	{
+		values[Anum_pg_subscription_rel_srrelslotname - 1] = CStringGetTextDatum(relslotname);
+	}
+	else
+	{
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	}
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
-
+ 
 	/* Insert tuple into catalog. */
 	CatalogTupleInsert(rel, tup);
 
@@ -275,6 +328,42 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Internal function to modify columns for relation state update
+ */
+static void
+UpdateSubscriptionRelState_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char state,
+						   			XLogRecPtr sublsn)
+{
+	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
+	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+
+	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (sublsn != InvalidXLogRecPtr)
+		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
+	else
+		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication slot update
+ */
+static void
+UpdateSubscriptionRelReplicationSlot_internal(Datum *values,
+											bool *nulls,
+											bool *replaces,
+											char *relslotname)
+{
+	replaces[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] = CStringGetTextDatum(relslotname);
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+}
+
 /*
  * Update the state of a subscription table.
  */
@@ -305,14 +394,94 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	memset(nulls, false, sizeof(nulls));
 	memset(replaces, false, sizeof(replaces));
 
-	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
-	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
 
-	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
-	if (sublsn != InvalidXLogRecPtr)
-		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
-	else
-		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update the replication slot name of a subscription table.
+ */
+void
+UpdateSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *relslotname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
+
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update replication slot name and state of a subscription table in one transaction.
+ */
+void
+UpdateSubscriptionRelStateAndSlot(Oid subid, 
+								Oid relid, 
+								char state,
+						   		XLogRecPtr sublsn,
+								char *relslotname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
 
 	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
 							replaces);
@@ -324,6 +493,46 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Get replication slot name of subscription table.
+ *
+ * Returns null if the subscription table does not have a replication slot.
+ */
+void
+GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*relrepslot;
+	bool		isnull;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srrelslotname, &isnull);
+	if (!isnull)
+	{
+		relrepslot = TextDatumGetCString(d);
+		memcpy(slotname, relrepslot, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
 /*
  * Get state of subscription table.
  *
@@ -542,4 +751,4 @@ GetSubscriptionRelations(Oid subid, bool not_ready)
 	table_close(rel, AccessShareLock);
 
 	return res;
-}
+}
\ No newline at end of file
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index f73dfb6067..b893700a84 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -646,6 +646,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 		publicationListToArray(publications);
 	values[Anum_pg_subscription_suborigin - 1] =
 		CStringGetTextDatum(opts.origin);
+	values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(1);
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -704,7 +705,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 										 rv->schemaname, rv->relname);
 
 				AddSubscriptionRelState(subid, relid, table_state,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL);
 			}
 
 			/*
@@ -794,6 +795,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	} SubRemoveRels;
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -866,7 +869,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				AddSubscriptionRelState(sub->oid, relid,
 										copy_data ? SUBREL_STATE_INIT : SUBREL_STATE_READY,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL);
 				ereport(DEBUG1,
 						(errmsg_internal("table \"%s.%s\" added to subscription \"%s\"",
 										 rv->schemaname, rv->relname, sub->name)));
@@ -916,7 +919,18 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/* 
+				 * Find the logical replication sync worker if exists 
+				 * Store the slot number for dropping associated replication slot later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot_name);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
@@ -950,31 +964,23 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
+		 * Drop the replication slots associated with tablesync workers for removed tables.
+		 * This has to be at the end because otherwise if there is an error while doing
 		 * the database operations we won't be able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
+			memcpy(syncslotname, lfirst(lc), sizeof(NAMEDATALEN));
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			/*
+			 * There is no certainty, maybe the slot
+			 * does not exist yet. Also, if we fail after removing some of
+			 * the slots, next time, it will again try to drop already
+			 * dropped slots and fail. For these reasons, we allow
+			 * missing_ok = true for the drop.
+			 */
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1567,39 +1573,16 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
-
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
-
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		List *slots = NULL;
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+		
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char *syncslotname = (char *) lfirst(lc);
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1622,6 +1605,69 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * WRITE COMMENT HERE
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok){
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+						" FROM pg_replication_slots"
+						" WHERE slot_name LIKE 'pg_%i_sync_%%';",
+						 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+					 errmsg("not tuple returned."));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();\
+	return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -1863,6 +1909,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -1873,18 +1920,20 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+		
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
-			char		syncslotname[NAMEDATALEN] = {0};
-
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
-				 syncslotname);
+				 worker->slot_name);
 		}
 	}
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 3bbd522724..96cc38eeb5 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -263,7 +263,7 @@ logicalrep_workers_find(Oid subid, bool only_running)
  */
 void
 logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
-						 Oid relid)
+						 Oid relid, int64 slotid)
 {
 	BackgroundWorker bgw;
 	BackgroundWorkerHandle *bgw_handle;
@@ -370,7 +370,11 @@ retry:
 	/* Prepare the worker slot. */
 	worker->launch_time = now;
 	worker->in_use = true;
+	worker->is_first_run = true;
 	worker->generation++;
+	worker->created_slot = false;
+	worker->rep_slot_id = slotid;
+	worker->slot_name = (char *) palloc(NAMEDATALEN);
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -378,6 +382,7 @@ retry:
 	worker->relid = relid;
 	worker->relstate = SUBREL_STATE_UNKNOWN;
 	worker->relstate_lsn = InvalidXLogRecPtr;
+	worker->move_to_next_rel = false;
 	worker->stream_fileset = NULL;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
@@ -868,7 +873,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, 0);
 				}
 			}
 
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 6a01ffd273..a561e6f29d 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -126,12 +126,8 @@ static bool FetchTableStates(bool *started_tx);
 
 static StringInfo copybuf = NULL;
 
-/*
- * Exit routine for synchronization worker.
- */
 static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+clean_sync_worker(void)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -143,18 +139,27 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
-	/* And flush all writes. */
-	XLogFlush(GetXLogWriteRecPtr());
-
-	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
-	CommitTransactionCommand();
+	/* Disconnect from publisher.
+	 * Otherwise reused sync workers causes exceeding max_wal_senders 
+	 */
+	walrcv_disconnect(LogRepWorkerWalRcvConn);
+	LogRepWorkerWalRcvConn = NULL;
 
 	/* Find the main apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
+}
+
+/*
+ * Exit routine for synchronization worker.
+ */
+static void
+pg_attribute_noreturn()
+finish_sync_worker(void)
+{
+	clean_sync_worker();
+	
+	/* And flush all writes. */
+	XLogFlush(GetXLogWriteRecPtr());
 
 	/* Stop gracefully */
 	proc_exit(0);
@@ -180,7 +185,7 @@ wait_for_relation_state_change(Oid relid, char expected_state)
 		LogicalRepWorker *worker;
 		XLogRecPtr	statelsn;
 
-		CHECK_FOR_INTERRUPTS();
+		CHECK_FOR_INTERRUPTS();		
 
 		InvalidateCatalogSnapshot();
 		state = GetSubscriptionRelState(MyLogicalRepWorker->subid,
@@ -284,6 +289,10 @@ invalidate_syncing_table_states(Datum arg, int cacheid, uint32 hashvalue)
 static void
 process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 {
+	List	   *rstates;
+	SubscriptionRelState *rstate;
+	ListCell   *lc;
+
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_CATCHUP &&
@@ -299,7 +308,6 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 
 		/*
 		 * UpdateSubscriptionRelState must be called within a transaction.
-		 * That transaction will be ended within the finish_sync_worker().
 		 */
 		if (!IsTransactionState())
 			StartTransactionCommand();
@@ -308,6 +316,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 								   MyLogicalRepWorker->relid,
 								   MyLogicalRepWorker->relstate,
 								   MyLogicalRepWorker->relstate_lsn);
+		CommitTransactionCommand();
 
 		/*
 		 * End streaming so that LogRepWorkerWalRcvConn can be used to drop
@@ -317,24 +326,88 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 
 		/*
 		 * Cleanup the tablesync slot.
+		 * If the slot name used by this worker is different from the default 
+		 * slot name for the worker, this means the current table had started to being 
+		 * synchronized by another worker and replication slot. And this worker
+		 * is reusing a replication slot from a previous attempt.
+		 * We do not need that replication slot anymore.
 		 *
 		 * This has to be done after updating the state because otherwise if
 		 * there is an error while doing the database operations we won't be
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
+										MyLogicalRepWorker->rep_slot_id,
 										syncslotname,
 										sizeof(syncslotname));
 
+
+		/* This transaction will be ended within the clean_sync_worker(). */
+		StartTransactionCommand();
+		if (MyLogicalRepWorker->slot_name && strcmp(syncslotname, MyLogicalRepWorker->slot_name) != 0)
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
+			UpdateSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+												 MyLogicalRepWorker->relid,
+												 NULL);
+		}
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid))));
+
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
+		 * Check if any table whose relation state is still INIT. 
+		 * If a table in INIT state is found, the worker will not be finished,
+		 * it will be reused instead.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
+		rstates = GetSubscriptionRelations(MySubscription->oid, true);
+		
+		foreach (lc, rstates)
+		{
+			rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
+			memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
+
+			/* 
+			 * Pick the table for the next run
+			 * if there is not another worker already picked that table.
+			 */
+			LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+			if (rstate->state != SUBREL_STATE_SYNCDONE && 
+				!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+			{
+				/* Update worker state for the next table */
+				MyLogicalRepWorker->is_first_run = false;
+				MyLogicalRepWorker->relid = rstate->relid;
+				MyLogicalRepWorker->relstate = rstate->state;
+				MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+				MyLogicalRepWorker->move_to_next_rel = true;
+				LWLockRelease(LogicalRepWorkerLock);
+				break;
+			}
+			LWLockRelease(LogicalRepWorkerLock);
+		}
+
+		/* Cleanup before next run or ending the worker. */
+		if(!MyLogicalRepWorker->move_to_next_rel)
+		{
+		   /*
+			* It is important to give an error if we are unable to drop the slot,
+			* otherwise, it won't be dropped till the corresponding subscription
+			* is dropped. So passing missing_ok = false.
+			*/
+			if (MyLogicalRepWorker->created_slot)
+			{
+				ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
+			}
 
-		finish_sync_worker();
+			finish_sync_worker();
+		}
+		else
+		{
+			clean_sync_worker();
+		}
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -473,9 +546,9 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				/*
 				 * Update the state to READY only after the origin cleanup.
 				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
+				UpdateSubscriptionRelStateAndSlot(MyLogicalRepWorker->subid,
 										   rstate->relid, rstate->state,
-										   rstate->lsn);
+										   rstate->lsn, NULL);
 			}
 		}
 		else
@@ -564,11 +637,21 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						TimestampDifferenceExceeds(hentry->last_start_time, now,
 												   wal_retrieve_retry_interval))
 					{
+						if (IsTransactionState())
+							CommitTransactionCommand();
+						StartTransactionCommand();
+						started_tx = true;
+
+						MySubscription->lastusedid++;
+						UpdateSubscriptionLastSlotId(MyLogicalRepWorker->subid,
+													MySubscription->lastusedid);
+
 						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
 												 MySubscription->oid,
 												 MySubscription->name,
 												 MyLogicalRepWorker->userid,
-												 rstate->relid);
+												 rstate->relid,
+												 MySubscription->lastusedid);
 						hentry->last_start_time = now;
 					}
 				}
@@ -1140,8 +1223,8 @@ copy_table(Relation rel)
  * The name must not exceed NAMEDATALEN - 1 because of remote node constraints
  * on slot name length. We append system_identifier to avoid slot_name
  * collision with subscriptions in other clusters. With the current scheme
- * pg_%u_sync_%u_UINT64_FORMAT (3 + 10 + 6 + 10 + 20 + '\0'), the maximum
- * length of slot_name will be 50.
+ * pg_%u_sync_%lu_UINT64_FORMAT (3 + 10 + 6 + 20 + 20 + '\0'), the maximum
+ * length of slot_name will be 45.
  *
  * The returned slot name is stored in the supplied buffer (syncslotname) with
  * the given size.
@@ -1152,11 +1235,10 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
-								char *syncslotname, int szslot)
+ReplicationSlotNameForTablesync(Oid suboid, int64 slotid, char *syncslotname, int szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%lu_" UINT64_FORMAT, suboid,
+			 slotid, GetSystemIdentifier());
 }
 
 /*
@@ -1191,6 +1273,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	WalRcvExecResult *res;
 	char		originname[NAMEDATALEN];
 	RepOriginId originid;
+	char  		*prev_slotname;
 
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
@@ -1219,7 +1302,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
+									MyLogicalRepWorker->rep_slot_id,
 									slotname,
 									NAMEDATALEN);
 
@@ -1245,6 +1328,20 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 									  originname,
 									  sizeof(originname));
 
+
+	/*
+	 * See if tablesync of the current relation has been 
+	 * started with another replication slot. 
+	 * 
+	 * Read previous slot name from the catalog, if exists.
+	 */
+	prev_slotname = (char *) palloc0(NAMEDATALEN);
+	StartTransactionCommand();
+	GetSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+									  MyLogicalRepWorker->relid,
+									  prev_slotname);
+	CommitTransactionCommand();
+
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC)
 	{
 		/*
@@ -1258,10 +1355,34 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		 * breakdown then it wouldn't have succeeded so trying it next time
 		 * seems like a better bet.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, slotname, true);
+		if (strcmp(prev_slotname, ""))
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);
+		
+			StartTransactionCommand();
+			UpdateSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+												  MyLogicalRepWorker->relid,
+												  NULL);
+			CommitTransactionCommand();
+		}
 	}
 	else if (MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY)
 	{
+		/*
+		 * At this point, the table that is currently being synchronized should have
+		 * its replication slot name filled in the catalog. The tablesync process was started 
+		 * with another sync worker and replication slot.
+		 * We need to continue using the same replication slot in this worker too.
+		 */
+		if (!strcmp(prev_slotname, ""))
+		{
+			elog(ERROR, "Replication slot could not be found for relation %u",
+				 MyLogicalRepWorker->relid);
+		}
+
+		/* Proceed with the correct replication slot. */
+		slotname = prev_slotname;
+
 		/*
 		 * The COPY phase was previously done, but tablesync then crashed
 		 * before it was able to finish normally.
@@ -1289,10 +1410,11 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
+	UpdateSubscriptionRelStateAndSlot(MyLogicalRepWorker->subid,
 							   MyLogicalRepWorker->relid,
 							   MyLogicalRepWorker->relstate,
-							   MyLogicalRepWorker->relstate_lsn);
+							   MyLogicalRepWorker->relstate_lsn,
+							   slotname);
 	CommitTransactionCommand();
 	pgstat_report_stat(true);
 
@@ -1351,16 +1473,31 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	 * for the catchup phase after COPY is done, so tell it to use the
 	 * snapshot to make the final data consistent.
 	 *
+	 * Replication slot will only be created if either this is the first run 
+	 * of the worker or we're not using a previous replication slot. 
+	 * 
 	 * Prevent cancel/die interrupts while creating slot here because it is
 	 * possible that before the server finishes this command, a concurrent
 	 * drop subscription happens which would complete without removing this
 	 * slot leading to a dangling slot on the server.
 	 */
-	HOLD_INTERRUPTS();
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-	RESUME_INTERRUPTS();
+	if (MyLogicalRepWorker->is_first_run ||
+		(!strcmp(prev_slotname, "") && !MyLogicalRepWorker->created_slot))
+	{
+		HOLD_INTERRUPTS();
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						slotname, false /* permanent */ , false /* two_phase */ ,
+						CRS_USE_SNAPSHOT, origin_startpos);
+		RESUME_INTERRUPTS();
+
+		/*
+		 * Remember that we created the slot so that
+		 * we will not try to create it again.
+		 */
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->created_slot = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
+	}
 
 	/*
 	 * Setup replication origin tracking. The purpose of doing this before the
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 5f8c541763..163c0ea921 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -315,6 +315,7 @@ static void stream_cleanup_files(Oid subid, TransactionId xid);
 static void stream_open_file(Oid subid, TransactionId xid, bool first);
 static void stream_write_change(char action, StringInfo s);
 static void stream_close_file(void);
+static void stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
@@ -2814,6 +2815,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+			if (MyLogicalRepWorker->move_to_next_rel)
+			{
+				endofstream = true;
+			}
 		}
 
 		/* Cleanup the memory. */
@@ -2915,8 +2920,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	/* Pop the error context stack */
 	error_context_stack = errcallback.previous;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/* 
+	 * If it's moving to next relation, this is a sync worker.
+	 * Sync workers end the streaming during process_syncing_tables_for_sync.
+	 * Calling endstreaming twice causes "no COPY in progress" errors.
+	 */
+	if (!MyLogicalRepWorker->move_to_next_rel)
+	{
+		/* All done */
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	}
 }
 
 /*
@@ -3458,6 +3471,35 @@ stream_write_change(char action, StringInfo s)
 	BufFileWrite(stream_fd, &s->data[s->cursor], len);
 }
 
+/*
+ * stream_build_options_replication
+ * 		Build logical replication streaming options.
+ *
+ * This function sets streaming options including replication slot name
+ * and origin start position. Workers need these options for logical replication.
+ */
+static void
+stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos)
+{
+	int server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+	options->proto.logical.streaming = MySubscription->stream;
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -3532,6 +3574,9 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 
 	/* allocate slot name in long-lived context */
 	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+
+	/* Keep the replication slot name used for this sync. */
+	MyLogicalRepWorker->slot_name = *myslotname;
 	pfree(syncslotname);
 }
 
@@ -3569,6 +3614,136 @@ start_apply(XLogRecPtr origin_startpos)
 	PG_END_TRY();
 }
 
+/*
+ * Runs the tablesync worker.
+ * It starts table sync. After successful sync, 
+ * builds streaming options and starts streaming. 
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options, 
+					 char *slotname,
+					 char *originname,
+					 int originame_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Set this to false for safety, in case we're already reusing the worker */
+    MyLogicalRepWorker->move_to_next_rel = false;
+
+    start_table_sync(origin_startpos, &slotname);
+
+    /*
+        * Allocate the origin name in long-lived context for error context
+        * message.
+        */
+    ReplicationOriginNameForTablesync(MySubscription->oid,
+                                        MyLogicalRepWorker->relid,
+                                        originname,
+                                        originame_size);
+    apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
+                                                                originname);
+    
+    stream_build_options(options, slotname, origin_startpos);
+
+    /* Start normal logical streaming replication. */
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+}
+
+/*
+ * Runs the apply worker.
+ * It sets up replication origin, the streaming options 
+ * and then starts streaming. 
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
+{
+    RepOriginId originid;
+    TimeLineID	startpointTLI;
+    char	   *err;
+
+    slotname = MySubscription->slotname;
+
+    /*
+	 * This shouldn't happen if the subscription is enabled, but guard
+	 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+	 * crash if slot is NULL.)
+	 */
+    if (!slotname)
+        ereport(ERROR,
+                (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+                    errmsg("subscription has no replication slot set")));
+
+    /* Setup replication origin tracking. */
+    StartTransactionCommand();
+    snprintf(originname, originname_size, "pg_%u", MySubscription->oid);
+    originid = replorigin_by_name(originname, true);
+    if (!OidIsValid(originid))
+        originid = replorigin_create(originname);
+    replorigin_session_setup(originid);
+    replorigin_session_origin = originid;
+    *origin_startpos = replorigin_session_get_progress(false);
+    CommitTransactionCommand();
+
+    LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+                                            MySubscription->name, &err);
+    if (LogRepWorkerWalRcvConn == NULL)
+        ereport(ERROR,
+                (errcode(ERRCODE_CONNECTION_FAILURE),
+                    errmsg("could not connect to the publisher: %s", err)));
+
+    /*
+	 * We don't really use the output identify_system for anything but it
+	 * does some initializations on the upstream so let's still call it.
+	 */
+    (void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+    /*
+	 * Allocate the origin name in long-lived context for error context
+	 * message.
+	 */
+    apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
+                                                                originname);
+
+    stream_build_options(options, slotname, origin_startpos);
+
+    /*
+     * Even when the two_phase mode is requested by the user, it remains
+     * as the tri-state PENDING until all tablesyncs have reached READY
+     * state. Only then, can it become ENABLED.
+     *
+     * Note: If the subscription has no tables then leave the state as
+     * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+     * work.
+     */
+    if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+        AllTablesyncsReady())
+    {
+        /* Start streaming with two_phase enabled */
+        options->proto.logical.twophase = true;
+        walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+        StartTransactionCommand();
+        UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+        MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+        CommitTransactionCommand();
+    }
+    else
+    {
+        walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+    }
+
+    ereport(DEBUG1,
+            (errmsg("logical replication apply worker for subscription \"%s\" two_phase is %s",
+                    MySubscription->name,
+                    MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+                    MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+                    MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+                    "?")));
+}
+
 /* Logical Replication Apply worker entry point */
 void
 ApplyWorkerMain(Datum main_arg)
@@ -3579,7 +3754,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	/* Attach to slot */
 	logicalrep_worker_attach(worker_slot);
@@ -3670,142 +3844,55 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		/*
-		 * Allocate the origin name in long-lived context for error context
-		 * message.
-		 */
-		ReplicationOriginNameForTablesync(MySubscription->oid,
-										  MyLogicalRepWorker->relid,
-										  originname,
-										  sizeof(originname));
-		apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
-																   originname);
-	}
-	else
-	{
-		/* This is main apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* 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);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		/*
-		 * Allocate the origin name in long-lived context for error context
-		 * message.
-		 */
-		apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
-																   originname);
-	}
-
 	/*
-	 * Setup callback for syscache so that we know when something changes in
-	 * the subscription relation state.
-	 */
+	* Setup callback for syscache so that we know when something changes in
+	* the subscription relation state.
+	* Do this outside the loop to avoid exceeding MAX_SYSCACHE_CALLBACKS
+	*/
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
-								  invalidate_syncing_table_states,
-								  (Datum) 0);
-
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
-
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
-
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-	options.proto.logical.streaming = MySubscription->stream;
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
+								invalidate_syncing_table_states,
+								(Datum) 0);
 
-	if (!am_tablesync_worker())
+	/*
+	 * The loop where worker does its job.
+	 * It loops until the worker is not reused. 
+	 */
+	while (MyLogicalRepWorker->is_first_run || 
+			MyLogicalRepWorker->move_to_next_rel)
 	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
+		if (am_tablesync_worker())
+			{
+				/* 
+				* This is a tablesync worker. 
+				* Start syncing tables before starting the apply loop.  
+				*/
+				run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
+			}
+			else
+			{
+				/* This is main apply worker */
+				run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
+			}
+		
+		/* Run the main loop. */
+		start_apply(origin_startpos);
+
+		if (MyLogicalRepWorker->move_to_next_rel)
 		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+			/* Reset the currenct replication origin session.
+			* Since we'll use the same process for another relation, it needs to be reset 
+			* and will be created again later while syncing the new relation.
+			*/
+			replorigin_session_origin = InvalidRepOriginId;
+			replorigin_session_reset();
 
 			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+			ereport(LOG,
+					(errmsg("logical replication table synchronization worker for subscription \"%s\" has moved to sync table \"%s\".",
+							MySubscription->name, get_rel_name(MyLogicalRepWorker->relid))));
 			CommitTransactionCommand();
 		}
-		else
-		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-		}
-
-		ereport(DEBUG1,
-				(errmsg("logical replication apply worker for subscription \"%s\" two_phase is %s",
-						MySubscription->name,
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-						"?")));
 	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
-
-	/* Run the main loop. */
-	start_apply(origin_startpos);
-
 	proc_exit(0);
 }
 
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 7b98714f30..2d391615e8 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -102,6 +102,9 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
 
 	/* Only publish data originating from the specified origin */
 	text		suborigin BKI_DEFAULT(LOGICALREP_ORIGIN_ANY);
+
+	/* The last used ID to create a replication slot for tablesync */
+	int64		sublastusedid BKI_DEFAULT(0);	
 #endif
 } FormData_pg_subscription;
 
@@ -135,11 +138,13 @@ typedef struct Subscription
 	List	   *publications;	/* List of publication names to subscribe to */
 	char	   *origin;			/* Only publish data originating from the
 								 * specified origin */
+	int64		lastusedid;		/* Last used unique ID to create replication slots in tablesync */
 } Subscription;
 
 extern Subscription *GetSubscription(Oid subid, bool missing_ok);
 extern void FreeSubscription(Subscription *sub);
 extern void DisableSubscription(Oid subid);
+extern void UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid);
 
 extern int	CountDBSubscriptions(Oid dbid);
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 8e88de7b2b..7373df40ab 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -44,6 +44,8 @@ CATALOG(pg_subscription_rel,6102,SubscriptionRelRelationId)
 											 * used for synchronization
 											 * coordination, or NULL if not
 											 * valid */
+	text		srrelslotname BKI_FORCE_NULL;	/* name of the replication slot for relatio in subscription */
+
 #endif
 } FormData_pg_subscription_rel;
 
@@ -81,12 +83,18 @@ typedef struct SubscriptionRelState
 } SubscriptionRelState;
 
 extern void AddSubscriptionRelState(Oid subid, Oid relid, char state,
-									XLogRecPtr sublsn);
+									XLogRecPtr sublsn, char *relslotname);
 extern void UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 									   XLogRecPtr sublsn);
 extern char GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn);
 extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 
+extern void UpdateSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *relslotname);
+extern void GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname);
+
+extern void UpdateSubscriptionRelStateAndSlot(Oid subid, Oid relid, char state,
+									XLogRecPtr sublsn, char *relslotname);
+
 extern bool HasSubscriptionRelations(Oid subid);
 extern List *GetSubscriptionRelations(Oid subid, bool not_ready);
 
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 8c9f3321d5..4b3e2993e3 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -217,8 +217,9 @@ extern void ReplicationSlotsDropDBSlots(Oid dboid);
 extern bool InvalidateObsoleteReplicationSlots(XLogSegNo oldestSegno);
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, int szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, int64 slotid, char *syncslotname, int szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List* GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 901845abc2..5fabd1a829 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -30,6 +30,27 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/* 
+	 * Indicates if worker is running for the first time
+	 * or in reuse
+	 */
+	bool		is_first_run;
+
+	/* 
+	 * Indicates if the sync worker created a replication slot
+	 * or it reuses an existing one created by another worker.
+	 */
+	bool		created_slot;
+
+	/* 
+	 * Unique identifier for replication slot to be 
+	 * created by tablesnync workers, if needed.
+	 */
+	int64 	 	rep_slot_id;
+
+	/* Replication slot name used by the worker. */
+	char		*slot_name;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
@@ -51,6 +72,11 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/* 
+	 * Used to indicate whether sync worker will be reused for another relation
+	 */
+	bool		move_to_next_rel;
+	
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction, the
@@ -85,7 +111,7 @@ 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);
+									 Oid userid, Oid relid, int64 slotid);
 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/test/regress/expected/misc_sanity.out b/src/test/regress/expected/misc_sanity.out
index a57fd142a9..a2a6f14944 100644
--- a/src/test/regress/expected/misc_sanity.out
+++ b/src/test/regress/expected/misc_sanity.out
@@ -60,7 +60,8 @@ ORDER BY 1, 2;
  pg_index                | indpred       | pg_node_tree
  pg_largeobject          | data          | bytea
  pg_largeobject_metadata | lomacl        | aclitem[]
-(11 rows)
+ pg_subscription_rel     | srrelslotname | text
+(12 rows)
 
 -- system catalogs without primary keys
 --
-- 
2.25.1

#14Amit Kapila
amit.kapila16@gmail.com
In reply to: Melih Mutlu (#13)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Fri, Aug 5, 2022 at 7:25 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Why can't it be used to sync the other tables if any?

It can be used. But I thought it would be better not to, for example in the following case:
Let's say a sync worker starts with a table in INIT state. The worker creates a new replication slot to sync that table.
When sync of the table is completed, it will move to the next one. This time the new table may be in FINISHEDCOPY state, so the worker may need to use the new table's existing replication slot.
Before the worker will move to the next table again, there will be two replication slots used by the worker. We might want to keep one and drop the other.
At this point, I thought it would be better to keep the replication slot created by this worker in the first place. I think it's easier to track slots this way since we know how to generate the rep slots name.
Otherwise we would need to store the replication slot name somewhere too.

I think there is some basic flaw in slot reuse design. Currently, we
copy the table by starting a repeatable read transaction (BEGIN READ
ONLY ISOLATION LEVEL REPEATABLE READ) and create a slot that
establishes a snapshot which is first used for copy and then LSN
returned by it is used in the catchup phase after the copy is done.
The patch won't establish such a snapshot before a table copy as it
won't create a slot each time. If this understanding is correct, I
think we need to use ExportSnapshot/ImportSnapshot functionality to
achieve it or do something else to avoid the problem mentioned.

This sounds reasonable. Let's do this unless we get some better idea.

There is no such restriction that origins should belong to only one
table. What makes you think like that?

I did not reuse origins since I didn't think it would significantly improve the performance as reusing replication slots does.
So I just kept the origins as they were, even if it was possible to reuse them. Does that make sense?

For small tables, it could have a visible performance difference as it
involves database write operations to each time create and drop the
origin. But if we don't want to reuse then also you need to set its
origin_lsn appropriately. Currently (without this patch), after
creating the slot, we directly use the origin_lsn returned by
create_slot API whereas now it won't be the same case as the patch
doesn't create a slot every time.

--
With Regards,
Amit Kapila.

#15Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Amit Kapila (#14)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Amit,

Amit Kapila <amit.kapila16@gmail.com>, 6 Ağu 2022 Cmt, 16:01 tarihinde şunu
yazdı:

I think there is some basic flaw in slot reuse design. Currently, we
copy the table by starting a repeatable read transaction (BEGIN READ
ONLY ISOLATION LEVEL REPEATABLE READ) and create a slot that
establishes a snapshot which is first used for copy and then LSN
returned by it is used in the catchup phase after the copy is done.
The patch won't establish such a snapshot before a table copy as it
won't create a slot each time. If this understanding is correct, I
think we need to use ExportSnapshot/ImportSnapshot functionality to
achieve it or do something else to avoid the problem mentioned.

I did not really think about the snapshot created by replication slot while
making this change. Thanks for pointing it out.
I've been thinking about how to fix this issue. There are some points I'm
still not sure about.
If the worker will not create a new replication slot, which snapshot should
we actually export and then import?
At the line where the worker was supposed to create replication slot but
now will reuse an existing slot instead, calling pg_export_snapshot() can
export the snapshot instead of CREATE_REPLICATION_SLOT.
However, importing that snapshot into the current transaction may not make
any difference since we exported that snapshot from the same transaction. I
think this wouldn't make sense.
How else an export/import snapshot logic can be placed in this change?

LSN also should be set accurately. The current change does not handle LSN
properly.
I see that CREATE_REPLICATION_SLOT returns consistent_point which indicates
the earliest location which streaming can start from. And this
consistent_point is used as origin_startpos.
If that's the case, would it make sense to use "confirmed_flush_lsn" of the
replication slot in case the slot is being reused?
Since confirmed_flush_lsn can be considered as the safest, earliest
location which streaming can start from, I think it would work.

And at this point, with the correct LSN, I'm wondering whether this
export/import logic is really necessary if the worker does not create a
replication slot. What do you think?

For small tables, it could have a visible performance difference as it

involves database write operations to each time create and drop the
origin. But if we don't want to reuse then also you need to set its
origin_lsn appropriately. Currently (without this patch), after
creating the slot, we directly use the origin_lsn returned by
create_slot API whereas now it won't be the same case as the patch
doesn't create a slot every time.

Correct. For this issue, please consider the LSN logic explained above.

Thanks,
Melih

#16Amit Kapila
amit.kapila16@gmail.com
In reply to: Melih Mutlu (#15)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Mon, Aug 15, 2022 at 4:56 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi Amit,

Amit Kapila <amit.kapila16@gmail.com>, 6 Ağu 2022 Cmt, 16:01 tarihinde şunu yazdı:

I think there is some basic flaw in slot reuse design. Currently, we
copy the table by starting a repeatable read transaction (BEGIN READ
ONLY ISOLATION LEVEL REPEATABLE READ) and create a slot that
establishes a snapshot which is first used for copy and then LSN
returned by it is used in the catchup phase after the copy is done.
The patch won't establish such a snapshot before a table copy as it
won't create a slot each time. If this understanding is correct, I
think we need to use ExportSnapshot/ImportSnapshot functionality to
achieve it or do something else to avoid the problem mentioned.

I did not really think about the snapshot created by replication slot while making this change. Thanks for pointing it out.
I've been thinking about how to fix this issue. There are some points I'm still not sure about.
If the worker will not create a new replication slot, which snapshot should we actually export and then import?

Can we (export/import) use the snapshot we used the first time when a
slot is created for future transactions that copy other tables?
Because if we can do that then I think we can use the same LSN as
returned for the slot when it was created for all other table syncs.

--
With Regards,
Amit Kapila.

#17Ian Lawrence Barwick
barwick@gmail.com
In reply to: Melih Mutlu (#13)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

2022年8月5日(金) 22:55 Melih Mutlu <m.melihmutlu@gmail.com>:

Hi Amit,

Why after step 4, do you need to drop the replication slot? Won't just
clearing the required info from the catalog be sufficient?

The replication slots that we read from the catalog will not be used for anything else after we're done with syncing the table which the rep slot belongs to.
It's removed from the catalog when the sync is completed and it basically becomes a slot that is not linked to any table or worker. That's why I think it should be dropped rather than left behind.

Note that if a worker dies and its replication slot continues to exist, that slot will only be used to complete the sync process of the one table that the dead worker was syncing but couldn't finish.
When that particular table is synced and becomes ready, the replication slot has no use anymore.

Why can't it be used to sync the other tables if any?

It can be used. But I thought it would be better not to, for example in the following case:
Let's say a sync worker starts with a table in INIT state. The worker creates a new replication slot to sync that table.
When sync of the table is completed, it will move to the next one. This time the new table may be in FINISHEDCOPY state, so the worker may need to use the new table's existing replication slot.
Before the worker will move to the next table again, there will be two replication slots used by the worker. We might want to keep one and drop the other.
At this point, I thought it would be better to keep the replication slot created by this worker in the first place. I think it's easier to track slots this way since we know how to generate the rep slots name.
Otherwise we would need to store the replication slot name somewhere too.

This sounds reasonable. Let's do this unless we get some better idea.

I updated the patch to use an unique id for replication slot names and store the last used id in the catalog.
Can you look into it again please?

There is no such restriction that origins should belong to only one
table. What makes you think like that?

I did not reuse origins since I didn't think it would significantly improve the performance as reusing replication slots does.
So I just kept the origins as they were, even if it was possible to reuse them. Does that make sense?

Hi

cfbot reports the patch no longer applies [1]http://cfbot.cputube.org/patch_40_3784.log. As CommitFest 2022-11 is
currently underway, this would be an excellent time to update the patch.

[1]: http://cfbot.cputube.org/patch_40_3784.log

Thanks

Ian Barwick

#18Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Amit Kapila (#16)
2 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi hackers,

I've been working on/struggling with this patch for a while. But I haven't
updated this thread regularly.
So sharing what I did with this patch so far.

Amit Kapila <amit.kapila16@gmail.com>, 6 Ağu 2022 Cmt, 16:01 tarihinde
şunu yazdı:

I think there is some basic flaw in slot reuse design. Currently, we
copy the table by starting a repeatable read transaction (BEGIN READ
ONLY ISOLATION LEVEL REPEATABLE READ) and create a slot that
establishes a snapshot which is first used for copy and then LSN
returned by it is used in the catchup phase after the copy is done.
The patch won't establish such a snapshot before a table copy as it
won't create a slot each time. If this understanding is correct, I
think we need to use ExportSnapshot/ImportSnapshot functionality to
achieve it or do something else to avoid the problem mentioned.

This issue that Amit mentioned causes some problems such as duplicated rows
in the subscriber.
Basically, with this patch, tablesync worker creates a replication slot
only in its first run. To ensure table copy and sync are consistent with
each other, the worker needs the correct snapshot and LSN which both are
returned by slot create operation.
Since this patch does not create a rep. slot in each table copy and instead
reuses the one created in the beginning, we do not get a new snapshot and
LSN for each table anymore. Snapshot gets lost right after the transaction
is committed, but the patch continues to use the same LSN for next tables
without the proper snapshot.
In the end, for example, the worker might first copy some rows, then apply
changes from rep. slot and inserts those rows again for the tables in
later iterations.

I discussed some possible ways to resolve this with Amit offline:
1- Copy all tables in one transaction so that we wouldn't need to deal with
snapshots.
Not easy to keep track of the progress. If the transaction fails, we would
need to start all over again.

2- Don't lose the first snapshot (by keeping a transaction open with the
snapshot imported or some other way) and use the same snapshot and LSN for
all tables.
I'm not sure about the side effects of keeping a transaction open that long
or using a snapshot that might be too old after some time.
Still seems like it might work.

3- For each table, get a new snapshot and LSN by using an existing
replication slot.
Even though this approach wouldn't create a new replication slot, preparing
the slot for snapshot and then taking the snapshot may be costly.
Need some numbers here to see how much this approach would improve the
performance.

I decided to go with approach 3 (creating a new snapshot with an existing
replication slot) for now since it would require less change in the
tablesync worker logic than the other options would.
To achieve this, this patch introduces a new command for Streaming
Replication Protocol.
The new REPLICATION_SLOT_SNAPSHOT command basically mimics how
CREATE_REPLICATION_SLOT creates a snapshot, but without actually creating a
new replication slot.
Later the tablesync worker calls this command if it decides not to create a
new rep. slot, uses the snapshot created and LSN returned by the command.

Also;
After the changes discussed here [1]/messages/by-id/20220714115155.GA5439@depesz.com, concurrent replication origin drops
by apply worker and tablesync workers may hold each other on wait due to
locks taken by replorigin_drop_by_name.
I see that this harms the performance of logical replication quite a bit in
terms of speed.
Even though reusing replication origins was discussed in this thread
before, the patch didn't include any change to do so.
The updated version of the patch now also reuses replication origins too.
Seems like even only changes to reuse origins by itself improves the
performance significantly.

Attached two patches:
0001: adds REPLICATION_SLOT_SNAPSHOT command for replication protocol.
0002: Reuses workers/replication slots and origins for tablesync

I would appreciate any feedback/review/thought on the approach and both
patches.
I will also share some numbers to compare performances of the patch and
master branch soon.

[1]: /messages/by-id/20220714115155.GA5439@depesz.com
/messages/by-id/20220714115155.GA5439@depesz.com

Best,
--
Melih Mutlu
Microsoft

Attachments:

0001-Add-replication-protocol-cmd-to-create-a-snapshot.patchapplication/octet-stream; name=0001-Add-replication-protocol-cmd-to-create-a-snapshot.patchDownload
From 4bf4d25c2d34063d0018825349b61de0a8d80bf7 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 13 Oct 2022 17:05:45 +0300
Subject: [PATCH 1/2] Add replication protocol cmd to create a snapshot

Introduced REPLICATION_SLOT_SNAPSHOT to be able to create and use a
snapshot without creating a new replication slot, but by using an
existing slot.

REPLICATION_SLOT_SNAPSHOT command imports the snapshot into the current
transaction and returns consistent_point. The changes earlier than the
consistent_point will be applied by importing the snapshot. All changes
later than the consistent_point will be available to be consumed from
the replication slot.
---
 doc/src/sgml/protocol.sgml                    | 32 ++++++
 .../libpqwalreceiver/libpqwalreceiver.c       | 69 ++++++++++++-
 src/backend/replication/logical/logical.c     | 33 ++++++-
 .../replication/logical/logicalfuncs.c        |  1 +
 src/backend/replication/repl_gram.y           | 18 +++-
 src/backend/replication/repl_scanner.l        |  2 +
 src/backend/replication/slotfuncs.c           |  1 +
 src/backend/replication/walsender.c           | 97 ++++++++++++++++++-
 src/include/nodes/replnodes.h                 | 11 +++
 src/include/replication/logical.h             |  1 +
 src/include/replication/walreceiver.h         | 13 +++
 src/tools/pgindent/typedefs.list              |  1 +
 12 files changed, 275 insertions(+), 4 deletions(-)

diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 5fdd429e05..e148e3baf0 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2612,6 +2612,38 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
      </listitem>
     </varlistentry>
 
+    <varlistentry id="protocol-replication-replication-slot-snapshot">
+     <term><literal>REPLICATION_SLOT_SNAPSHOT</literal> <replaceable class="parameter">slot_name</replaceable> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
+      <indexterm><primary>REPLICATION_SLOT_SNAPSHOT</primary></indexterm>
+     </term>
+     <listitem>
+      <para>
+       Creates a snapshot including all the changes from the replication slot until
+       the point at which the replication slot becomes consistent. Then the snapshot
+       is used in the currenct transaction. This command is currently only supported
+       for logical replication.
+       slots.
+      </para>
+
+      <para>
+       In response to this command, the server will return a one-row result set,
+       containing the following field:
+       <variablelist>
+        <varlistentry>
+         <term><literal>consistent_point</literal> (<type>text</type>)</term>
+         <listitem>
+          <para>
+           The WAL location at which the slot became consistent.  This is the
+           earliest location from which streaming can start on this replication
+           slot.
+          </para>
+         </listitem>
+        </varlistentry>
+       </variablelist>
+      </para>
+     </listitem>
+    </varlistentry>
+
     <varlistentry id="protocol-replication-base-backup" xreflabel="BASE_BACKUP">
      <term><literal>BASE_BACKUP</literal> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
       <indexterm><primary>BASE_BACKUP</primary></indexterm>
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index 7f697b0f29..e367c5f757 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -80,6 +80,8 @@ static WalRcvExecResult *libpqrcv_exec(WalReceiverConn *conn,
 									   const int nRetTypes,
 									   const Oid *retTypes);
 static void libpqrcv_disconnect(WalReceiverConn *conn);
+static void libpqrcv_slot_snapshot(WalReceiverConn *conn, char *slotname,
+								   const WalRcvStreamOptions *options, XLogRecPtr *lsn);
 
 static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_connect = libpqrcv_connect,
@@ -96,7 +98,8 @@ static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_create_slot = libpqrcv_create_slot,
 	.walrcv_get_backend_pid = libpqrcv_get_backend_pid,
 	.walrcv_exec = libpqrcv_exec,
-	.walrcv_disconnect = libpqrcv_disconnect
+	.walrcv_disconnect = libpqrcv_disconnect,
+	.walrcv_slot_snapshot = libpqrcv_slot_snapshot
 };
 
 /* Prototypes for private functions */
@@ -968,6 +971,70 @@ libpqrcv_create_slot(WalReceiverConn *conn, const char *slotname,
 	return snapshot;
 }
 
+/*
+ * TODO
+ */
+static void
+libpqrcv_slot_snapshot(WalReceiverConn *conn,
+					   char *slotname,
+					   const WalRcvStreamOptions *options,
+					   XLogRecPtr *lsn)
+{
+	StringInfoData cmd;
+	PGresult   *res;
+	char	   *pubnames_str;
+	List	   *pubnames;
+	char	   *pubnames_literal;
+
+	initStringInfo(&cmd);
+
+	/* Build the command. */
+	appendStringInfo(&cmd, "REPLICATION_SLOT_SNAPSHOT \"%s\"", slotname);
+	appendStringInfoString(&cmd, " (");
+	appendStringInfo(&cmd, " proto_version '%u'",
+					 options->proto.logical.proto_version);
+
+	/* Add publication names. */
+	pubnames = options->proto.logical.publication_names;
+	pubnames_str = stringlist_to_identifierstr(conn->streamConn, pubnames);
+	if (!pubnames_str)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	pubnames_literal = PQescapeLiteral(conn->streamConn, pubnames_str,
+									   strlen(pubnames_str));
+	if (!pubnames_literal)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	appendStringInfo(&cmd, ", publication_names %s", pubnames_literal);
+	PQfreemem(pubnames_literal);
+	pfree(pubnames_str);
+
+	appendStringInfoString(&cmd, " )");
+
+	/* Execute the command. */
+	res = libpqrcv_PQexec(conn->streamConn, cmd.data);
+	pfree(cmd.data);
+
+	if (PQresultStatus(res) != PGRES_TUPLES_OK)
+	{
+		PQclear(res);
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg("Could not create a snapshot by replication slot \"%s\": %s",
+						slotname, pchomp(PQerrorMessage(conn->streamConn)))));
+	}
+
+	if (lsn)
+		*lsn = DatumGetLSN(DirectFunctionCall1Coll(pg_lsn_in, InvalidOid,
+												   CStringGetDatum(PQgetvalue(res, 0, 0))));
+
+	PQclear(res);
+}
+
 /*
  * Return PID of remote backend process.
  */
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 625a7f4273..56d2256c26 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -461,6 +461,10 @@ CreateInitDecodingContext(const char *plugin,
  * fast_forward
  *		bypass the generation of logical changes.
  *
+ * need_full_snapshot
+ * 		if true, create a snapshot able to read all tables,
+ * 		otherwise do not create any snapshot.
+ *
  * xl_routine
  *		XLogReaderRoutine used by underlying xlogreader
  *
@@ -479,6 +483,7 @@ LogicalDecodingContext *
 CreateDecodingContext(XLogRecPtr start_lsn,
 					  List *output_plugin_options,
 					  bool fast_forward,
+					  bool need_full_snapshot,
 					  XLogReaderRoutine *xl_routine,
 					  LogicalOutputPluginWriterPrepareWrite prepare_write,
 					  LogicalOutputPluginWriterWrite do_write,
@@ -487,6 +492,7 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 	LogicalDecodingContext *ctx;
 	ReplicationSlot *slot;
 	MemoryContext old_context;
+	TransactionId xmin_horizon = InvalidTransactionId;
 
 	/* shorter lines... */
 	slot = MyReplicationSlot;
@@ -533,8 +539,33 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 		start_lsn = slot->data.confirmed_flush;
 	}
 
+
+	/*
+	 * We need to determine a safe xmin horizon to start decoding from if we
+	 * want to create a snapshot too. Otherwise we would end up with a
+	 * snapshot that cannot be imported since xmin value from the snapshot may
+	 * be less than the oldest safe xmin. To avoid this call
+	 * GetOldestSafeDecodingTransactionId() to return a safe xmin value, which
+	 * can be used while exporting/importing the snapshot.
+	 *
+	 * So we have to acquire the ProcArrayLock to prevent computation of new
+	 * xmin horizons by other backends, get the safe decoding xid, and inform
+	 * the slot machinery about the new limit. Once that's done the
+	 * ProcArrayLock can be released as the slot machinery now is protecting
+	 * against vacuum.
+	 */
+	if (need_full_snapshot)
+	{
+		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+		xmin_horizon = GetOldestSafeDecodingTransactionId(!need_full_snapshot);
+		ReplicationSlotsComputeRequiredXmin(true);
+
+		LWLockRelease(ProcArrayLock);
+	}
+
 	ctx = StartupDecodingContext(output_plugin_options,
-								 start_lsn, InvalidTransactionId, false,
+								 start_lsn, xmin_horizon, need_full_snapshot,
 								 fast_forward, xl_routine, prepare_write,
 								 do_write, update_progress);
 
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index 5c23178570..e4b80f2764 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -208,6 +208,7 @@ pg_logical_slot_get_changes_guts(FunctionCallInfo fcinfo, bool confirm, bool bin
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									options,
 									false,
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/repl_gram.y b/src/backend/replication/repl_gram.y
index 7ba33d6672..f70b2e087a 100644
--- a/src/backend/replication/repl_gram.y
+++ b/src/backend/replication/repl_gram.y
@@ -65,6 +65,7 @@ Node *replication_parse_result;
 %token K_CREATE_REPLICATION_SLOT
 %token K_DROP_REPLICATION_SLOT
 %token K_TIMELINE_HISTORY
+%token K_REPLICATION_SLOT_SNAPSHOT
 %token K_WAIT
 %token K_TIMELINE
 %token K_PHYSICAL
@@ -80,7 +81,7 @@ Node *replication_parse_result;
 %type <node>	command
 %type <node>	base_backup start_replication start_logical_replication
 				create_replication_slot drop_replication_slot identify_system
-				read_replication_slot timeline_history show
+				read_replication_slot timeline_history show replication_slot_snapshot
 %type <list>	generic_option_list
 %type <defelt>	generic_option
 %type <uintval>	opt_timeline
@@ -114,6 +115,7 @@ command:
 			| read_replication_slot
 			| timeline_history
 			| show
+			| replication_slot_snapshot
 			;
 
 /*
@@ -307,6 +309,19 @@ timeline_history:
 				}
 			;
 
+/*
+ * REPLICATION_SLOT_SNAPSHOT %s options
+ */
+replication_slot_snapshot:
+			K_REPLICATION_SLOT_SNAPSHOT var_name plugin_options
+				{
+					ReplicationSlotSnapshotCmd *n = makeNode(ReplicationSlotSnapshotCmd);
+					n->slotname = $2;
+					n->options = $3;
+					$$ = (Node *) n;
+				}
+			;
+
 opt_physical:
 			K_PHYSICAL
 			| /* EMPTY */
@@ -400,6 +415,7 @@ ident_or_keyword:
 			| K_CREATE_REPLICATION_SLOT	{ $$ = "create_replication_slot"; }
 			| K_DROP_REPLICATION_SLOT		{ $$ = "drop_replication_slot"; }
 			| K_TIMELINE_HISTORY			{ $$ = "timeline_history"; }
+			| K_REPLICATION_SLOT_SNAPSHOT	{ $$ = "replication_slot_snapshot"; }
 			| K_WAIT						{ $$ = "wait"; }
 			| K_TIMELINE					{ $$ = "timeline"; }
 			| K_PHYSICAL					{ $$ = "physical"; }
diff --git a/src/backend/replication/repl_scanner.l b/src/backend/replication/repl_scanner.l
index 72ef12225e..e33030da61 100644
--- a/src/backend/replication/repl_scanner.l
+++ b/src/backend/replication/repl_scanner.l
@@ -126,6 +126,7 @@ START_REPLICATION	{ return K_START_REPLICATION; }
 CREATE_REPLICATION_SLOT		{ return K_CREATE_REPLICATION_SLOT; }
 DROP_REPLICATION_SLOT		{ return K_DROP_REPLICATION_SLOT; }
 TIMELINE_HISTORY	{ return K_TIMELINE_HISTORY; }
+REPLICATION_SLOT_SNAPSHOT	{ return K_REPLICATION_SLOT_SNAPSHOT; }
 PHYSICAL			{ return K_PHYSICAL; }
 RESERVE_WAL			{ return K_RESERVE_WAL; }
 LOGICAL				{ return K_LOGICAL; }
@@ -303,6 +304,7 @@ replication_scanner_is_replication_command(void)
 		case K_DROP_REPLICATION_SLOT:
 		case K_READ_REPLICATION_SLOT:
 		case K_TIMELINE_HISTORY:
+		case K_REPLICATION_SLOT_SNAPSHOT:
 		case K_SHOW:
 			/* Yes; push back the first token so we can parse later. */
 			repl_pushed_back_token = first_token;
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 79d7f1372d..13d099fe06 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -478,6 +478,7 @@ pg_logical_replication_slot_advance(XLogRecPtr moveto)
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									NIL,
 									true,	/* fast_forward */
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index c11bb3716f..f9a22c8b16 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -238,6 +238,7 @@ static void CreateReplicationSlot(CreateReplicationSlotCmd *cmd);
 static void DropReplicationSlot(DropReplicationSlotCmd *cmd);
 static void StartReplication(StartReplicationCmd *cmd);
 static void StartLogicalReplication(StartReplicationCmd *cmd);
+static void ReplicationSlotSnapshot(ReplicationSlotSnapshotCmd *cmd);
 static void ProcessStandbyMessage(void);
 static void ProcessStandbyReplyMessage(void);
 static void ProcessStandbyHSFeedbackMessage(void);
@@ -1280,7 +1281,7 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	 * are reported early.
 	 */
 	logical_decoding_ctx =
-		CreateDecodingContext(cmd->startpoint, cmd->options, false,
+		CreateDecodingContext(cmd->startpoint, cmd->options, false, false,
 							  XL_ROUTINE(.page_read = logical_read_xlog_page,
 										 .segment_open = WalSndSegmentOpen,
 										 .segment_close = wal_segment_close),
@@ -1332,6 +1333,91 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	EndCommand(&qc, DestRemote, false);
 }
 
+/*
+ * Create a snapshot from an existing replication slot.
+ */
+static void
+ReplicationSlotSnapshot(ReplicationSlotSnapshotCmd *cmd)
+{
+	Snapshot	snap;
+	LogicalDecodingContext *ctx;
+	char		xloc[MAXFNAMELEN];
+	DestReceiver *dest;
+	TupOutputState *tstate;
+	TupleDesc	tupdesc;
+	Datum		values[1];
+	bool		nulls[1] = {0};
+
+	Assert(!MyReplicationSlot);
+
+	if (!IsTransactionBlock())
+		ereport(ERROR,
+				(errmsg("%s must be called inside a transaction",
+						"REPLICATION_SLOT_SNAPSHOT ...")));
+
+	if (XactIsoLevel != XACT_REPEATABLE_READ)
+		ereport(ERROR,
+				(errmsg("%s must be called in REPEATABLE READ isolation mode transaction",
+						"REPLICATION_SLOT_SNAPSHOT ...")));
+
+	if (FirstSnapshotSet)
+		ereport(ERROR,
+				(errmsg("%s must be called before any query",
+						"REPLICATION_SLOT_SNAPSHOT ...")));
+
+	if (IsSubTransaction())
+		ereport(ERROR,
+				(errmsg("%s must not be called in a subtransaction",
+						"REPLICATION_SLOT_SNAPSHOT ...")));
+
+	ReplicationSlotAcquire(cmd->slotname, false);
+
+	ctx = CreateDecodingContext(MyReplicationSlot->data.restart_lsn,
+								cmd->options,
+								false,
+								true,
+								XL_ROUTINE(.page_read = logical_read_xlog_page,
+										   .segment_open = WalSndSegmentOpen,
+										   .segment_close = wal_segment_close),
+								WalSndPrepareWrite, WalSndWriteData,
+								WalSndUpdateProgress);
+
+	/*
+	 * Signal that we don't need the timeout mechanism. We're just creating
+	 * the replication slot and don't yet accept feedback messages or send
+	 * keepalives. As we possibly need to wait for further WAL the walsender
+	 * would otherwise possibly be killed too soon.
+	 */
+	last_reply_timestamp = 0;
+
+	/* build initial snapshot, might take a while */
+	DecodingContextFindStartpoint(ctx);
+
+	snap = SnapBuildInitialSnapshot(ctx->snapshot_builder);
+	RestoreTransactionSnapshot(snap, MyProc);
+
+	/* Don't need the decoding context anymore */
+	FreeDecodingContext(ctx);
+
+	/* Create a tuple to send consisten WAL location */
+	snprintf(xloc, sizeof(xloc), "%X/%X",
+			 LSN_FORMAT_ARGS(MyReplicationSlot->data.confirmed_flush));
+
+	dest = CreateDestReceiver(DestRemoteSimple);
+	tupdesc = CreateTemplateTupleDesc(1);
+	TupleDescInitBuiltinEntry(tupdesc, (AttrNumber) 1, "consistent_point",
+							  TEXTOID, -1, 0);
+	tstate = begin_tup_output_tupdesc(dest, tupdesc, &TTSOpsVirtual);
+
+	/* consistent wal location */
+	values[0] = CStringGetTextDatum(xloc);
+
+	do_tup_output(tstate, values, nulls);
+	end_tup_output(tstate);
+
+	ReplicationSlotRelease();
+}
+
 /*
  * LogicalDecodingContext 'prepare_write' callback.
  *
@@ -1860,6 +1946,15 @@ exec_replication_command(const char *cmd_string)
 			}
 			break;
 
+		case T_ReplicationSlotSnapshotCmd:
+			{
+				cmdtag = "REPLICATION_SLOT_SNAPSHOT";
+				set_ps_display(cmdtag);
+				ReplicationSlotSnapshot((ReplicationSlotSnapshotCmd *) cmd_node);
+				EndReplicationCommand(cmdtag);
+				break;
+			}
+
 		default:
 			elog(ERROR, "unrecognized replication command node tag: %u",
 				 cmd_node->type);
diff --git a/src/include/nodes/replnodes.h b/src/include/nodes/replnodes.h
index 8ae9c517ff..94aa7d2253 100644
--- a/src/include/nodes/replnodes.h
+++ b/src/include/nodes/replnodes.h
@@ -108,4 +108,15 @@ typedef struct TimeLineHistoryCmd
 	TimeLineID	timeline;
 } TimeLineHistoryCmd;
 
+/* ----------------------
+ *		REPLICATION_SLOT_SNAPSHOT command
+ * ----------------------
+ */
+typedef struct ReplicationSlotSnapshotCmd
+{
+	NodeTag		type;
+	char	   *slotname;
+	List	   *options;
+} ReplicationSlotSnapshotCmd;
+
 #endif							/* REPLNODES_H */
diff --git a/src/include/replication/logical.h b/src/include/replication/logical.h
index 4ad019e25a..3875b32f9d 100644
--- a/src/include/replication/logical.h
+++ b/src/include/replication/logical.h
@@ -125,6 +125,7 @@ extern LogicalDecodingContext *CreateInitDecodingContext(const char *plugin,
 extern LogicalDecodingContext *CreateDecodingContext(XLogRecPtr start_lsn,
 													 List *output_plugin_options,
 													 bool fast_forward,
+													 bool need_full_snapshot,
 													 XLogReaderRoutine *xl_routine,
 													 LogicalOutputPluginWriterPrepareWrite prepare_write,
 													 LogicalOutputPluginWriterWrite do_write,
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 9339f29303..274a162b48 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -384,6 +384,16 @@ typedef WalRcvExecResult *(*walrcv_exec_fn) (WalReceiverConn *conn,
  */
 typedef void (*walrcv_disconnect_fn) (WalReceiverConn *conn);
 
+/*
+ * walrcv_slot_snapshot_fn
+ *
+ * Create a snapshot by an existing replication slot
+ */
+typedef void (*walrcv_slot_snapshot_fn) (WalReceiverConn *conn,
+										 char *slotname,
+										 const WalRcvStreamOptions *options,
+										 XLogRecPtr *lsn);
+
 typedef struct WalReceiverFunctionsType
 {
 	walrcv_connect_fn walrcv_connect;
@@ -401,6 +411,7 @@ typedef struct WalReceiverFunctionsType
 	walrcv_get_backend_pid_fn walrcv_get_backend_pid;
 	walrcv_exec_fn walrcv_exec;
 	walrcv_disconnect_fn walrcv_disconnect;
+	walrcv_slot_snapshot_fn walrcv_slot_snapshot;
 } WalReceiverFunctionsType;
 
 extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
@@ -435,6 +446,8 @@ extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
 	WalReceiverFunctions->walrcv_exec(conn, exec, nRetTypes, retTypes)
 #define walrcv_disconnect(conn) \
 	WalReceiverFunctions->walrcv_disconnect(conn)
+#define walrcv_slot_snapshot(conn, slotname, options, lsn) \
+	WalReceiverFunctions->walrcv_slot_snapshot(conn, slotname, options, lsn)
 
 static inline void
 walrcv_clear_result(WalRcvExecResult *walres)
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 58daeca831..cc21aae582 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2313,6 +2313,7 @@ ReplicationSlotCtlData
 ReplicationSlotOnDisk
 ReplicationSlotPersistency
 ReplicationSlotPersistentData
+ReplicationSlotSnapshotCmd
 ReplicationState
 ReplicationStateCtl
 ReplicationStateOnDisk
-- 
2.25.1

v4-0002-Reuse-Logical-Replication-Background-worker.patchapplication/octet-stream; name=v4-0002-Reuse-Logical-Replication-Background-worker.patchDownload
From 340f7aa726ccab46f5ea14b2ab406235fd1ef521 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH 2/2] Reuse Logical Replication Background worker

This commit allows tablesync workers to move to another table that needs synchronization,
when they're done with the current table in tablesync phase of Logical Replication.

It reduces the overhead of launching/killing a new background worker for each table.
By reusing tablesync workers, replication slots created for tablesync can be reused as well.
Removing the burden of creating/dropping replication slot improves tablesync speed.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/catalogs.sgml                  |  29 ++
 src/backend/catalog/pg_subscription.c       | 285 ++++++++++++-
 src/backend/commands/subscriptioncmds.c     | 238 +++++++----
 src/backend/replication/logical/launcher.c  |   9 +-
 src/backend/replication/logical/tablesync.c | 434 +++++++++++++++-----
 src/backend/replication/logical/worker.c    | 370 +++++++++++------
 src/include/catalog/pg_subscription.h       |   6 +
 src/include/catalog/pg_subscription_rel.h   |  15 +-
 src/include/replication/slot.h              |   3 +-
 src/include/replication/worker_internal.h   |  32 +-
 src/test/regress/expected/misc_sanity.out   |  30 +-
 11 files changed, 1095 insertions(+), 356 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 9ed2b020b7..e81a60c90e 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -7987,6 +7987,17 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        origin.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>sublastusedid</structfield> <type>int8</type>
+      </para>
+      <para>
+       The last used ID for tablesync workers. This ID is used to
+       create replication slots. The last used ID needs to be stored
+       to make logical replication can safely proceed after an interruption.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
@@ -8071,6 +8082,24 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        otherwise null
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srrelslotname</structfield> <type>text</type>
+      </para>
+      <para>
+       Replication slot name that is used for synchronization of relation
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srreloriginname</structfield> <type>text</type>
+      </para>
+      <para>
+       Origin name that is used for tracking synchronization of relation
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index a506fc3ec8..ef0c1ec48a 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -71,7 +71,6 @@ GetSubscription(Oid subid, bool missing_ok)
 	sub->stream = subform->substream;
 	sub->twophasestate = subform->subtwophasestate;
 	sub->disableonerr = subform->subdisableonerr;
-
 	/* Get conninfo */
 	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
 							tup,
@@ -114,6 +113,14 @@ GetSubscription(Oid subid, bool missing_ok)
 	Assert(!isnull);
 	sub->origin = TextDatumGetCString(datum);
 
+	/* Get last used id */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
+							tup,
+							Anum_pg_subscription_sublastusedid,
+							&isnull);
+	Assert(!isnull);
+	sub->lastusedid = DatumGetInt64(datum);
+
 	ReleaseSysCache(tup);
 
 	return sub;
@@ -205,6 +212,44 @@ DisableSubscription(Oid subid)
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update the last used replication slot ID for the given subscription.
+ */
+void
+UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid)
+{
+	Relation	rel;
+	bool		nulls[Natts_pg_subscription];
+	bool		replaces[Natts_pg_subscription];
+	Datum		values[Natts_pg_subscription];
+	HeapTuple	tup;
+
+	/* Look up the subscription in the catalog */
+	rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+	tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	/* Form a new tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	replaces[Anum_pg_subscription_sublastusedid - 1] = true;
+	values[Anum_pg_subscription_sublastusedid- 1] = Int64GetDatum(lastusedid);
+
+	/* Update the catalog */
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+	heap_freetuple(tup);
+
+	table_close(rel, NoLock);
+}
+
 /*
  * Convert text array to list of strings.
  *
@@ -234,7 +279,7 @@ textarray_to_stringlist(ArrayType *textarray)
  */
 void
 AddSubscriptionRelState(Oid subid, Oid relid, char state,
-						XLogRecPtr sublsn)
+						XLogRecPtr sublsn, char *relslotname, char *reloriginname)
 {
 	Relation	rel;
 	HeapTuple	tup;
@@ -263,6 +308,14 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
 	else
 		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] = CStringGetTextDatum(relslotname);
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] = CStringGetTextDatum(reloriginname);
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -275,6 +328,58 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Internal function to modify columns for relation state update
+ */
+static void
+UpdateSubscriptionRelState_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char state,
+									XLogRecPtr sublsn)
+{
+	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
+	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+
+	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (sublsn != InvalidXLogRecPtr)
+		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
+	else
+		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication slot update
+ */
+static void
+UpdateSubscriptionRelReplicationSlot_internal(Datum *values,
+											bool *nulls,
+											bool *replaces,
+											char *relslotname)
+{
+	replaces[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] = CStringGetTextDatum(relslotname);
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for origin update
+ */
+static void
+UpdateSubscriptionRelOrigin_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char *reloriginname)
+{
+	replaces[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] = CStringGetTextDatum(reloriginname);
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+}
+
 /*
  * Update the state of a subscription table.
  */
@@ -305,14 +410,48 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	memset(nulls, false, sizeof(nulls));
 	memset(replaces, false, sizeof(replaces));
 
-	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
-	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
 
-	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
-	if (sublsn != InvalidXLogRecPtr)
-		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
-	else
-		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update the replication slot name of a subscription table.
+ */
+void
+UpdateSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *relslotname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
 
 	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
 							replaces);
@@ -324,6 +463,134 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update replication slot name, origin name and state of
+ * a subscription table in one transaction.
+ */
+void
+UpdateSubscriptionRel(Oid subid,
+					  Oid relid,
+					  char state,
+					  XLogRecPtr sublsn,
+					  char *relslotname,
+					  char *reloriginname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
+	UpdateSubscriptionRelOrigin_internal(values, nulls, replaces, reloriginname);
+
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Get origin name of subscription table.
+ *
+ * Returns null if the subscription table does not have a origin.
+ */
+void
+GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*originname;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srreloriginname, isnull);
+	if (!*isnull)
+	{
+		originname = TextDatumGetCString(d);
+		memcpy(reloriginname, originname, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
+/*
+ * Get replication slot name of subscription table.
+ *
+ * Returns null if the subscription table does not have a replication slot.
+ */
+void
+GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*relrepslot;
+	bool		isnull;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srrelslotname, &isnull);
+	if (!isnull)
+	{
+		relrepslot = TextDatumGetCString(d);
+		memcpy(slotname, relrepslot, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
 /*
  * Get state of subscription table.
  *
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index d673557ea4..1b861f129c 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -648,6 +648,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 		publicationListToArray(publications);
 	values[Anum_pg_subscription_suborigin - 1] =
 		CStringGetTextDatum(opts.origin);
+	values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(1);
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -657,7 +658,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 
 	recordDependencyOnOwner(SubscriptionRelationId, subid, owner);
 
-	ReplicationOriginNameForLogicalRep(subid, InvalidOid, originname, sizeof(originname));
+	ReplicationOriginNameForLogicalRep(subid, originname, sizeof(originname), false);
 	replorigin_create(originname);
 
 	/*
@@ -708,7 +709,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 										 rv->schemaname, rv->relname);
 
 				AddSubscriptionRelState(subid, relid, table_state,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 			}
 
 			/*
@@ -798,6 +799,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	} SubRemoveRels;
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -875,7 +878,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				AddSubscriptionRelState(sub->oid, relid,
 										copy_data ? SUBREL_STATE_INIT : SUBREL_STATE_READY,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 				ereport(DEBUG1,
 						(errmsg_internal("table \"%s.%s\" added to subscription \"%s\"",
 										 rv->schemaname, rv->relname, sub->name)));
@@ -899,6 +902,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				char		state;
 				XLogRecPtr	statelsn;
+				char		slotname[NAMEDATALEN] = {0};
 
 				/*
 				 * Lock pg_subscription_rel with AccessExclusiveLock to
@@ -925,7 +929,29 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/*
+				 * Find the logical replication sync worker if exists Store
+				 * the slot number for dropping associated replication slot
+				 * later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot_name);
+				}
+				else
+				{
+					/*
+					 * Sync of this relation might be failed in an earlier
+					 * attempt, but the replication slot might still exist.
+					 */
+					GetSubscriptionRelReplicationSlot(sub->oid, relid, slotname);
+					if (strlen(slotname) > 0)
+						sub_remove_slots = lappend(sub_remove_slots, slotname);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
@@ -945,8 +971,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 					 * origin and by this time the origin might be already
 					 * removed. For these reasons, passing missing_ok = true.
 					 */
-					ReplicationOriginNameForLogicalRep(sub->oid, relid, originname,
-													   sizeof(originname));
+					ReplicationOriginNameForLogicalRep(sub->oid, originname,
+													   sizeof(originname), true);
 					replorigin_drop_by_name(originname, true, false);
 				}
 
@@ -959,31 +985,24 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
-		 * the database operations we won't be able to rollback dropped slots.
+		 * Drop the replication slots associated with tablesync workers for
+		 * removed tables. This has to be at the end because otherwise if
+		 * there is an error while doing the database operations we won't be
+		 * able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			memcpy(syncslotname, lfirst(lc), sizeof(NAMEDATALEN));
+
+			/*
+			 * There is no certainty, maybe the slot does not exist yet. Also,
+			 * if we fail after removing some of the slots, next time, it will
+			 * again try to drop already dropped slots and fail. For these
+			 * reasons, we allow missing_ok = true for the drop.
+			 */
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1319,8 +1338,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
 					char		originname[NAMEDATALEN];
 					XLogRecPtr	remote_lsn;
 
-					ReplicationOriginNameForLogicalRep(subid, InvalidOid,
-													   originname, sizeof(originname));
+					ReplicationOriginNameForLogicalRep(subid, originname,
+													   sizeof(originname), false);
 					originid = replorigin_by_name(originname, false);
 					remote_lsn = replorigin_get_progress(originid, false);
 
@@ -1380,6 +1399,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	char	   *subname;
 	char	   *conninfo;
 	char	   *slotname;
+	int64		lastusedid;
 	List	   *subworkers;
 	ListCell   *lc;
 	char		originname[NAMEDATALEN];
@@ -1451,6 +1471,14 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	else
 		slotname = NULL;
 
+	/* Get the last used identifier by the subscription */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID, tup,
+							Anum_pg_subscription_sublastusedid, &isnull);
+	if (!isnull)
+		lastusedid = DatumGetInt64(datum);
+	else
+		lastusedid = 0;
+
 	/*
 	 * Since dropping a replication slot is not transactional, the replication
 	 * slot stays dropped even if the transaction rolls back.  So we cannot
@@ -1500,37 +1528,29 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	}
 	list_free(subworkers);
 
+	rstates = GetSubscriptionRelations(subid, true);
+
 	/*
-	 * Cleanup of tablesync replication origins.
-	 *
-	 * Any READY-state relations would already have dealt with clean-ups.
+	 * Cleanup of tablesync replication origins associated with the
+	 * subscription, if exists. Try to drop origins by creating all origin
+	 * names created for this subscription.
 	 *
 	 * Note that the state can't change because we have already stopped both
 	 * the apply and tablesync workers and they can't restart because of
 	 * exclusive lock on the subscription.
+	 *
+	 * TODO - should handle better instead of looping through all possible
 	 */
-	rstates = GetSubscriptionRelations(subid, true);
-	foreach(lc, rstates)
+	for (int64 i = 1; i <= lastusedid; i++)
 	{
-		SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-		Oid			relid = rstate->relid;
-
-		/* Only cleanup resources of tablesync workers */
-		if (!OidIsValid(relid))
-			continue;
+		char		originname_to_drop[NAMEDATALEN] = {0};
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * It is possible that the origin is not yet created for tablesync
-		 * worker so passing missing_ok = true. This can happen for the states
-		 * before SUBREL_STATE_FINISHEDCOPY.
-		 */
-		ReplicationOriginNameForLogicalRep(subid, relid, originname,
-										   sizeof(originname));
-		replorigin_drop_by_name(originname, true, false);
+		snprintf(originname_to_drop, sizeof(originname_to_drop), "pg_%u_%ld", subid, i);
+		/* missin_ok = true, since the origin might be already dropped. */
+		replorigin_drop_by_name(originname_to_drop, true, false);
 	}
 
+
 	/* Clean up dependencies */
 	deleteSharedDependencyRecordsFor(SubscriptionRelationId, subid, 0);
 
@@ -1538,7 +1558,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	RemoveSubscriptionRel(subid, InvalidOid);
 
 	/* Remove the origin tracking if exists. */
-	ReplicationOriginNameForLogicalRep(subid, InvalidOid, originname, sizeof(originname));
+	ReplicationOriginNameForLogicalRep(subid, originname, sizeof(originname), false);
 	replorigin_drop_by_name(originname, true, false);
 
 	/*
@@ -1582,39 +1602,17 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
+		List	   *slots = NULL;
 
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
 
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char	   *syncslotname = (char *) lfirst(lc);
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1637,6 +1635,71 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * WRITE COMMENT HERE
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok)
+{
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+					 " FROM pg_replication_slots"
+					 " WHERE slot_name LIKE 'pg_%i_sync_%%';",
+					 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+					errmsg("not tuple returned."));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();
+	\
+		return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -1989,6 +2052,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -1999,18 +2063,20 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
-			char		syncslotname[NAMEDATALEN] = {0};
-
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
-				 syncslotname);
+				 worker->slot_name);
 		}
 	}
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 73594c698a..5b5dead996 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -264,7 +264,7 @@ logicalrep_workers_find(Oid subid, bool only_running)
  */
 void
 logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
-						 Oid relid)
+						 Oid relid, int64 slotid)
 {
 	BackgroundWorker bgw;
 	BackgroundWorkerHandle *bgw_handle;
@@ -371,7 +371,11 @@ retry:
 	/* Prepare the worker slot. */
 	worker->launch_time = now;
 	worker->in_use = true;
+	worker->is_first_run = true;
 	worker->generation++;
+	worker->created_slot = false;
+	worker->rep_slot_id = slotid;
+	worker->slot_name = (char *) palloc(NAMEDATALEN);
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -379,6 +383,7 @@ retry:
 	worker->relid = relid;
 	worker->relstate = SUBREL_STATE_UNKNOWN;
 	worker->relstate_lsn = InvalidXLogRecPtr;
+	worker->move_to_next_rel = false;
 	worker->stream_fileset = NULL;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
@@ -869,7 +874,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, 0);
 				}
 			}
 
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 94e813ac53..a6364d9433 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -126,12 +126,8 @@ static bool FetchTableStates(bool *started_tx);
 
 static StringInfo copybuf = NULL;
 
-/*
- * Exit routine for synchronization worker.
- */
 static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+clean_sync_worker(void)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -143,18 +139,28 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
-	/* And flush all writes. */
-	XLogFlush(GetXLogWriteRecPtr());
-
-	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
-	CommitTransactionCommand();
+	/*
+	 * Disconnect from publisher. Otherwise reused sync workers causes
+	 * exceeding max_wal_senders
+	 */
+	walrcv_disconnect(LogRepWorkerWalRcvConn);
+	LogRepWorkerWalRcvConn = NULL;
 
 	/* Find the main apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
+}
+
+/*
+ * Exit routine for synchronization worker.
+ */
+static void
+pg_attribute_noreturn()
+finish_sync_worker(void)
+{
+	clean_sync_worker();
+
+	/* And flush all writes. */
+	XLogFlush(GetXLogWriteRecPtr());
 
 	/* Stop gracefully */
 	proc_exit(0);
@@ -284,6 +290,10 @@ invalidate_syncing_table_states(Datum arg, int cacheid, uint32 hashvalue)
 static void
 process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 {
+	List	   *rstates;
+	SubscriptionRelState *rstate;
+	ListCell   *lc;
+
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_CATCHUP &&
@@ -292,6 +302,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		TimeLineID	tli;
 		char		syncslotname[NAMEDATALEN] = {0};
 		char		originname[NAMEDATALEN] = {0};
+		bool		is_streaming_ended = false;
 
 		MyLogicalRepWorker->relstate = SUBREL_STATE_SYNCDONE;
 		MyLogicalRepWorker->relstate_lsn = current_lsn;
@@ -308,40 +319,29 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 								   MyLogicalRepWorker->relid,
 								   MyLogicalRepWorker->relstate,
 								   MyLogicalRepWorker->relstate_lsn);
+		CommitTransactionCommand();
 
 		/*
-		 * End streaming so that LogRepWorkerWalRcvConn can be used to drop
-		 * the slot.
-		 */
-		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
-
-		/*
-		 * Cleanup the tablesync slot.
+		 * Cleanup the tablesync slot. If the slot name used by this worker is
+		 * different from the default slot name for the worker, this means the
+		 * current table had started to being synchronized by another worker
+		 * and replication slot. And this worker is reusing a replication slot
+		 * from a previous attempt. We do not need that replication slot
+		 * anymore.
 		 *
 		 * This has to be done after updating the state because otherwise if
 		 * there is an error while doing the database operations we won't be
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
+										MyLogicalRepWorker->rep_slot_id,
 										syncslotname,
 										sizeof(syncslotname));
 
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
-		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
-
-		CommitTransactionCommand();
-		pgstat_report_stat(false);
-
-		/*
-		 * Start a new transaction to clean up the tablesync origin tracking.
-		 * This transaction will be ended within the finish_sync_worker().
-		 * Now, even, if we fail to remove this here, the apply worker will
-		 * ensure to clean it up afterward.
+		 * We are safe to drop the replication trackin origin after this
+		 * point. Now, even, if we fail to remove this here, the apply worker
+		 * will ensure to clean it up afterward.
 		 *
 		 * We need to do this after the table state is set to SYNCDONE.
 		 * Otherwise, if an error occurs while performing the database
@@ -350,34 +350,125 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 * have been cleared before restart. So, the restarted worker will use
 		 * invalid replication progress state resulting in replay of
 		 * transactions that have already been applied.
+		 *
+		 * Firstly reset the origin session to remove the ownership of the
+		 * slot. This is needed to allow the origin to be dropped or reused
+		 * later.
+		 */
+		replorigin_session_reset();
+		replorigin_session_origin = InvalidRepOriginId;
+		replorigin_session_origin_lsn = InvalidXLogRecPtr;
+		replorigin_session_origin_timestamp = 0;
+
+		StartTransactionCommand();
+		if (MyLogicalRepWorker->slot_name && strcmp(syncslotname, MyLogicalRepWorker->slot_name) != 0)
+		{
+			/*
+			 * End streaming so that LogRepWorkerWalRcvConn can be used to
+			 * drop the slot.
+			 */
+			walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+			is_streaming_ended = true;
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
+
+			ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+											   originname,
+											   sizeof(originname),
+											   true);
+
+			/* Drop replication origin */
+			replorigin_drop_by_name(originname, true, false);
+		}
+
+		/*
+		 * We are safe to remove persisted replication slot and origin data,
+		 * since it's already in SYNCDONE state. They will not be needed
+		 * anymore.
 		 */
+		UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+							  MyLogicalRepWorker->relid,
+							  MyLogicalRepWorker->relstate,
+							  MyLogicalRepWorker->relstate_lsn,
+							  NULL,
+							  NULL);
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid))));
+
+		CommitTransactionCommand();
+		pgstat_report_stat(false);
+
 		StartTransactionCommand();
 
+		/*
+		 * This should return the default origin name for the worker. Even if
+		 * the worker used a different origin for this table, it should be
+		 * dropped and removed from the catalog so far.
+		 */
 		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-										   MyLogicalRepWorker->relid,
 										   originname,
-										   sizeof(originname));
+										   sizeof(originname),
+										   true);
 
 		/*
-		 * Resetting the origin session removes the ownership of the slot.
-		 * This is needed to allow the origin to be dropped.
+		 * Check if any table whose relation state is still INIT. If a table
+		 * in INIT state is found, the worker will not be finished, it will be
+		 * reused instead.
 		 */
-		replorigin_session_reset();
-		replorigin_session_origin = InvalidRepOriginId;
-		replorigin_session_origin_lsn = InvalidXLogRecPtr;
-		replorigin_session_origin_timestamp = 0;
+		rstates = GetSubscriptionRelations(MySubscription->oid, true);
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * There is a chance that the user is concurrently performing refresh
-		 * for the subscription where we remove the table state and its origin
-		 * or the apply worker would have removed this origin. So passing
-		 * missing_ok = true.
-		 */
-		replorigin_drop_by_name(originname, true, false);
+		foreach(lc, rstates)
+		{
+			rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
+			memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
+
+			/*
+			 * Pick the table for the next run if there is not another worker
+			 * already picked that table.
+			 */
+			LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+			if (rstate->state != SUBREL_STATE_SYNCDONE &&
+				!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+			{
+				/* Update worker state for the next table */
+				MyLogicalRepWorker->is_first_run = false;
+				MyLogicalRepWorker->relid = rstate->relid;
+				MyLogicalRepWorker->relstate = rstate->state;
+				MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+				MyLogicalRepWorker->move_to_next_rel = true;
+				LWLockRelease(LogicalRepWorkerLock);
+				break;
+			}
+			LWLockRelease(LogicalRepWorkerLock);
+		}
+
+		/* Cleanup before next run or ending the worker. */
+		if (!MyLogicalRepWorker->move_to_next_rel)
+		{
+			/*
+			 * It is important to give an error if we are unable to drop the
+			 * slot, otherwise, it won't be dropped till the corresponding
+			 * subscription is dropped. So passing missing_ok = false.
+			 */
+			if (MyLogicalRepWorker->created_slot)
+			{
+				/* End streaming if it's not already ended. */
+				if (!is_streaming_ended)
+					walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+				ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
+			}
 
-		finish_sync_worker();
+			/* Drop replication origin before exiting. */
+			replorigin_drop_by_name(originname, true, false);
+
+			finish_sync_worker();
+		}
+		else
+		{
+			clean_sync_worker();
+		}
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -485,6 +576,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 			if (current_lsn >= rstate->lsn)
 			{
 				char		originname[NAMEDATALEN];
+				bool		is_origin_null = true;
 
 				rstate->state = SUBREL_STATE_READY;
 				rstate->lsn = current_lsn;
@@ -505,18 +597,27 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				 * error while dropping we won't restart it to drop the
 				 * origin. So passing missing_ok = true.
 				 */
-				ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-												   rstate->relid,
-												   originname,
-												   sizeof(originname));
-				replorigin_drop_by_name(originname, true, false);
+				GetSubscriptionRelOrigin(MyLogicalRepWorker->subid,
+										 rstate->relid, originname,
+										 &is_origin_null);
+
+				if (!is_origin_null)
+				{
+					replorigin_drop_by_name(originname, true, false);
+				}
 
 				/*
 				 * Update the state to READY only after the origin cleanup.
 				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-										   rstate->relid, rstate->state,
-										   rstate->lsn);
+				UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+									  rstate->relid,
+									  rstate->state,
+									  rstate->lsn,
+									  NULL,
+									  NULL);
+
+				CommitTransactionCommand();
+				started_tx = false;
 			}
 		}
 		else
@@ -605,11 +706,21 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						TimestampDifferenceExceeds(hentry->last_start_time, now,
 												   wal_retrieve_retry_interval))
 					{
+						if (IsTransactionState())
+							CommitTransactionCommand();
+						StartTransactionCommand();
+						started_tx = true;
+
+						MySubscription->lastusedid++;
+						UpdateSubscriptionLastSlotId(MyLogicalRepWorker->subid,
+													 MySubscription->lastusedid);
+
 						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
 												 MySubscription->oid,
 												 MySubscription->name,
 												 MyLogicalRepWorker->userid,
-												 rstate->relid);
+												 rstate->relid,
+												 MySubscription->lastusedid);
 						hentry->last_start_time = now;
 					}
 				}
@@ -802,6 +913,7 @@ fetch_remote_table_info(char *nspname, char *relname,
 		TupleTableSlot *tslot;
 		Oid			attrsRow[] = {INT2VECTOROID};
 		StringInfoData pub_names;
+
 		initStringInfo(&pub_names);
 		foreach(lc, MySubscription->publications)
 		{
@@ -1174,8 +1286,8 @@ copy_table(Relation rel)
  * The name must not exceed NAMEDATALEN - 1 because of remote node constraints
  * on slot name length. We append system_identifier to avoid slot_name
  * collision with subscriptions in other clusters. With the current scheme
- * pg_%u_sync_%u_UINT64_FORMAT (3 + 10 + 6 + 10 + 20 + '\0'), the maximum
- * length of slot_name will be 50.
+ * pg_%u_sync_%lu_UINT64_FORMAT (3 + 10 + 6 + 20 + 20 + '\0'), the maximum
+ * length of slot_name will be 45.
  *
  * The returned slot name is stored in the supplied buffer (syncslotname) with
  * the given size.
@@ -1186,11 +1298,11 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
+ReplicationSlotNameForTablesync(Oid suboid, int64 slotid,
 								char *syncslotname, Size szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%ld_" UINT64_FORMAT, suboid,
+			 slotid, GetSystemIdentifier());
 }
 
 /*
@@ -1213,6 +1325,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	WalRcvExecResult *res;
 	char		originname[NAMEDATALEN];
 	RepOriginId originid;
+	char	   *prev_slotname;
 
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
@@ -1241,7 +1354,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
+									MyLogicalRepWorker->rep_slot_id,
 									slotname,
 									NAMEDATALEN);
 
@@ -1261,11 +1374,25 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC ||
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY);
 
+	/*
+	 * See if tablesync of the current relation has been started with another
+	 * replication slot.
+	 *
+	 * Read previous slot name from the catalog, if exists.
+	 */
+	prev_slotname = (char *) palloc0(NAMEDATALEN);
+	StartTransactionCommand();
+	GetSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+									  MyLogicalRepWorker->relid,
+									  prev_slotname);
+
 	/* Assign the origin tracking record name. */
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
-									   MyLogicalRepWorker->relid,
 									   originname,
-									   sizeof(originname));
+									   sizeof(originname),
+									   true);
+
+	CommitTransactionCommand();
 
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC)
 	{
@@ -1280,10 +1407,48 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		 * breakdown then it wouldn't have succeeded so trying it next time
 		 * seems like a better bet.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, slotname, true);
+		if (strlen(prev_slotname) > 0)
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);
+
+			StartTransactionCommand();
+			/* Replication drop might still exist. Try to drop */
+			replorigin_drop_by_name(originname, true, false);
+
+			/*
+			 * Remove replication slot and origin name from the relation's
+			 * catalog record
+			 */
+			UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+								  MyLogicalRepWorker->relid,
+								  MyLogicalRepWorker->relstate,
+								  MyLogicalRepWorker->relstate_lsn,
+								  NULL,
+								  NULL);
+			CommitTransactionCommand();
+		}
 	}
 	else if (MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY)
 	{
+		/*
+		 * At this point, the table that is currently being synchronized
+		 * should have its replication slot name filled in the catalog. The
+		 * tablesync process was started with another sync worker and
+		 * replication slot. We need to continue using the same replication
+		 * slot in this worker too.
+		 */
+		if (strlen(prev_slotname) == 0)
+		{
+			elog(ERROR, "Replication slot could not be found for relation %u",
+				 MyLogicalRepWorker->relid);
+		}
+
+		/*
+		 * Proceed with the correct replication slot. Use previously created
+		 * replication slot to sync this table.
+		 */
+		slotname = prev_slotname;
+
 		/*
 		 * The COPY phase was previously done, but tablesync then crashed
 		 * before it was able to finish normally.
@@ -1304,6 +1469,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		goto copy_table_done;
 	}
 
+	/* Preparing for table copy operation */
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 	MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
 	MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
@@ -1311,10 +1477,12 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   MyLogicalRepWorker->relstate,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  MyLogicalRepWorker->relstate,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 	CommitTransactionCommand();
 	pgstat_report_stat(true);
 
@@ -1353,6 +1521,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						GetUserNameFromId(GetUserId(), true),
 						RelationGetRelationName(rel))));
 
+
 	/*
 	 * Start a transaction in the remote node in REPEATABLE READ mode.  This
 	 * ensures that both the replication slot we create (see below) and the
@@ -1368,55 +1537,100 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						res->err)));
 	walrcv_clear_result(res);
 
+	originid = replorigin_by_name(originname, true);
+
 	/*
 	 * Create a new permanent logical decoding slot. This slot will be used
 	 * for the catchup phase after COPY is done, so tell it to use the
 	 * snapshot to make the final data consistent.
 	 *
+	 * Replication slot will only be created if either this is the first run
+	 * of the worker or we're not using a previous replication slot.
+	 *
 	 * Prevent cancel/die interrupts while creating slot here because it is
 	 * possible that before the server finishes this command, a concurrent
 	 * drop subscription happens which would complete without removing this
 	 * slot leading to a dangling slot on the server.
+	 *
 	 */
-	HOLD_INTERRUPTS();
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-	RESUME_INTERRUPTS();
-
-	/*
-	 * Setup replication origin tracking. The purpose of doing this before the
-	 * copy is to avoid doing the copy again due to any error in setting up
-	 * origin tracking.
-	 */
-	originid = replorigin_by_name(originname, true);
-	if (!OidIsValid(originid))
+	if (!MyLogicalRepWorker->created_slot)
 	{
+		HOLD_INTERRUPTS();
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						   slotname, false /* permanent */ , false /* two_phase */ ,
+						   CRS_USE_SNAPSHOT, origin_startpos);
+		RESUME_INTERRUPTS();
+
 		/*
-		 * Origin tracking does not exist, so create it now.
-		 *
-		 * Then advance to the LSN got from walrcv_create_slot. This is WAL
-		 * logged for the purpose of recovery. Locks are to prevent the
-		 * replication origin from vanishing while advancing.
+		 * Remember that we created the slot so that we will not try to create
+		 * it again.
 		 */
-		originid = replorigin_create(originname);
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->created_slot = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 
-		LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
-		replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
-						   true /* go backward */ , true /* WAL log */ );
-		UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+		/*
+		 * Setup replication origin tracking. The purpose of doing this before
+		 * the copy is to avoid doing the copy again due to any error in
+		 * setting up origin tracking.
+		 */
 
-		replorigin_session_setup(originid);
-		replorigin_session_origin = originid;
+		if (!OidIsValid(originid))
+		{
+			/*
+			 * Origin tracking does not exist, so create it now.
+			 */
+			originid = replorigin_create(originname);
+		}
+		else
+		{
+			/*
+			 * At this point, there shouldn't be any existing replication
+			 * origin wit the same name.
+			 */
+			ereport(ERROR,
+					(errcode(ERRCODE_DUPLICATE_OBJECT),
+					 errmsg("replication origin \"%s\" already exists",
+							originname)));
+		}
 	}
 	else
 	{
-		ereport(ERROR,
-				(errcode(ERRCODE_DUPLICATE_OBJECT),
-				 errmsg("replication origin \"%s\" already exists",
-						originname)));
+		/*
+		 * Do not create a new replication slot, reuse the existing one
+		 * instead. Use a new snapshot for the replication slot to ensure that
+		 * tablesync and apply proceses are consistent with each other.
+		 */
+		WalRcvStreamOptions options;
+		int			server_version;
+
+		server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+		options.proto.logical.proto_version =
+			server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+			server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+			LOGICALREP_PROTO_VERSION_NUM;
+		options.proto.logical.publication_names = MySubscription->publications;
+
+		HOLD_INTERRUPTS();
+		walrcv_slot_snapshot(LogRepWorkerWalRcvConn, slotname, &options, origin_startpos);
+		RESUME_INTERRUPTS();
 	}
 
+	/*
+	 * Advance to the LSN got from walrcv_create_slot. This is WAL
+	 * logged for the purpose of recovery. Locks are to prevent the
+	 * replication origin from vanishing while advancing.
+	 *
+	 * Then setup replication origin tracking.
+	 */
+	LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+	replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
+					   true /* go backward */ , true /* WAL log */ );
+	UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+
+	replorigin_session_setup(originid);
+	replorigin_session_origin = originid;
+
 	/* Now do the initial data copy */
 	PushActiveSnapshot(GetTransactionSnapshot());
 	copy_table(rel);
@@ -1439,10 +1653,12 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	 * Update the persisted state to indicate the COPY phase is done; make it
 	 * visible to others.
 	 */
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   SUBREL_STATE_FINISHEDCOPY,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  SUBREL_STATE_FINISHEDCOPY,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 
 	CommitTransactionCommand();
 
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index f9efe6c4c6..08349e8b5f 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -316,6 +316,7 @@ static void stream_open_file(Oid subid, TransactionId xid,
 							 bool first_segment);
 static void stream_write_change(char action, StringInfo s);
 static void stream_close_file(void);
+static void stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
@@ -368,18 +369,26 @@ static inline void reset_apply_error_context_info(void);
  * Form the origin name for the subscription.
  *
  * This is a common function for tablesync and other workers. Tablesync workers
- * must pass a valid relid. Other callers must pass relid = InvalidOid.
+ * must pass is_tablesync true so that origin name includes slot id.
  *
  * Return the name in the supplied buffer.
  */
 void
-ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
-								   char *originname, Size szoriginname)
+ReplicationOriginNameForLogicalRep(Oid suboid, char *originname,
+								   Size szoriginname, bool is_tablesync)
 {
-	if (OidIsValid(relid))
+	if (is_tablesync)
 	{
-		/* Replication origin name for tablesync workers. */
-		snprintf(originname, szoriginname, "pg_%u_%u", suboid, relid);
+		bool		is_null = true;
+
+		/*
+		 * Replication origin name for tablesync workers. First, look into the
+		 * catalog. If originname does not exist, then use the default name.
+		 */
+		GetSubscriptionRelOrigin(suboid, MyLogicalRepWorker->relid,
+								 originname, &is_null);
+		if (is_null)
+			snprintf(originname, szoriginname, "pg_%u_%ld", suboid, MyLogicalRepWorker->rep_slot_id);
 	}
 	else
 	{
@@ -2834,6 +2843,23 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can move to next table while running
+				 * process_syncing_tables_for_sync. Before we were able to
+				 * reuse tablesync workers, that
+				 * process_syncing_tables_for_sync call would exit the worker
+				 * instead of moving to next table. Now that tablesync workers
+				 * can be reused, we need to take care of memory contexts here
+				 * before moving to sync a table.
+				 */
+				if (MyLogicalRepWorker->move_to_next_rel)
+				{
+					MemoryContextResetAndDeleteChildren(ApplyMessageContext);
+					MemoryContextSwitchTo(TopMemoryContext);
+					return;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -2853,6 +2879,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+			if (MyLogicalRepWorker->move_to_next_rel)
+			{
+				endofstream = true;
+			}
 		}
 
 		/* Cleanup the memory. */
@@ -2954,8 +2984,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	/* Pop the error context stack */
 	error_context_stack = errcallback.previous;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * If it's moving to next relation, this is a sync worker. Sync workers
+	 * end the streaming during process_syncing_tables_for_sync. Calling
+	 * endstreaming twice causes "no COPY in progress" errors.
+	 */
+	if (!MyLogicalRepWorker->move_to_next_rel)
+	{
+		/* All done */
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	}
 }
 
 /*
@@ -3497,6 +3535,35 @@ stream_write_change(char action, StringInfo s)
 	BufFileWrite(stream_fd, &s->data[s->cursor], len);
 }
 
+/*
+ * stream_build_options_replication
+ * 		Build logical replication streaming options.
+ *
+ * This function sets streaming options including replication slot name
+ * and origin start position. Workers need these options for logical replication.
+ */
+static void
+stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+	options->proto.logical.streaming = MySubscription->stream;
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -3571,6 +3638,9 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 
 	/* allocate slot name in long-lived context */
 	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+
+	/* Keep the replication slot name used for this sync. */
+	MyLogicalRepWorker->slot_name = *myslotname;
 	pfree(syncslotname);
 }
 
@@ -3608,6 +3678,140 @@ start_apply(XLogRecPtr origin_startpos)
 	PG_END_TRY();
 }
 
+/*
+ * Runs the tablesync worker.
+ * It starts table sync. After successful sync,
+ * builds streaming options and starts streaming.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Set this to false for safety, in case we're already reusing the worker */
+	MyLogicalRepWorker->move_to_next_rel = false;
+
+	start_table_sync(origin_startpos, &slotname);
+
+	/*
+	 * Allocate the origin name in long-lived context for error context
+	 * message.
+	 */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   originname,
+									   originname_size,
+									   true);
+	CommitTransactionCommand();
+
+	apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
+															   originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/* Start normal logical streaming replication. */
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+}
+
+/*
+ * Runs the apply worker.
+ * It sets up replication origin, the streaming options
+ * and then starts streaming.
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
+{
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+
+	slotname = MySubscription->slotname;
+
+	/*
+	 * This shouldn't happen if the subscription is enabled, but guard against
+	 * DDL bugs or manual catalog changes.  (libpqwalreceiver will crash if
+	 * slot is NULL.)
+	 */
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+				 errmsg("subscription has no replication slot set")));
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, originname,
+									   originname_size, false);
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											MySubscription->name, &err);
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+				 errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+	 * We don't really use the output identify_system for anything but it does
+	 * some initializations on the upstream so let's still call it.
+	 */
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	/*
+	 * Allocate the origin name in long-lived context for error context
+	 * message.
+	 */
+	apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
+															   originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+	{
+		/* Start streaming with two_phase enabled */
+		options->proto.logical.twophase = true;
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
+	}
+	else
+	{
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+	}
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+}
+
 /* Logical Replication Apply worker entry point */
 void
 ApplyWorkerMain(Datum main_arg)
@@ -3618,7 +3822,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	/* Attach to slot */
 	logicalrep_worker_attach(worker_slot);
@@ -3709,143 +3912,48 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		/*
-		 * Allocate the origin name in long-lived context for error context
-		 * message.
-		 */
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
-																   originname);
-	}
-	else
-	{
-		/* This is main apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		/*
-		 * Allocate the origin name in long-lived context for error context
-		 * message.
-		 */
-		apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
-																   originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
-	 * the subscription relation state.
+	 * the subscription relation state. Do this outside the loop to avoid
+	 * exceeding MAX_SYSCACHE_CALLBACKS
 	 */
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
-
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
-
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-	options.proto.logical.streaming = MySubscription->stream;
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
-
-	if (!am_tablesync_worker())
+	/*
+	 * The loop where worker does its job. It loops until the worker is not
+	 * reused.
+	 */
+	while (MyLogicalRepWorker->is_first_run ||
+		   MyLogicalRepWorker->move_to_next_rel)
 	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
+		if (am_tablesync_worker())
 		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
+			/*
+			 * This is a tablesync worker. Start syncing tables before
+			 * starting the apply loop.
+			 */
+			run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 		}
 		else
 		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+			/* This is main apply worker */
+			run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 		}
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-						MySubscription->name,
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-						"?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
-
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+		/* Run the main loop. */
+		start_apply(origin_startpos);
 
+		if (MyLogicalRepWorker->move_to_next_rel)
+		{
+			StartTransactionCommand();
+			ereport(LOG,
+					(errmsg("logical replication table synchronization worker for subscription \"%s\" has moved to sync table \"%s\".",
+							MySubscription->name, get_rel_name(MyLogicalRepWorker->relid))));
+			CommitTransactionCommand();
+		}
+	}
 	proc_exit(0);
 }
 
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 7b98714f30..0b13dd34cb 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -102,6 +102,9 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
 
 	/* Only publish data originating from the specified origin */
 	text		suborigin BKI_DEFAULT(LOGICALREP_ORIGIN_ANY);
+
+	/* The last used ID to create a replication slot for tablesync */
+	int64		sublastusedid BKI_DEFAULT(0);
 #endif
 } FormData_pg_subscription;
 
@@ -135,11 +138,14 @@ typedef struct Subscription
 	List	   *publications;	/* List of publication names to subscribe to */
 	char	   *origin;			/* Only publish data originating from the
 								 * specified origin */
+	int64		lastusedid;		/* Last used unique ID to create replication
+								 * slots in tablesync */
 } Subscription;
 
 extern Subscription *GetSubscription(Oid subid, bool missing_ok);
 extern void FreeSubscription(Subscription *sub);
 extern void DisableSubscription(Oid subid);
+extern void UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid);
 
 extern int	CountDBSubscriptions(Oid dbid);
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 8e88de7b2b..9cafb521f9 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -44,6 +44,12 @@ CATALOG(pg_subscription_rel,6102,SubscriptionRelRelationId)
 											 * used for synchronization
 											 * coordination, or NULL if not
 											 * valid */
+	text		srrelslotname BKI_FORCE_NULL;	/* name of the replication
+												 * slot for relation in
+												 * subscription */
+	text		srreloriginname BKI_FORCE_NULL; /* origin name for relation in
+												 * subscription */
+
 #endif
 } FormData_pg_subscription_rel;
 
@@ -81,10 +87,17 @@ typedef struct SubscriptionRelState
 } SubscriptionRelState;
 
 extern void AddSubscriptionRelState(Oid subid, Oid relid, char state,
-									XLogRecPtr sublsn);
+									XLogRecPtr sublsn, char *relslotname, char *reloriginname);
 extern void UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 									   XLogRecPtr sublsn);
+extern void UpdateSubscriptionRel(Oid subid, Oid relid, char state,
+								  XLogRecPtr sublsn, char *relslotname, char *reloriginname);
+extern void UpdateSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *relslotname);
+
 extern char GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn);
+extern void GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname);
+extern void GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull);
+
 extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 
 extern bool HasSubscriptionRelations(Oid subid);
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 65f2c74239..aca6b590f9 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -219,8 +219,9 @@ extern bool InvalidateObsoleteReplicationSlots(XLogSegNo oldestSegno);
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
 extern bool ReplicationSlotName(int index, Name name);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, Size szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, int64 slotid, char *syncslotname, Size szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List *GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 2b7114ff6d..e7f74e0f9f 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -30,6 +30,26 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/*
+	 * Indicates if worker is running for the first time or in reuse
+	 */
+	bool		is_first_run;
+
+	/*
+	 * Indicates if the sync worker created a replication slot or it reuses an
+	 * existing one created by another worker.
+	 */
+	bool		created_slot;
+
+	/*
+	 * Unique identifier for replication slot to be created by tablesnync
+	 * workers, if needed.
+	 */
+	int64		rep_slot_id;
+
+	/* Replication slot name used by the worker. */
+	char	   *slot_name;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
@@ -51,6 +71,12 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/*
+	 * Used to indicate whether sync worker will be reused for another
+	 * relation
+	 */
+	bool		move_to_next_rel;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction, the
@@ -85,15 +111,15 @@ 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);
+									 Oid userid, Oid relid, int64 slotid);
 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);
 
 extern int	logicalrep_sync_worker_count(Oid subid);
 
-extern void ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
-											   char *originname, Size szoriginname);
+extern void ReplicationOriginNameForLogicalRep(Oid suboid, char *originname,
+											   Size szoriginname, bool is_tablesync);
 extern char *LogicalRepSyncTableStart(XLogRecPtr *origin_startpos);
 
 extern bool AllTablesyncsReady(void);
diff --git a/src/test/regress/expected/misc_sanity.out b/src/test/regress/expected/misc_sanity.out
index a57fd142a9..3d34a21421 100644
--- a/src/test/regress/expected/misc_sanity.out
+++ b/src/test/regress/expected/misc_sanity.out
@@ -47,20 +47,22 @@ WHERE c.oid < 16384 AND
       relkind = 'r' AND
       attstorage != 'p'
 ORDER BY 1, 2;
-         relname         |    attname    |   atttypid   
--------------------------+---------------+--------------
- pg_attribute            | attacl        | aclitem[]
- pg_attribute            | attfdwoptions | text[]
- pg_attribute            | attmissingval | anyarray
- pg_attribute            | attoptions    | text[]
- pg_class                | relacl        | aclitem[]
- pg_class                | reloptions    | text[]
- pg_class                | relpartbound  | pg_node_tree
- pg_index                | indexprs      | pg_node_tree
- pg_index                | indpred       | pg_node_tree
- pg_largeobject          | data          | bytea
- pg_largeobject_metadata | lomacl        | aclitem[]
-(11 rows)
+         relname         |     attname     |   atttypid   
+-------------------------+-----------------+--------------
+ pg_attribute            | attacl          | aclitem[]
+ pg_attribute            | attfdwoptions   | text[]
+ pg_attribute            | attmissingval   | anyarray
+ pg_attribute            | attoptions      | text[]
+ pg_class                | relacl          | aclitem[]
+ pg_class                | reloptions      | text[]
+ pg_class                | relpartbound    | pg_node_tree
+ pg_index                | indexprs        | pg_node_tree
+ pg_index                | indpred         | pg_node_tree
+ pg_largeobject          | data            | bytea
+ pg_largeobject_metadata | lomacl          | aclitem[]
+ pg_subscription_rel     | srreloriginname | text
+ pg_subscription_rel     | srrelslotname   | text
+(13 rows)
 
 -- system catalogs without primary keys
 --
-- 
2.25.1

#19Amit Kapila
amit.kapila16@gmail.com
In reply to: Melih Mutlu (#18)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Mon, Dec 5, 2022 at 6:30 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Attached two patches:
0001: adds REPLICATION_SLOT_SNAPSHOT command for replication protocol.
0002: Reuses workers/replication slots and origins for tablesync

I would appreciate any feedback/review/thought on the approach and both patches.
I will also share some numbers to compare performances of the patch and master branch soon.

It would be interesting to see the numbers differently for resue of
replication slots and origins. This will let us know how much each of
those optimizations helps with the reuse of workers.

--
With Regards,
Amit Kapila.

#20Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Melih Mutlu (#18)
2 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi,

Attached new versions of the patch with some changes/fixes.

Here also some numbers to compare the performance of log. rep. with this
patch against the current master branch.

My method of benchmarking is the same with what I did earlier in this
thread. (on a different environment, so not compare the result from this
email with the ones from earlier emails)

With those changes, I did some benchmarking to see if it improves anything.
This results compares this patch with the latest version of master branch.
"max_sync_workers_per_subscription" is set to 2 as default.
Got some results simply averaging timings from 5 consecutive runs for each
branch.

Since this patch is expected to improve log. rep. of empty/close-to-empty
tables, started with measuring performance with empty tables.

| 10 tables | 100 tables | 1000 tables
------------------------------------------------------------------------------
master | 283.430 ms | 22739.107 ms | 105226.177 ms
------------------------------------------------------------------------------
patch | 189.139 ms | 1554.802 ms | 23091.434 ms

After the changes discussed here [1], concurrent replication origin drops

by apply worker and tablesync workers may hold each other on wait due to
locks taken by replorigin_drop_by_name.
I see that this harms the performance of logical replication quite a bit
in terms of speed.
[1]
/messages/by-id/20220714115155.GA5439@depesz.com

Firstly, as I mentioned, replication origin drops made things worse for the
master branch.
Locks start being a more serious issue when the number of tables increases.
The patch reuses the origin so does not need to drop them in each
iteration. That's why the difference between the master and the patch is
more significant now than it was when I first sent the patch.

To just show that the improvement is not only the result of reuse of
origins, but also reuse of rep. slots and workers, I just reverted those
commits which causes the origin drop issue.

| 10 tables | 100 tables | 1000 tables
-----------------------------------------------------------------------------
reverted | 270.012 ms | 2483.907 ms | 31660.758 ms
-----------------------------------------------------------------------------
patch | 189.139 ms | 1554.802 ms | 23091.434 ms

With this patch, logical replication is still faster, even if we wouldn't
have an issue with rep. origin drops.

Also here are some numbers with 10 tables loaded with some data :

| 10 MB | 100 MB
----------------------------------------------------------
master | 2868.524 ms | 14281.711 ms
----------------------------------------------------------
patch | 1750.226 ms | 14592.800 ms

The difference between the master and the patch is getting close when the
size of tables increase, as expected.

I would appreciate any feedback/thought on the approach/patch/numbers etc.

Thanks,
--
Melih Mutlu
Microsoft

Attachments:

v2-0001-Add-replication-protocol-cmd-to-create-a-snapshot.patchapplication/octet-stream; name=v2-0001-Add-replication-protocol-cmd-to-create-a-snapshot.patchDownload
From 9ffa22049582ac0464306d301b57aba8610376e7 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 13 Oct 2022 17:05:45 +0300
Subject: [PATCH 1/2] Add replication protocol cmd to create a snapshot

Introduced REPLICATION_SLOT_SNAPSHOT to be able to create and use a
snapshot without creating a new replication slot, but by using an
existing slot.

REPLICATION_SLOT_SNAPSHOT simply does what CREATE_REPLICATION_SLOT does
without creating a new replication slot.

REPLICATION_SLOT_SNAPSHOT command imports the snapshot into the current
transaction and returns consistent_point. The changes earlier than the
consistent_point will be applied by importing the snapshot. All changes
later than the consistent_point will be available to be consumed from
the replication slot.

This is useful for reusing replication slots in logical replication.
Otherwise, tablesync workers cannot start from a consistent point to copy a relation and then apply changes by consuming from replication slot.
---
 doc/src/sgml/protocol.sgml                    | 32 ++++++
 .../libpqwalreceiver/libpqwalreceiver.c       | 69 ++++++++++++-
 src/backend/replication/logical/logical.c     | 39 +++++++-
 .../replication/logical/logicalfuncs.c        |  1 +
 src/backend/replication/repl_gram.y           | 18 +++-
 src/backend/replication/repl_scanner.l        |  2 +
 src/backend/replication/slotfuncs.c           |  1 +
 src/backend/replication/walsender.c           | 97 ++++++++++++++++++-
 src/include/nodes/replnodes.h                 | 11 +++
 src/include/replication/logical.h             |  1 +
 src/include/replication/walreceiver.h         | 13 +++
 src/tools/pgindent/typedefs.list              |  1 +
 12 files changed, 281 insertions(+), 4 deletions(-)

diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 03312e07e2..f4f29079ac 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2613,6 +2613,38 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
      </listitem>
     </varlistentry>
 
+    <varlistentry id="protocol-replication-replication-slot-snapshot">
+     <term><literal>REPLICATION_SLOT_SNAPSHOT</literal> <replaceable class="parameter">slot_name</replaceable> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
+      <indexterm><primary>REPLICATION_SLOT_SNAPSHOT</primary></indexterm>
+     </term>
+     <listitem>
+      <para>
+       Creates a snapshot including all the changes from the replication slot until
+       the point at which the replication slot becomes consistent. Then the snapshot
+       is used in the currenct transaction. This command is currently only supported
+       for logical replication.
+       slots.
+      </para>
+
+      <para>
+       In response to this command, the server will return a one-row result set,
+       containing the following field:
+       <variablelist>
+        <varlistentry>
+         <term><literal>consistent_point</literal> (<type>text</type>)</term>
+         <listitem>
+          <para>
+           The WAL location at which the slot became consistent.  This is the
+           earliest location from which streaming can start on this replication
+           slot.
+          </para>
+         </listitem>
+        </varlistentry>
+       </variablelist>
+      </para>
+     </listitem>
+    </varlistentry>
+
     <varlistentry id="protocol-replication-base-backup" xreflabel="BASE_BACKUP">
      <term><literal>BASE_BACKUP</literal> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
       <indexterm><primary>BASE_BACKUP</primary></indexterm>
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index 7f697b0f29..e367c5f757 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -80,6 +80,8 @@ static WalRcvExecResult *libpqrcv_exec(WalReceiverConn *conn,
 									   const int nRetTypes,
 									   const Oid *retTypes);
 static void libpqrcv_disconnect(WalReceiverConn *conn);
+static void libpqrcv_slot_snapshot(WalReceiverConn *conn, char *slotname,
+								   const WalRcvStreamOptions *options, XLogRecPtr *lsn);
 
 static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_connect = libpqrcv_connect,
@@ -96,7 +98,8 @@ static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_create_slot = libpqrcv_create_slot,
 	.walrcv_get_backend_pid = libpqrcv_get_backend_pid,
 	.walrcv_exec = libpqrcv_exec,
-	.walrcv_disconnect = libpqrcv_disconnect
+	.walrcv_disconnect = libpqrcv_disconnect,
+	.walrcv_slot_snapshot = libpqrcv_slot_snapshot
 };
 
 /* Prototypes for private functions */
@@ -968,6 +971,70 @@ libpqrcv_create_slot(WalReceiverConn *conn, const char *slotname,
 	return snapshot;
 }
 
+/*
+ * TODO
+ */
+static void
+libpqrcv_slot_snapshot(WalReceiverConn *conn,
+					   char *slotname,
+					   const WalRcvStreamOptions *options,
+					   XLogRecPtr *lsn)
+{
+	StringInfoData cmd;
+	PGresult   *res;
+	char	   *pubnames_str;
+	List	   *pubnames;
+	char	   *pubnames_literal;
+
+	initStringInfo(&cmd);
+
+	/* Build the command. */
+	appendStringInfo(&cmd, "REPLICATION_SLOT_SNAPSHOT \"%s\"", slotname);
+	appendStringInfoString(&cmd, " (");
+	appendStringInfo(&cmd, " proto_version '%u'",
+					 options->proto.logical.proto_version);
+
+	/* Add publication names. */
+	pubnames = options->proto.logical.publication_names;
+	pubnames_str = stringlist_to_identifierstr(conn->streamConn, pubnames);
+	if (!pubnames_str)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	pubnames_literal = PQescapeLiteral(conn->streamConn, pubnames_str,
+									   strlen(pubnames_str));
+	if (!pubnames_literal)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	appendStringInfo(&cmd, ", publication_names %s", pubnames_literal);
+	PQfreemem(pubnames_literal);
+	pfree(pubnames_str);
+
+	appendStringInfoString(&cmd, " )");
+
+	/* Execute the command. */
+	res = libpqrcv_PQexec(conn->streamConn, cmd.data);
+	pfree(cmd.data);
+
+	if (PQresultStatus(res) != PGRES_TUPLES_OK)
+	{
+		PQclear(res);
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg("Could not create a snapshot by replication slot \"%s\": %s",
+						slotname, pchomp(PQerrorMessage(conn->streamConn)))));
+	}
+
+	if (lsn)
+		*lsn = DatumGetLSN(DirectFunctionCall1Coll(pg_lsn_in, InvalidOid,
+												   CStringGetDatum(PQgetvalue(res, 0, 0))));
+
+	PQclear(res);
+}
+
 /*
  * Return PID of remote backend process.
  */
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 625a7f4273..16c0a27ba5 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -461,6 +461,10 @@ CreateInitDecodingContext(const char *plugin,
  * fast_forward
  *		bypass the generation of logical changes.
  *
+ * need_full_snapshot
+ * 		if true, create a snapshot able to read all tables,
+ * 		otherwise do not create any snapshot.
+ *
  * xl_routine
  *		XLogReaderRoutine used by underlying xlogreader
  *
@@ -479,6 +483,7 @@ LogicalDecodingContext *
 CreateDecodingContext(XLogRecPtr start_lsn,
 					  List *output_plugin_options,
 					  bool fast_forward,
+					  bool need_full_snapshot,
 					  XLogReaderRoutine *xl_routine,
 					  LogicalOutputPluginWriterPrepareWrite prepare_write,
 					  LogicalOutputPluginWriterWrite do_write,
@@ -487,6 +492,7 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 	LogicalDecodingContext *ctx;
 	ReplicationSlot *slot;
 	MemoryContext old_context;
+	TransactionId xmin_horizon = InvalidTransactionId;
 
 	/* shorter lines... */
 	slot = MyReplicationSlot;
@@ -533,8 +539,39 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 		start_lsn = slot->data.confirmed_flush;
 	}
 
+
+	/*
+	 * We need to determine a safe xmin horizon to start decoding from if we
+	 * want to create a snapshot too. Otherwise we would end up with a
+	 * snapshot that cannot be imported since xmin value from the snapshot may
+	 * be less than the oldest safe xmin. To avoid this call
+	 * GetOldestSafeDecodingTransactionId() to return a safe xmin value, which
+	 * can be used while exporting/importing the snapshot.
+	 *
+	 * So we have to acquire the ProcArrayLock to prevent computation of new
+	 * xmin horizons by other backends, get the safe decoding xid, and inform
+	 * the slot machinery about the new limit. Once that's done the
+	 * ProcArrayLock can be released as the slot machinery now is protecting
+	 * against vacuum.
+	 */
+	if (need_full_snapshot)
+	{
+		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+		SpinLockAcquire(&slot->mutex);
+		slot->effective_catalog_xmin = xmin_horizon;
+		slot->data.catalog_xmin = xmin_horizon;
+		slot->effective_xmin = xmin_horizon;
+		SpinLockRelease(&slot->mutex);
+
+		xmin_horizon = GetOldestSafeDecodingTransactionId(!need_full_snapshot);
+		ReplicationSlotsComputeRequiredXmin(true);
+
+		LWLockRelease(ProcArrayLock);
+	}
+
 	ctx = StartupDecodingContext(output_plugin_options,
-								 start_lsn, InvalidTransactionId, false,
+								 start_lsn, xmin_horizon, need_full_snapshot,
 								 fast_forward, xl_routine, prepare_write,
 								 do_write, update_progress);
 
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index 5c23178570..e4b80f2764 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -208,6 +208,7 @@ pg_logical_slot_get_changes_guts(FunctionCallInfo fcinfo, bool confirm, bool bin
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									options,
 									false,
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/repl_gram.y b/src/backend/replication/repl_gram.y
index 7ba33d6672..f70b2e087a 100644
--- a/src/backend/replication/repl_gram.y
+++ b/src/backend/replication/repl_gram.y
@@ -65,6 +65,7 @@ Node *replication_parse_result;
 %token K_CREATE_REPLICATION_SLOT
 %token K_DROP_REPLICATION_SLOT
 %token K_TIMELINE_HISTORY
+%token K_REPLICATION_SLOT_SNAPSHOT
 %token K_WAIT
 %token K_TIMELINE
 %token K_PHYSICAL
@@ -80,7 +81,7 @@ Node *replication_parse_result;
 %type <node>	command
 %type <node>	base_backup start_replication start_logical_replication
 				create_replication_slot drop_replication_slot identify_system
-				read_replication_slot timeline_history show
+				read_replication_slot timeline_history show replication_slot_snapshot
 %type <list>	generic_option_list
 %type <defelt>	generic_option
 %type <uintval>	opt_timeline
@@ -114,6 +115,7 @@ command:
 			| read_replication_slot
 			| timeline_history
 			| show
+			| replication_slot_snapshot
 			;
 
 /*
@@ -307,6 +309,19 @@ timeline_history:
 				}
 			;
 
+/*
+ * REPLICATION_SLOT_SNAPSHOT %s options
+ */
+replication_slot_snapshot:
+			K_REPLICATION_SLOT_SNAPSHOT var_name plugin_options
+				{
+					ReplicationSlotSnapshotCmd *n = makeNode(ReplicationSlotSnapshotCmd);
+					n->slotname = $2;
+					n->options = $3;
+					$$ = (Node *) n;
+				}
+			;
+
 opt_physical:
 			K_PHYSICAL
 			| /* EMPTY */
@@ -400,6 +415,7 @@ ident_or_keyword:
 			| K_CREATE_REPLICATION_SLOT	{ $$ = "create_replication_slot"; }
 			| K_DROP_REPLICATION_SLOT		{ $$ = "drop_replication_slot"; }
 			| K_TIMELINE_HISTORY			{ $$ = "timeline_history"; }
+			| K_REPLICATION_SLOT_SNAPSHOT	{ $$ = "replication_slot_snapshot"; }
 			| K_WAIT						{ $$ = "wait"; }
 			| K_TIMELINE					{ $$ = "timeline"; }
 			| K_PHYSICAL					{ $$ = "physical"; }
diff --git a/src/backend/replication/repl_scanner.l b/src/backend/replication/repl_scanner.l
index 72ef12225e..e33030da61 100644
--- a/src/backend/replication/repl_scanner.l
+++ b/src/backend/replication/repl_scanner.l
@@ -126,6 +126,7 @@ START_REPLICATION	{ return K_START_REPLICATION; }
 CREATE_REPLICATION_SLOT		{ return K_CREATE_REPLICATION_SLOT; }
 DROP_REPLICATION_SLOT		{ return K_DROP_REPLICATION_SLOT; }
 TIMELINE_HISTORY	{ return K_TIMELINE_HISTORY; }
+REPLICATION_SLOT_SNAPSHOT	{ return K_REPLICATION_SLOT_SNAPSHOT; }
 PHYSICAL			{ return K_PHYSICAL; }
 RESERVE_WAL			{ return K_RESERVE_WAL; }
 LOGICAL				{ return K_LOGICAL; }
@@ -303,6 +304,7 @@ replication_scanner_is_replication_command(void)
 		case K_DROP_REPLICATION_SLOT:
 		case K_READ_REPLICATION_SLOT:
 		case K_TIMELINE_HISTORY:
+		case K_REPLICATION_SLOT_SNAPSHOT:
 		case K_SHOW:
 			/* Yes; push back the first token so we can parse later. */
 			repl_pushed_back_token = first_token;
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 79d7f1372d..13d099fe06 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -478,6 +478,7 @@ pg_logical_replication_slot_advance(XLogRecPtr moveto)
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									NIL,
 									true,	/* fast_forward */
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index c11bb3716f..f9a22c8b16 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -238,6 +238,7 @@ static void CreateReplicationSlot(CreateReplicationSlotCmd *cmd);
 static void DropReplicationSlot(DropReplicationSlotCmd *cmd);
 static void StartReplication(StartReplicationCmd *cmd);
 static void StartLogicalReplication(StartReplicationCmd *cmd);
+static void ReplicationSlotSnapshot(ReplicationSlotSnapshotCmd *cmd);
 static void ProcessStandbyMessage(void);
 static void ProcessStandbyReplyMessage(void);
 static void ProcessStandbyHSFeedbackMessage(void);
@@ -1280,7 +1281,7 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	 * are reported early.
 	 */
 	logical_decoding_ctx =
-		CreateDecodingContext(cmd->startpoint, cmd->options, false,
+		CreateDecodingContext(cmd->startpoint, cmd->options, false, false,
 							  XL_ROUTINE(.page_read = logical_read_xlog_page,
 										 .segment_open = WalSndSegmentOpen,
 										 .segment_close = wal_segment_close),
@@ -1332,6 +1333,91 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	EndCommand(&qc, DestRemote, false);
 }
 
+/*
+ * Create a snapshot from an existing replication slot.
+ */
+static void
+ReplicationSlotSnapshot(ReplicationSlotSnapshotCmd *cmd)
+{
+	Snapshot	snap;
+	LogicalDecodingContext *ctx;
+	char		xloc[MAXFNAMELEN];
+	DestReceiver *dest;
+	TupOutputState *tstate;
+	TupleDesc	tupdesc;
+	Datum		values[1];
+	bool		nulls[1] = {0};
+
+	Assert(!MyReplicationSlot);
+
+	if (!IsTransactionBlock())
+		ereport(ERROR,
+				(errmsg("%s must be called inside a transaction",
+						"REPLICATION_SLOT_SNAPSHOT ...")));
+
+	if (XactIsoLevel != XACT_REPEATABLE_READ)
+		ereport(ERROR,
+				(errmsg("%s must be called in REPEATABLE READ isolation mode transaction",
+						"REPLICATION_SLOT_SNAPSHOT ...")));
+
+	if (FirstSnapshotSet)
+		ereport(ERROR,
+				(errmsg("%s must be called before any query",
+						"REPLICATION_SLOT_SNAPSHOT ...")));
+
+	if (IsSubTransaction())
+		ereport(ERROR,
+				(errmsg("%s must not be called in a subtransaction",
+						"REPLICATION_SLOT_SNAPSHOT ...")));
+
+	ReplicationSlotAcquire(cmd->slotname, false);
+
+	ctx = CreateDecodingContext(MyReplicationSlot->data.restart_lsn,
+								cmd->options,
+								false,
+								true,
+								XL_ROUTINE(.page_read = logical_read_xlog_page,
+										   .segment_open = WalSndSegmentOpen,
+										   .segment_close = wal_segment_close),
+								WalSndPrepareWrite, WalSndWriteData,
+								WalSndUpdateProgress);
+
+	/*
+	 * Signal that we don't need the timeout mechanism. We're just creating
+	 * the replication slot and don't yet accept feedback messages or send
+	 * keepalives. As we possibly need to wait for further WAL the walsender
+	 * would otherwise possibly be killed too soon.
+	 */
+	last_reply_timestamp = 0;
+
+	/* build initial snapshot, might take a while */
+	DecodingContextFindStartpoint(ctx);
+
+	snap = SnapBuildInitialSnapshot(ctx->snapshot_builder);
+	RestoreTransactionSnapshot(snap, MyProc);
+
+	/* Don't need the decoding context anymore */
+	FreeDecodingContext(ctx);
+
+	/* Create a tuple to send consisten WAL location */
+	snprintf(xloc, sizeof(xloc), "%X/%X",
+			 LSN_FORMAT_ARGS(MyReplicationSlot->data.confirmed_flush));
+
+	dest = CreateDestReceiver(DestRemoteSimple);
+	tupdesc = CreateTemplateTupleDesc(1);
+	TupleDescInitBuiltinEntry(tupdesc, (AttrNumber) 1, "consistent_point",
+							  TEXTOID, -1, 0);
+	tstate = begin_tup_output_tupdesc(dest, tupdesc, &TTSOpsVirtual);
+
+	/* consistent wal location */
+	values[0] = CStringGetTextDatum(xloc);
+
+	do_tup_output(tstate, values, nulls);
+	end_tup_output(tstate);
+
+	ReplicationSlotRelease();
+}
+
 /*
  * LogicalDecodingContext 'prepare_write' callback.
  *
@@ -1860,6 +1946,15 @@ exec_replication_command(const char *cmd_string)
 			}
 			break;
 
+		case T_ReplicationSlotSnapshotCmd:
+			{
+				cmdtag = "REPLICATION_SLOT_SNAPSHOT";
+				set_ps_display(cmdtag);
+				ReplicationSlotSnapshot((ReplicationSlotSnapshotCmd *) cmd_node);
+				EndReplicationCommand(cmdtag);
+				break;
+			}
+
 		default:
 			elog(ERROR, "unrecognized replication command node tag: %u",
 				 cmd_node->type);
diff --git a/src/include/nodes/replnodes.h b/src/include/nodes/replnodes.h
index 8ae9c517ff..94aa7d2253 100644
--- a/src/include/nodes/replnodes.h
+++ b/src/include/nodes/replnodes.h
@@ -108,4 +108,15 @@ typedef struct TimeLineHistoryCmd
 	TimeLineID	timeline;
 } TimeLineHistoryCmd;
 
+/* ----------------------
+ *		REPLICATION_SLOT_SNAPSHOT command
+ * ----------------------
+ */
+typedef struct ReplicationSlotSnapshotCmd
+{
+	NodeTag		type;
+	char	   *slotname;
+	List	   *options;
+} ReplicationSlotSnapshotCmd;
+
 #endif							/* REPLNODES_H */
diff --git a/src/include/replication/logical.h b/src/include/replication/logical.h
index 4ad019e25a..3875b32f9d 100644
--- a/src/include/replication/logical.h
+++ b/src/include/replication/logical.h
@@ -125,6 +125,7 @@ extern LogicalDecodingContext *CreateInitDecodingContext(const char *plugin,
 extern LogicalDecodingContext *CreateDecodingContext(XLogRecPtr start_lsn,
 													 List *output_plugin_options,
 													 bool fast_forward,
+													 bool need_full_snapshot,
 													 XLogReaderRoutine *xl_routine,
 													 LogicalOutputPluginWriterPrepareWrite prepare_write,
 													 LogicalOutputPluginWriterWrite do_write,
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 9339f29303..274a162b48 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -384,6 +384,16 @@ typedef WalRcvExecResult *(*walrcv_exec_fn) (WalReceiverConn *conn,
  */
 typedef void (*walrcv_disconnect_fn) (WalReceiverConn *conn);
 
+/*
+ * walrcv_slot_snapshot_fn
+ *
+ * Create a snapshot by an existing replication slot
+ */
+typedef void (*walrcv_slot_snapshot_fn) (WalReceiverConn *conn,
+										 char *slotname,
+										 const WalRcvStreamOptions *options,
+										 XLogRecPtr *lsn);
+
 typedef struct WalReceiverFunctionsType
 {
 	walrcv_connect_fn walrcv_connect;
@@ -401,6 +411,7 @@ typedef struct WalReceiverFunctionsType
 	walrcv_get_backend_pid_fn walrcv_get_backend_pid;
 	walrcv_exec_fn walrcv_exec;
 	walrcv_disconnect_fn walrcv_disconnect;
+	walrcv_slot_snapshot_fn walrcv_slot_snapshot;
 } WalReceiverFunctionsType;
 
 extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
@@ -435,6 +446,8 @@ extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
 	WalReceiverFunctions->walrcv_exec(conn, exec, nRetTypes, retTypes)
 #define walrcv_disconnect(conn) \
 	WalReceiverFunctions->walrcv_disconnect(conn)
+#define walrcv_slot_snapshot(conn, slotname, options, lsn) \
+	WalReceiverFunctions->walrcv_slot_snapshot(conn, slotname, options, lsn)
 
 static inline void
 walrcv_clear_result(WalRcvExecResult *walres)
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 60c71d05fe..6b90a4a693 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2315,6 +2315,7 @@ ReplicationSlotCtlData
 ReplicationSlotOnDisk
 ReplicationSlotPersistency
 ReplicationSlotPersistentData
+ReplicationSlotSnapshotCmd
 ReplicationState
 ReplicationStateCtl
 ReplicationStateOnDisk
-- 
2.25.1

v5-0002-Reuse-Logical-Replication-Background-worker.patchapplication/octet-stream; name=v5-0002-Reuse-Logical-Replication-Background-worker.patchDownload
From ce74b9a18a2a425ed39533f435fefaa2431285c2 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH 2/2] Reuse Logical Replication Background worker

This commit allows tablesync workers to move to another table that needs synchronization,
when they're done with the current table in tablesync phase of Logical Replication.

Before this commit, tablesync workers were capable of syncing only one
relation. A new worker, replication slot and origin were needed for each
relation in the subscription.

Now, tablesync workers are not only limited with one relation and can move to another relation and reuse existing
replication slots and origins

This reduces the overhead of launching/killing a new background worker for each relation.
By reusing tablesync workers, replication slots and origins created for tablesync can be reused as well.
Removing the burden of creating/dropping replication slot/origin improves tablesync speed significantly especially for empty or small tables.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/catalogs.sgml                  |  29 ++
 src/backend/catalog/pg_subscription.c       | 284 ++++++++++++-
 src/backend/commands/subscriptioncmds.c     | 238 +++++++----
 src/backend/replication/logical/launcher.c  |   9 +-
 src/backend/replication/logical/tablesync.c | 434 +++++++++++++++-----
 src/backend/replication/logical/worker.c    | 370 +++++++++++------
 src/include/catalog/pg_subscription.h       |   6 +
 src/include/catalog/pg_subscription_rel.h   |  15 +-
 src/include/replication/slot.h              |   3 +-
 src/include/replication/worker_internal.h   |  32 +-
 src/test/regress/expected/misc_sanity.out   |  30 +-
 11 files changed, 1095 insertions(+), 355 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 9316b811ac..7d55db6354 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -7997,6 +7997,17 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        origin.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>sublastusedid</structfield> <type>int8</type>
+      </para>
+      <para>
+       The last used ID for tablesync workers. This ID is used to
+       create replication slots. The last used ID needs to be stored
+       to make logical replication can safely proceed after any interruption.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
@@ -8081,6 +8092,24 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        otherwise null
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srrelslotname</structfield> <type>text</type>
+      </para>
+      <para>
+       Replication slot name that is used for synchronization of relation
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srreloriginname</structfield> <type>text</type>
+      </para>
+      <para>
+       Origin name that is used for tracking synchronization of relation
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index a506fc3ec8..1f637df242 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -114,6 +114,14 @@ GetSubscription(Oid subid, bool missing_ok)
 	Assert(!isnull);
 	sub->origin = TextDatumGetCString(datum);
 
+	/* Get last used id */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
+							tup,
+							Anum_pg_subscription_sublastusedid,
+							&isnull);
+	Assert(!isnull);
+	sub->lastusedid = DatumGetInt64(datum);
+
 	ReleaseSysCache(tup);
 
 	return sub;
@@ -205,6 +213,44 @@ DisableSubscription(Oid subid)
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update the last used replication slot ID for the given subscription.
+ */
+void
+UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid)
+{
+	Relation	rel;
+	bool		nulls[Natts_pg_subscription];
+	bool		replaces[Natts_pg_subscription];
+	Datum		values[Natts_pg_subscription];
+	HeapTuple	tup;
+
+	/* Look up the subscription in the catalog */
+	rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+	tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	/* Form a new tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	replaces[Anum_pg_subscription_sublastusedid - 1] = true;
+	values[Anum_pg_subscription_sublastusedid- 1] = Int64GetDatum(lastusedid);
+
+	/* Update the catalog */
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+	heap_freetuple(tup);
+
+	table_close(rel, NoLock);
+}
+
 /*
  * Convert text array to list of strings.
  *
@@ -234,7 +280,7 @@ textarray_to_stringlist(ArrayType *textarray)
  */
 void
 AddSubscriptionRelState(Oid subid, Oid relid, char state,
-						XLogRecPtr sublsn)
+						XLogRecPtr sublsn, char *relslotname, char *reloriginname)
 {
 	Relation	rel;
 	HeapTuple	tup;
@@ -263,6 +309,14 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
 	else
 		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] = CStringGetTextDatum(relslotname);
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] = CStringGetTextDatum(reloriginname);
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -275,6 +329,58 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Internal function to modify columns for relation state update
+ */
+static void
+UpdateSubscriptionRelState_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char state,
+									XLogRecPtr sublsn)
+{
+	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
+	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+
+	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (sublsn != InvalidXLogRecPtr)
+		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
+	else
+		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication slot update
+ */
+static void
+UpdateSubscriptionRelReplicationSlot_internal(Datum *values,
+											bool *nulls,
+											bool *replaces,
+											char *relslotname)
+{
+	replaces[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] = CStringGetTextDatum(relslotname);
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication origin update
+ */
+static void
+UpdateSubscriptionRelOrigin_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char *reloriginname)
+{
+	replaces[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] = CStringGetTextDatum(reloriginname);
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+}
+
 /*
  * Update the state of a subscription table.
  */
@@ -305,14 +411,48 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	memset(nulls, false, sizeof(nulls));
 	memset(replaces, false, sizeof(replaces));
 
-	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
-	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
 
-	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
-	if (sublsn != InvalidXLogRecPtr)
-		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
-	else
-		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update the replication slot name of a subscription table.
+ */
+void
+UpdateSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *relslotname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
 
 	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
 							replaces);
@@ -324,6 +464,134 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update replication slot name, origin name and state of
+ * a subscription table in one transaction.
+ */
+void
+UpdateSubscriptionRel(Oid subid,
+					  Oid relid,
+					  char state,
+					  XLogRecPtr sublsn,
+					  char *relslotname,
+					  char *reloriginname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
+	UpdateSubscriptionRelOrigin_internal(values, nulls, replaces, reloriginname);
+
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Get origin name of subscription table.
+ *
+ * Returns null if the subscription table does not have a origin.
+ */
+void
+GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*originname;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srreloriginname, isnull);
+	if (!*isnull)
+	{
+		originname = TextDatumGetCString(d);
+		memcpy(reloriginname, originname, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
+/*
+ * Get replication slot name of subscription table.
+ *
+ * Returns null if the subscription table does not have a replication slot.
+ */
+void
+GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*relrepslot;
+	bool		isnull;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srrelslotname, &isnull);
+	if (!isnull)
+	{
+		relrepslot = TextDatumGetCString(d);
+		memcpy(slotname, relrepslot, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
 /*
  * Get state of subscription table.
  *
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index d673557ea4..ccdd844201 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -648,6 +648,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 		publicationListToArray(publications);
 	values[Anum_pg_subscription_suborigin - 1] =
 		CStringGetTextDatum(opts.origin);
+	values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(1);
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -657,7 +658,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 
 	recordDependencyOnOwner(SubscriptionRelationId, subid, owner);
 
-	ReplicationOriginNameForLogicalRep(subid, InvalidOid, originname, sizeof(originname));
+	ReplicationOriginNameForLogicalRep(subid, originname, sizeof(originname), false);
 	replorigin_create(originname);
 
 	/*
@@ -708,7 +709,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 										 rv->schemaname, rv->relname);
 
 				AddSubscriptionRelState(subid, relid, table_state,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 			}
 
 			/*
@@ -798,6 +799,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	} SubRemoveRels;
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -875,7 +878,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				AddSubscriptionRelState(sub->oid, relid,
 										copy_data ? SUBREL_STATE_INIT : SUBREL_STATE_READY,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 				ereport(DEBUG1,
 						(errmsg_internal("table \"%s.%s\" added to subscription \"%s\"",
 										 rv->schemaname, rv->relname, sub->name)));
@@ -899,6 +902,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				char		state;
 				XLogRecPtr	statelsn;
+				char		slotname[NAMEDATALEN] = {0};
 
 				/*
 				 * Lock pg_subscription_rel with AccessExclusiveLock to
@@ -925,7 +929,29 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/*
+				 * Find the logical replication sync worker if exists store
+				 * the slot number for dropping associated replication slots
+				 * later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot_name);
+				}
+				else
+				{
+					/*
+					 * Sync of this relation might be failed in an earlier
+					 * attempt, but the replication slot might still exist.
+					 */
+					GetSubscriptionRelReplicationSlot(sub->oid, relid, slotname);
+					if (strlen(slotname) > 0)
+						sub_remove_slots = lappend(sub_remove_slots, slotname);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
@@ -945,8 +971,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 					 * origin and by this time the origin might be already
 					 * removed. For these reasons, passing missing_ok = true.
 					 */
-					ReplicationOriginNameForLogicalRep(sub->oid, relid, originname,
-													   sizeof(originname));
+					ReplicationOriginNameForLogicalRep(sub->oid, originname,
+													   sizeof(originname), true);
 					replorigin_drop_by_name(originname, true, false);
 				}
 
@@ -959,31 +985,24 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
-		 * the database operations we won't be able to rollback dropped slots.
+		 * Drop the replication slots associated with tablesync workers for
+		 * removed tables. This has to be at the end because otherwise if
+		 * there is an error while doing the database operations we won't be
+		 * able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			memcpy(syncslotname, lfirst(lc), sizeof(NAMEDATALEN));
+
+			/*
+			 * There is no certainty, maybe the slot does not exist yet. Also,
+			 * if we fail after removing some of the slots, next time, it will
+			 * again try to drop already dropped slots and fail. For these
+			 * reasons, we allow missing_ok = true for the drop.
+			 */
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1319,8 +1338,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
 					char		originname[NAMEDATALEN];
 					XLogRecPtr	remote_lsn;
 
-					ReplicationOriginNameForLogicalRep(subid, InvalidOid,
-													   originname, sizeof(originname));
+					ReplicationOriginNameForLogicalRep(subid, originname,
+													   sizeof(originname), false);
 					originid = replorigin_by_name(originname, false);
 					remote_lsn = replorigin_get_progress(originid, false);
 
@@ -1380,6 +1399,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	char	   *subname;
 	char	   *conninfo;
 	char	   *slotname;
+	int64		lastusedid;
 	List	   *subworkers;
 	ListCell   *lc;
 	char		originname[NAMEDATALEN];
@@ -1451,6 +1471,14 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	else
 		slotname = NULL;
 
+	/* Get the last used identifier by the subscription */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID, tup,
+							Anum_pg_subscription_sublastusedid, &isnull);
+	if (!isnull)
+		lastusedid = DatumGetInt64(datum);
+	else
+		lastusedid = 0;
+
 	/*
 	 * Since dropping a replication slot is not transactional, the replication
 	 * slot stays dropped even if the transaction rolls back.  So we cannot
@@ -1500,37 +1528,29 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	}
 	list_free(subworkers);
 
+	rstates = GetSubscriptionRelations(subid, true);
+
 	/*
-	 * Cleanup of tablesync replication origins.
-	 *
-	 * Any READY-state relations would already have dealt with clean-ups.
+	 * Cleanup of tablesync replication origins associated with the
+	 * subscription, if exists. Try to drop origins by creating all origin
+	 * names created for this subscription.
 	 *
 	 * Note that the state can't change because we have already stopped both
 	 * the apply and tablesync workers and they can't restart because of
 	 * exclusive lock on the subscription.
+	 *
+	 * XXX: This can be handled better instead of looping through all possible
 	 */
-	rstates = GetSubscriptionRelations(subid, true);
-	foreach(lc, rstates)
+	for (int64 i = 1; i <= lastusedid; i++)
 	{
-		SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-		Oid			relid = rstate->relid;
-
-		/* Only cleanup resources of tablesync workers */
-		if (!OidIsValid(relid))
-			continue;
+		char		originname_to_drop[NAMEDATALEN] = {0};
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * It is possible that the origin is not yet created for tablesync
-		 * worker so passing missing_ok = true. This can happen for the states
-		 * before SUBREL_STATE_FINISHEDCOPY.
-		 */
-		ReplicationOriginNameForLogicalRep(subid, relid, originname,
-										   sizeof(originname));
-		replorigin_drop_by_name(originname, true, false);
+		snprintf(originname_to_drop, sizeof(originname_to_drop), "pg_%u_%ld", subid, i);
+		/* missin_ok = true, since the origin might be already dropped. */
+		replorigin_drop_by_name(originname_to_drop, true, false);
 	}
 
+
 	/* Clean up dependencies */
 	deleteSharedDependencyRecordsFor(SubscriptionRelationId, subid, 0);
 
@@ -1538,7 +1558,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	RemoveSubscriptionRel(subid, InvalidOid);
 
 	/* Remove the origin tracking if exists. */
-	ReplicationOriginNameForLogicalRep(subid, InvalidOid, originname, sizeof(originname));
+	ReplicationOriginNameForLogicalRep(subid, originname, sizeof(originname), false);
 	replorigin_drop_by_name(originname, true, false);
 
 	/*
@@ -1582,39 +1602,17 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
+		List	   *slots = NULL;
 
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
 
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char	   *syncslotname = (char *) lfirst(lc);
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1637,6 +1635,71 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * Get the replication slot names associated with the subscription.
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok)
+{
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+					 " FROM pg_replication_slots"
+					 " WHERE slot_name LIKE 'pg_%i_sync_%%';",
+					 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+					errmsg("not tuple returned."));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();
+	\
+		return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -1989,6 +2052,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -1999,18 +2063,20 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
-			char		syncslotname[NAMEDATALEN] = {0};
-
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
-				 syncslotname);
+				 worker->slot_name);
 		}
 	}
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 73594c698a..5b5dead996 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -264,7 +264,7 @@ logicalrep_workers_find(Oid subid, bool only_running)
  */
 void
 logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
-						 Oid relid)
+						 Oid relid, int64 slotid)
 {
 	BackgroundWorker bgw;
 	BackgroundWorkerHandle *bgw_handle;
@@ -371,7 +371,11 @@ retry:
 	/* Prepare the worker slot. */
 	worker->launch_time = now;
 	worker->in_use = true;
+	worker->is_first_run = true;
 	worker->generation++;
+	worker->created_slot = false;
+	worker->rep_slot_id = slotid;
+	worker->slot_name = (char *) palloc(NAMEDATALEN);
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -379,6 +383,7 @@ retry:
 	worker->relid = relid;
 	worker->relstate = SUBREL_STATE_UNKNOWN;
 	worker->relstate_lsn = InvalidXLogRecPtr;
+	worker->move_to_next_rel = false;
 	worker->stream_fileset = NULL;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
@@ -869,7 +874,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, 0);
 				}
 			}
 
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 94e813ac53..a6364d9433 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -126,12 +126,8 @@ static bool FetchTableStates(bool *started_tx);
 
 static StringInfo copybuf = NULL;
 
-/*
- * Exit routine for synchronization worker.
- */
 static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+clean_sync_worker(void)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -143,18 +139,28 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
-	/* And flush all writes. */
-	XLogFlush(GetXLogWriteRecPtr());
-
-	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
-	CommitTransactionCommand();
+	/*
+	 * Disconnect from publisher. Otherwise reused sync workers causes
+	 * exceeding max_wal_senders
+	 */
+	walrcv_disconnect(LogRepWorkerWalRcvConn);
+	LogRepWorkerWalRcvConn = NULL;
 
 	/* Find the main apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
+}
+
+/*
+ * Exit routine for synchronization worker.
+ */
+static void
+pg_attribute_noreturn()
+finish_sync_worker(void)
+{
+	clean_sync_worker();
+
+	/* And flush all writes. */
+	XLogFlush(GetXLogWriteRecPtr());
 
 	/* Stop gracefully */
 	proc_exit(0);
@@ -284,6 +290,10 @@ invalidate_syncing_table_states(Datum arg, int cacheid, uint32 hashvalue)
 static void
 process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 {
+	List	   *rstates;
+	SubscriptionRelState *rstate;
+	ListCell   *lc;
+
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_CATCHUP &&
@@ -292,6 +302,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		TimeLineID	tli;
 		char		syncslotname[NAMEDATALEN] = {0};
 		char		originname[NAMEDATALEN] = {0};
+		bool		is_streaming_ended = false;
 
 		MyLogicalRepWorker->relstate = SUBREL_STATE_SYNCDONE;
 		MyLogicalRepWorker->relstate_lsn = current_lsn;
@@ -308,40 +319,29 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 								   MyLogicalRepWorker->relid,
 								   MyLogicalRepWorker->relstate,
 								   MyLogicalRepWorker->relstate_lsn);
+		CommitTransactionCommand();
 
 		/*
-		 * End streaming so that LogRepWorkerWalRcvConn can be used to drop
-		 * the slot.
-		 */
-		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
-
-		/*
-		 * Cleanup the tablesync slot.
+		 * Cleanup the tablesync slot. If the slot name used by this worker is
+		 * different from the default slot name for the worker, this means the
+		 * current table had started to being synchronized by another worker
+		 * and replication slot. And this worker is reusing a replication slot
+		 * from a previous attempt. We do not need that replication slot
+		 * anymore.
 		 *
 		 * This has to be done after updating the state because otherwise if
 		 * there is an error while doing the database operations we won't be
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
+										MyLogicalRepWorker->rep_slot_id,
 										syncslotname,
 										sizeof(syncslotname));
 
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
-		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
-
-		CommitTransactionCommand();
-		pgstat_report_stat(false);
-
-		/*
-		 * Start a new transaction to clean up the tablesync origin tracking.
-		 * This transaction will be ended within the finish_sync_worker().
-		 * Now, even, if we fail to remove this here, the apply worker will
-		 * ensure to clean it up afterward.
+		 * We are safe to drop the replication trackin origin after this
+		 * point. Now, even, if we fail to remove this here, the apply worker
+		 * will ensure to clean it up afterward.
 		 *
 		 * We need to do this after the table state is set to SYNCDONE.
 		 * Otherwise, if an error occurs while performing the database
@@ -350,34 +350,125 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 * have been cleared before restart. So, the restarted worker will use
 		 * invalid replication progress state resulting in replay of
 		 * transactions that have already been applied.
+		 *
+		 * Firstly reset the origin session to remove the ownership of the
+		 * slot. This is needed to allow the origin to be dropped or reused
+		 * later.
+		 */
+		replorigin_session_reset();
+		replorigin_session_origin = InvalidRepOriginId;
+		replorigin_session_origin_lsn = InvalidXLogRecPtr;
+		replorigin_session_origin_timestamp = 0;
+
+		StartTransactionCommand();
+		if (MyLogicalRepWorker->slot_name && strcmp(syncslotname, MyLogicalRepWorker->slot_name) != 0)
+		{
+			/*
+			 * End streaming so that LogRepWorkerWalRcvConn can be used to
+			 * drop the slot.
+			 */
+			walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+			is_streaming_ended = true;
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
+
+			ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+											   originname,
+											   sizeof(originname),
+											   true);
+
+			/* Drop replication origin */
+			replorigin_drop_by_name(originname, true, false);
+		}
+
+		/*
+		 * We are safe to remove persisted replication slot and origin data,
+		 * since it's already in SYNCDONE state. They will not be needed
+		 * anymore.
 		 */
+		UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+							  MyLogicalRepWorker->relid,
+							  MyLogicalRepWorker->relstate,
+							  MyLogicalRepWorker->relstate_lsn,
+							  NULL,
+							  NULL);
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid))));
+
+		CommitTransactionCommand();
+		pgstat_report_stat(false);
+
 		StartTransactionCommand();
 
+		/*
+		 * This should return the default origin name for the worker. Even if
+		 * the worker used a different origin for this table, it should be
+		 * dropped and removed from the catalog so far.
+		 */
 		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-										   MyLogicalRepWorker->relid,
 										   originname,
-										   sizeof(originname));
+										   sizeof(originname),
+										   true);
 
 		/*
-		 * Resetting the origin session removes the ownership of the slot.
-		 * This is needed to allow the origin to be dropped.
+		 * Check if any table whose relation state is still INIT. If a table
+		 * in INIT state is found, the worker will not be finished, it will be
+		 * reused instead.
 		 */
-		replorigin_session_reset();
-		replorigin_session_origin = InvalidRepOriginId;
-		replorigin_session_origin_lsn = InvalidXLogRecPtr;
-		replorigin_session_origin_timestamp = 0;
+		rstates = GetSubscriptionRelations(MySubscription->oid, true);
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * There is a chance that the user is concurrently performing refresh
-		 * for the subscription where we remove the table state and its origin
-		 * or the apply worker would have removed this origin. So passing
-		 * missing_ok = true.
-		 */
-		replorigin_drop_by_name(originname, true, false);
+		foreach(lc, rstates)
+		{
+			rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
+			memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
+
+			/*
+			 * Pick the table for the next run if there is not another worker
+			 * already picked that table.
+			 */
+			LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+			if (rstate->state != SUBREL_STATE_SYNCDONE &&
+				!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+			{
+				/* Update worker state for the next table */
+				MyLogicalRepWorker->is_first_run = false;
+				MyLogicalRepWorker->relid = rstate->relid;
+				MyLogicalRepWorker->relstate = rstate->state;
+				MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+				MyLogicalRepWorker->move_to_next_rel = true;
+				LWLockRelease(LogicalRepWorkerLock);
+				break;
+			}
+			LWLockRelease(LogicalRepWorkerLock);
+		}
+
+		/* Cleanup before next run or ending the worker. */
+		if (!MyLogicalRepWorker->move_to_next_rel)
+		{
+			/*
+			 * It is important to give an error if we are unable to drop the
+			 * slot, otherwise, it won't be dropped till the corresponding
+			 * subscription is dropped. So passing missing_ok = false.
+			 */
+			if (MyLogicalRepWorker->created_slot)
+			{
+				/* End streaming if it's not already ended. */
+				if (!is_streaming_ended)
+					walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+				ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
+			}
 
-		finish_sync_worker();
+			/* Drop replication origin before exiting. */
+			replorigin_drop_by_name(originname, true, false);
+
+			finish_sync_worker();
+		}
+		else
+		{
+			clean_sync_worker();
+		}
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -485,6 +576,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 			if (current_lsn >= rstate->lsn)
 			{
 				char		originname[NAMEDATALEN];
+				bool		is_origin_null = true;
 
 				rstate->state = SUBREL_STATE_READY;
 				rstate->lsn = current_lsn;
@@ -505,18 +597,27 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				 * error while dropping we won't restart it to drop the
 				 * origin. So passing missing_ok = true.
 				 */
-				ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-												   rstate->relid,
-												   originname,
-												   sizeof(originname));
-				replorigin_drop_by_name(originname, true, false);
+				GetSubscriptionRelOrigin(MyLogicalRepWorker->subid,
+										 rstate->relid, originname,
+										 &is_origin_null);
+
+				if (!is_origin_null)
+				{
+					replorigin_drop_by_name(originname, true, false);
+				}
 
 				/*
 				 * Update the state to READY only after the origin cleanup.
 				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-										   rstate->relid, rstate->state,
-										   rstate->lsn);
+				UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+									  rstate->relid,
+									  rstate->state,
+									  rstate->lsn,
+									  NULL,
+									  NULL);
+
+				CommitTransactionCommand();
+				started_tx = false;
 			}
 		}
 		else
@@ -605,11 +706,21 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						TimestampDifferenceExceeds(hentry->last_start_time, now,
 												   wal_retrieve_retry_interval))
 					{
+						if (IsTransactionState())
+							CommitTransactionCommand();
+						StartTransactionCommand();
+						started_tx = true;
+
+						MySubscription->lastusedid++;
+						UpdateSubscriptionLastSlotId(MyLogicalRepWorker->subid,
+													 MySubscription->lastusedid);
+
 						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
 												 MySubscription->oid,
 												 MySubscription->name,
 												 MyLogicalRepWorker->userid,
-												 rstate->relid);
+												 rstate->relid,
+												 MySubscription->lastusedid);
 						hentry->last_start_time = now;
 					}
 				}
@@ -802,6 +913,7 @@ fetch_remote_table_info(char *nspname, char *relname,
 		TupleTableSlot *tslot;
 		Oid			attrsRow[] = {INT2VECTOROID};
 		StringInfoData pub_names;
+
 		initStringInfo(&pub_names);
 		foreach(lc, MySubscription->publications)
 		{
@@ -1174,8 +1286,8 @@ copy_table(Relation rel)
  * The name must not exceed NAMEDATALEN - 1 because of remote node constraints
  * on slot name length. We append system_identifier to avoid slot_name
  * collision with subscriptions in other clusters. With the current scheme
- * pg_%u_sync_%u_UINT64_FORMAT (3 + 10 + 6 + 10 + 20 + '\0'), the maximum
- * length of slot_name will be 50.
+ * pg_%u_sync_%lu_UINT64_FORMAT (3 + 10 + 6 + 20 + 20 + '\0'), the maximum
+ * length of slot_name will be 45.
  *
  * The returned slot name is stored in the supplied buffer (syncslotname) with
  * the given size.
@@ -1186,11 +1298,11 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
+ReplicationSlotNameForTablesync(Oid suboid, int64 slotid,
 								char *syncslotname, Size szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%ld_" UINT64_FORMAT, suboid,
+			 slotid, GetSystemIdentifier());
 }
 
 /*
@@ -1213,6 +1325,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	WalRcvExecResult *res;
 	char		originname[NAMEDATALEN];
 	RepOriginId originid;
+	char	   *prev_slotname;
 
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
@@ -1241,7 +1354,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
+									MyLogicalRepWorker->rep_slot_id,
 									slotname,
 									NAMEDATALEN);
 
@@ -1261,11 +1374,25 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC ||
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY);
 
+	/*
+	 * See if tablesync of the current relation has been started with another
+	 * replication slot.
+	 *
+	 * Read previous slot name from the catalog, if exists.
+	 */
+	prev_slotname = (char *) palloc0(NAMEDATALEN);
+	StartTransactionCommand();
+	GetSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+									  MyLogicalRepWorker->relid,
+									  prev_slotname);
+
 	/* Assign the origin tracking record name. */
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
-									   MyLogicalRepWorker->relid,
 									   originname,
-									   sizeof(originname));
+									   sizeof(originname),
+									   true);
+
+	CommitTransactionCommand();
 
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC)
 	{
@@ -1280,10 +1407,48 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		 * breakdown then it wouldn't have succeeded so trying it next time
 		 * seems like a better bet.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, slotname, true);
+		if (strlen(prev_slotname) > 0)
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);
+
+			StartTransactionCommand();
+			/* Replication drop might still exist. Try to drop */
+			replorigin_drop_by_name(originname, true, false);
+
+			/*
+			 * Remove replication slot and origin name from the relation's
+			 * catalog record
+			 */
+			UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+								  MyLogicalRepWorker->relid,
+								  MyLogicalRepWorker->relstate,
+								  MyLogicalRepWorker->relstate_lsn,
+								  NULL,
+								  NULL);
+			CommitTransactionCommand();
+		}
 	}
 	else if (MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY)
 	{
+		/*
+		 * At this point, the table that is currently being synchronized
+		 * should have its replication slot name filled in the catalog. The
+		 * tablesync process was started with another sync worker and
+		 * replication slot. We need to continue using the same replication
+		 * slot in this worker too.
+		 */
+		if (strlen(prev_slotname) == 0)
+		{
+			elog(ERROR, "Replication slot could not be found for relation %u",
+				 MyLogicalRepWorker->relid);
+		}
+
+		/*
+		 * Proceed with the correct replication slot. Use previously created
+		 * replication slot to sync this table.
+		 */
+		slotname = prev_slotname;
+
 		/*
 		 * The COPY phase was previously done, but tablesync then crashed
 		 * before it was able to finish normally.
@@ -1304,6 +1469,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		goto copy_table_done;
 	}
 
+	/* Preparing for table copy operation */
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 	MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
 	MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
@@ -1311,10 +1477,12 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   MyLogicalRepWorker->relstate,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  MyLogicalRepWorker->relstate,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 	CommitTransactionCommand();
 	pgstat_report_stat(true);
 
@@ -1353,6 +1521,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						GetUserNameFromId(GetUserId(), true),
 						RelationGetRelationName(rel))));
 
+
 	/*
 	 * Start a transaction in the remote node in REPEATABLE READ mode.  This
 	 * ensures that both the replication slot we create (see below) and the
@@ -1368,55 +1537,100 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						res->err)));
 	walrcv_clear_result(res);
 
+	originid = replorigin_by_name(originname, true);
+
 	/*
 	 * Create a new permanent logical decoding slot. This slot will be used
 	 * for the catchup phase after COPY is done, so tell it to use the
 	 * snapshot to make the final data consistent.
 	 *
+	 * Replication slot will only be created if either this is the first run
+	 * of the worker or we're not using a previous replication slot.
+	 *
 	 * Prevent cancel/die interrupts while creating slot here because it is
 	 * possible that before the server finishes this command, a concurrent
 	 * drop subscription happens which would complete without removing this
 	 * slot leading to a dangling slot on the server.
+	 *
 	 */
-	HOLD_INTERRUPTS();
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-	RESUME_INTERRUPTS();
-
-	/*
-	 * Setup replication origin tracking. The purpose of doing this before the
-	 * copy is to avoid doing the copy again due to any error in setting up
-	 * origin tracking.
-	 */
-	originid = replorigin_by_name(originname, true);
-	if (!OidIsValid(originid))
+	if (!MyLogicalRepWorker->created_slot)
 	{
+		HOLD_INTERRUPTS();
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						   slotname, false /* permanent */ , false /* two_phase */ ,
+						   CRS_USE_SNAPSHOT, origin_startpos);
+		RESUME_INTERRUPTS();
+
 		/*
-		 * Origin tracking does not exist, so create it now.
-		 *
-		 * Then advance to the LSN got from walrcv_create_slot. This is WAL
-		 * logged for the purpose of recovery. Locks are to prevent the
-		 * replication origin from vanishing while advancing.
+		 * Remember that we created the slot so that we will not try to create
+		 * it again.
 		 */
-		originid = replorigin_create(originname);
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->created_slot = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 
-		LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
-		replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
-						   true /* go backward */ , true /* WAL log */ );
-		UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+		/*
+		 * Setup replication origin tracking. The purpose of doing this before
+		 * the copy is to avoid doing the copy again due to any error in
+		 * setting up origin tracking.
+		 */
 
-		replorigin_session_setup(originid);
-		replorigin_session_origin = originid;
+		if (!OidIsValid(originid))
+		{
+			/*
+			 * Origin tracking does not exist, so create it now.
+			 */
+			originid = replorigin_create(originname);
+		}
+		else
+		{
+			/*
+			 * At this point, there shouldn't be any existing replication
+			 * origin wit the same name.
+			 */
+			ereport(ERROR,
+					(errcode(ERRCODE_DUPLICATE_OBJECT),
+					 errmsg("replication origin \"%s\" already exists",
+							originname)));
+		}
 	}
 	else
 	{
-		ereport(ERROR,
-				(errcode(ERRCODE_DUPLICATE_OBJECT),
-				 errmsg("replication origin \"%s\" already exists",
-						originname)));
+		/*
+		 * Do not create a new replication slot, reuse the existing one
+		 * instead. Use a new snapshot for the replication slot to ensure that
+		 * tablesync and apply proceses are consistent with each other.
+		 */
+		WalRcvStreamOptions options;
+		int			server_version;
+
+		server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+		options.proto.logical.proto_version =
+			server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+			server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+			LOGICALREP_PROTO_VERSION_NUM;
+		options.proto.logical.publication_names = MySubscription->publications;
+
+		HOLD_INTERRUPTS();
+		walrcv_slot_snapshot(LogRepWorkerWalRcvConn, slotname, &options, origin_startpos);
+		RESUME_INTERRUPTS();
 	}
 
+	/*
+	 * Advance to the LSN got from walrcv_create_slot. This is WAL
+	 * logged for the purpose of recovery. Locks are to prevent the
+	 * replication origin from vanishing while advancing.
+	 *
+	 * Then setup replication origin tracking.
+	 */
+	LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+	replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
+					   true /* go backward */ , true /* WAL log */ );
+	UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+
+	replorigin_session_setup(originid);
+	replorigin_session_origin = originid;
+
 	/* Now do the initial data copy */
 	PushActiveSnapshot(GetTransactionSnapshot());
 	copy_table(rel);
@@ -1439,10 +1653,12 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	 * Update the persisted state to indicate the COPY phase is done; make it
 	 * visible to others.
 	 */
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   SUBREL_STATE_FINISHEDCOPY,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  SUBREL_STATE_FINISHEDCOPY,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 
 	CommitTransactionCommand();
 
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 96772e4d73..0b325b8fa0 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -317,6 +317,7 @@ static void stream_open_file(Oid subid, TransactionId xid,
 							 bool first_segment);
 static void stream_write_change(char action, StringInfo s);
 static void stream_close_file(void);
+static void stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
@@ -369,18 +370,26 @@ static inline void reset_apply_error_context_info(void);
  * Form the origin name for the subscription.
  *
  * This is a common function for tablesync and other workers. Tablesync workers
- * must pass a valid relid. Other callers must pass relid = InvalidOid.
+ * must pass is_tablesync true so that origin name includes slot id.
  *
  * Return the name in the supplied buffer.
  */
 void
-ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
-								   char *originname, Size szoriginname)
+ReplicationOriginNameForLogicalRep(Oid suboid, char *originname,
+								   Size szoriginname, bool is_tablesync)
 {
-	if (OidIsValid(relid))
+	if (is_tablesync)
 	{
-		/* Replication origin name for tablesync workers. */
-		snprintf(originname, szoriginname, "pg_%u_%u", suboid, relid);
+		bool		is_null = true;
+
+		/*
+		 * Replication origin name for tablesync workers. First, look into the
+		 * catalog. If originname does not exist, then use the default name.
+		 */
+		GetSubscriptionRelOrigin(suboid, MyLogicalRepWorker->relid,
+								 originname, &is_null);
+		if (is_null)
+			snprintf(originname, szoriginname, "pg_%u_%ld", suboid, MyLogicalRepWorker->rep_slot_id);
 	}
 	else
 	{
@@ -2839,6 +2848,23 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can move to next table while running
+				 * process_syncing_tables_for_sync. Before we were able to
+				 * reuse tablesync workers, that
+				 * process_syncing_tables_for_sync call would exit the worker
+				 * instead of moving to next table. Now that tablesync workers
+				 * can be reused, we need to take care of memory contexts here
+				 * before moving to sync a table.
+				 */
+				if (MyLogicalRepWorker->move_to_next_rel)
+				{
+					MemoryContextResetAndDeleteChildren(ApplyMessageContext);
+					MemoryContextSwitchTo(TopMemoryContext);
+					return;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -2858,6 +2884,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+			if (MyLogicalRepWorker->move_to_next_rel)
+			{
+				endofstream = true;
+			}
 		}
 
 		/* Cleanup the memory. */
@@ -2959,8 +2989,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	/* Pop the error context stack */
 	error_context_stack = errcallback.previous;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * If it's moving to next relation, this is a sync worker. Sync workers
+	 * end the streaming during process_syncing_tables_for_sync. Calling
+	 * endstreaming twice causes "no COPY in progress" errors.
+	 */
+	if (!MyLogicalRepWorker->move_to_next_rel)
+	{
+		/* All done */
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	}
 }
 
 /*
@@ -3502,6 +3540,35 @@ stream_write_change(char action, StringInfo s)
 	BufFileWrite(stream_fd, &s->data[s->cursor], len);
 }
 
+/*
+ * stream_build_options_replication
+ * 		Build logical replication streaming options.
+ *
+ * This function sets streaming options including replication slot name
+ * and origin start position. Workers need these options for logical replication.
+ */
+static void
+stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+	options->proto.logical.streaming = MySubscription->stream;
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -3576,6 +3643,9 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 
 	/* allocate slot name in long-lived context */
 	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+
+	/* Keep the replication slot name used for this sync. */
+	MyLogicalRepWorker->slot_name = *myslotname;
 	pfree(syncslotname);
 }
 
@@ -3613,6 +3683,140 @@ start_apply(XLogRecPtr origin_startpos)
 	PG_END_TRY();
 }
 
+/*
+ * Runs the tablesync worker.
+ * It starts table sync. After successful sync,
+ * builds streaming options and starts streaming.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Set this to false for safety, in case we're already reusing the worker */
+	MyLogicalRepWorker->move_to_next_rel = false;
+
+	start_table_sync(origin_startpos, &slotname);
+
+	/*
+	 * Allocate the origin name in long-lived context for error context
+	 * message.
+	 */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   originname,
+									   originname_size,
+									   true);
+	CommitTransactionCommand();
+
+	apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
+															   originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/* Start normal logical streaming replication. */
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+}
+
+/*
+ * Runs the apply worker.
+ * It sets up replication origin, the streaming options
+ * and then starts streaming.
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
+{
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+
+	slotname = MySubscription->slotname;
+
+	/*
+	 * This shouldn't happen if the subscription is enabled, but guard against
+	 * DDL bugs or manual catalog changes.  (libpqwalreceiver will crash if
+	 * slot is NULL.)
+	 */
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+				 errmsg("subscription has no replication slot set")));
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, originname,
+									   originname_size, false);
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											MySubscription->name, &err);
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+				 errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+	 * We don't really use the output identify_system for anything but it does
+	 * some initializations on the upstream so let's still call it.
+	 */
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	/*
+	 * Allocate the origin name in long-lived context for error context
+	 * message.
+	 */
+	apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
+															   originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+	{
+		/* Start streaming with two_phase enabled */
+		options->proto.logical.twophase = true;
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
+	}
+	else
+	{
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+	}
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+}
+
 /* Logical Replication Apply worker entry point */
 void
 ApplyWorkerMain(Datum main_arg)
@@ -3623,7 +3827,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	/* Attach to slot */
 	logicalrep_worker_attach(worker_slot);
@@ -3714,143 +3917,48 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		/*
-		 * Allocate the origin name in long-lived context for error context
-		 * message.
-		 */
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
-																   originname);
-	}
-	else
-	{
-		/* This is main apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		/*
-		 * Allocate the origin name in long-lived context for error context
-		 * message.
-		 */
-		apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
-																   originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
-	 * the subscription relation state.
+	 * the subscription relation state. Do this outside the loop to avoid
+	 * exceeding MAX_SYSCACHE_CALLBACKS
 	 */
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
-
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
-
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-	options.proto.logical.streaming = MySubscription->stream;
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
-
-	if (!am_tablesync_worker())
+	/*
+	 * The loop where worker does its job. It loops until the worker is not
+	 * reused.
+	 */
+	while (MyLogicalRepWorker->is_first_run ||
+		   MyLogicalRepWorker->move_to_next_rel)
 	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
+		if (am_tablesync_worker())
 		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
+			/*
+			 * This is a tablesync worker. Start syncing tables before
+			 * starting the apply loop.
+			 */
+			run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 		}
 		else
 		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+			/* This is main apply worker */
+			run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 		}
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-						MySubscription->name,
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-						"?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
-
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+		/* Run the main loop. */
+		start_apply(origin_startpos);
 
+		if (MyLogicalRepWorker->move_to_next_rel)
+		{
+			StartTransactionCommand();
+			ereport(LOG,
+					(errmsg("logical replication table synchronization worker for subscription \"%s\" has moved to sync table \"%s\".",
+							MySubscription->name, get_rel_name(MyLogicalRepWorker->relid))));
+			CommitTransactionCommand();
+		}
+	}
 	proc_exit(0);
 }
 
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 7b98714f30..0b13dd34cb 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -102,6 +102,9 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
 
 	/* Only publish data originating from the specified origin */
 	text		suborigin BKI_DEFAULT(LOGICALREP_ORIGIN_ANY);
+
+	/* The last used ID to create a replication slot for tablesync */
+	int64		sublastusedid BKI_DEFAULT(0);
 #endif
 } FormData_pg_subscription;
 
@@ -135,11 +138,14 @@ typedef struct Subscription
 	List	   *publications;	/* List of publication names to subscribe to */
 	char	   *origin;			/* Only publish data originating from the
 								 * specified origin */
+	int64		lastusedid;		/* Last used unique ID to create replication
+								 * slots in tablesync */
 } Subscription;
 
 extern Subscription *GetSubscription(Oid subid, bool missing_ok);
 extern void FreeSubscription(Subscription *sub);
 extern void DisableSubscription(Oid subid);
+extern void UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid);
 
 extern int	CountDBSubscriptions(Oid dbid);
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 8e88de7b2b..9cafb521f9 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -44,6 +44,12 @@ CATALOG(pg_subscription_rel,6102,SubscriptionRelRelationId)
 											 * used for synchronization
 											 * coordination, or NULL if not
 											 * valid */
+	text		srrelslotname BKI_FORCE_NULL;	/* name of the replication
+												 * slot for relation in
+												 * subscription */
+	text		srreloriginname BKI_FORCE_NULL; /* origin name for relation in
+												 * subscription */
+
 #endif
 } FormData_pg_subscription_rel;
 
@@ -81,10 +87,17 @@ typedef struct SubscriptionRelState
 } SubscriptionRelState;
 
 extern void AddSubscriptionRelState(Oid subid, Oid relid, char state,
-									XLogRecPtr sublsn);
+									XLogRecPtr sublsn, char *relslotname, char *reloriginname);
 extern void UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 									   XLogRecPtr sublsn);
+extern void UpdateSubscriptionRel(Oid subid, Oid relid, char state,
+								  XLogRecPtr sublsn, char *relslotname, char *reloriginname);
+extern void UpdateSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *relslotname);
+
 extern char GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn);
+extern void GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname);
+extern void GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull);
+
 extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 
 extern bool HasSubscriptionRelations(Oid subid);
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 65f2c74239..aca6b590f9 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -219,8 +219,9 @@ extern bool InvalidateObsoleteReplicationSlots(XLogSegNo oldestSegno);
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
 extern bool ReplicationSlotName(int index, Name name);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, Size szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, int64 slotid, char *syncslotname, Size szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List *GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 2b7114ff6d..e7f74e0f9f 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -30,6 +30,26 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/*
+	 * Indicates if worker is running for the first time or in reuse
+	 */
+	bool		is_first_run;
+
+	/*
+	 * Indicates if the sync worker created a replication slot or it reuses an
+	 * existing one created by another worker.
+	 */
+	bool		created_slot;
+
+	/*
+	 * Unique identifier for replication slot to be created by tablesnync
+	 * workers, if needed.
+	 */
+	int64		rep_slot_id;
+
+	/* Replication slot name used by the worker. */
+	char	   *slot_name;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
@@ -51,6 +71,12 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/*
+	 * Used to indicate whether sync worker will be reused for another
+	 * relation
+	 */
+	bool		move_to_next_rel;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction, the
@@ -85,15 +111,15 @@ 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);
+									 Oid userid, Oid relid, int64 slotid);
 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);
 
 extern int	logicalrep_sync_worker_count(Oid subid);
 
-extern void ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
-											   char *originname, Size szoriginname);
+extern void ReplicationOriginNameForLogicalRep(Oid suboid, char *originname,
+											   Size szoriginname, bool is_tablesync);
 extern char *LogicalRepSyncTableStart(XLogRecPtr *origin_startpos);
 
 extern bool AllTablesyncsReady(void);
diff --git a/src/test/regress/expected/misc_sanity.out b/src/test/regress/expected/misc_sanity.out
index a57fd142a9..3d34a21421 100644
--- a/src/test/regress/expected/misc_sanity.out
+++ b/src/test/regress/expected/misc_sanity.out
@@ -47,20 +47,22 @@ WHERE c.oid < 16384 AND
       relkind = 'r' AND
       attstorage != 'p'
 ORDER BY 1, 2;
-         relname         |    attname    |   atttypid   
--------------------------+---------------+--------------
- pg_attribute            | attacl        | aclitem[]
- pg_attribute            | attfdwoptions | text[]
- pg_attribute            | attmissingval | anyarray
- pg_attribute            | attoptions    | text[]
- pg_class                | relacl        | aclitem[]
- pg_class                | reloptions    | text[]
- pg_class                | relpartbound  | pg_node_tree
- pg_index                | indexprs      | pg_node_tree
- pg_index                | indpred       | pg_node_tree
- pg_largeobject          | data          | bytea
- pg_largeobject_metadata | lomacl        | aclitem[]
-(11 rows)
+         relname         |     attname     |   atttypid   
+-------------------------+-----------------+--------------
+ pg_attribute            | attacl          | aclitem[]
+ pg_attribute            | attfdwoptions   | text[]
+ pg_attribute            | attmissingval   | anyarray
+ pg_attribute            | attoptions      | text[]
+ pg_class                | relacl          | aclitem[]
+ pg_class                | reloptions      | text[]
+ pg_class                | relpartbound    | pg_node_tree
+ pg_index                | indexprs        | pg_node_tree
+ pg_index                | indpred         | pg_node_tree
+ pg_largeobject          | data            | bytea
+ pg_largeobject_metadata | lomacl          | aclitem[]
+ pg_subscription_rel     | srreloriginname | text
+ pg_subscription_rel     | srrelslotname   | text
+(13 rows)
 
 -- system catalogs without primary keys
 --
-- 
2.25.1

#21Amit Kapila
amit.kapila16@gmail.com
In reply to: Melih Mutlu (#20)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Thu, Dec 15, 2022 at 5:33 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Also here are some numbers with 10 tables loaded with some data :

| 10 MB | 100 MB
----------------------------------------------------------
master | 2868.524 ms | 14281.711 ms
----------------------------------------------------------
patch | 1750.226 ms | 14592.800 ms

The difference between the master and the patch is getting close when the size of tables increase, as expected.

Right, but when the size is 100MB, it seems to be taking a bit more
time. Do we want to evaluate with different sizes to see how it looks?
Other than that all the numbers are good.

--
With Regards,
Amit Kapila.

#22Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Amit Kapila (#21)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Amit,

Amit Kapila <amit.kapila16@gmail.com>, 16 Ara 2022 Cum, 05:46 tarihinde
şunu yazdı:

Right, but when the size is 100MB, it seems to be taking a bit more
time. Do we want to evaluate with different sizes to see how it looks?
Other than that all the numbers are good.

I did a similar testing with again 100MB and also 1GB this time.

| 100 MB | 1 GB
----------------------------------------------------------
master | 14761.425 ms | 160932.982 ms
----------------------------------------------------------
patch | 14398.408 ms | 160593.078 ms

This time, it seems like the patch seems slightly faster than the master.
Not sure if we can say the patch slows things down (or speeds up) if the
size of tables increases.
The difference may be something arbitrary or caused by other factors. What
do you think?

I also wondered what happens when "max_sync_workers_per_subscription" is
set to 1.
Which means tablesync will be done sequentially in both cases but the patch
will use only one worker and one replication slot during the whole
tablesync process.
Here are the numbers for that case:

| 100 MB | 1 GB
----------------------------------------------------------
master | 27751.463 ms | 312424.999 ms
----------------------------------------------------------
patch | 27342.760 ms | 310021.767 ms

Best,
--
Melih Mutlu
Microsoft

#23Amit Kapila
amit.kapila16@gmail.com
In reply to: Melih Mutlu (#22)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Tue, Dec 20, 2022 at 8:14 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Amit Kapila <amit.kapila16@gmail.com>, 16 Ara 2022 Cum, 05:46 tarihinde şunu yazdı:

Right, but when the size is 100MB, it seems to be taking a bit more
time. Do we want to evaluate with different sizes to see how it looks?
Other than that all the numbers are good.

I did a similar testing with again 100MB and also 1GB this time.

| 100 MB | 1 GB
----------------------------------------------------------
master | 14761.425 ms | 160932.982 ms
----------------------------------------------------------
patch | 14398.408 ms | 160593.078 ms

This time, it seems like the patch seems slightly faster than the master.
Not sure if we can say the patch slows things down (or speeds up) if the size of tables increases.
The difference may be something arbitrary or caused by other factors. What do you think?

Yes, I agree with you as I also can't see an obvious reason for any
slowdown with this patch's idea.

--
With Regards,
Amit Kapila.

#24Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Amit Kapila (#23)
2 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi hackers,

Sending an updated version of this patch to get rid of compiler warnings.

I would highly appreciate any feedback.

Thanks,
--
Melih Mutlu
Microsoft

Attachments:

v2-0001-Add-replication-protocol-cmd-to-create-a-snapshot.patchapplication/octet-stream; name=v2-0001-Add-replication-protocol-cmd-to-create-a-snapshot.patchDownload
From 9ffa22049582ac0464306d301b57aba8610376e7 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 13 Oct 2022 17:05:45 +0300
Subject: [PATCH 1/2] Add replication protocol cmd to create a snapshot

Introduced REPLICATION_SLOT_SNAPSHOT to be able to create and use a
snapshot without creating a new replication slot, but by using an
existing slot.

REPLICATION_SLOT_SNAPSHOT simply does what CREATE_REPLICATION_SLOT does
without creating a new replication slot.

REPLICATION_SLOT_SNAPSHOT command imports the snapshot into the current
transaction and returns consistent_point. The changes earlier than the
consistent_point will be applied by importing the snapshot. All changes
later than the consistent_point will be available to be consumed from
the replication slot.

This is useful for reusing replication slots in logical replication.
Otherwise, tablesync workers cannot start from a consistent point to copy a relation and then apply changes by consuming from replication slot.
---
 doc/src/sgml/protocol.sgml                    | 32 ++++++
 .../libpqwalreceiver/libpqwalreceiver.c       | 69 ++++++++++++-
 src/backend/replication/logical/logical.c     | 39 +++++++-
 .../replication/logical/logicalfuncs.c        |  1 +
 src/backend/replication/repl_gram.y           | 18 +++-
 src/backend/replication/repl_scanner.l        |  2 +
 src/backend/replication/slotfuncs.c           |  1 +
 src/backend/replication/walsender.c           | 97 ++++++++++++++++++-
 src/include/nodes/replnodes.h                 | 11 +++
 src/include/replication/logical.h             |  1 +
 src/include/replication/walreceiver.h         | 13 +++
 src/tools/pgindent/typedefs.list              |  1 +
 12 files changed, 281 insertions(+), 4 deletions(-)

diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 03312e07e2..f4f29079ac 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2613,6 +2613,38 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
      </listitem>
     </varlistentry>
 
+    <varlistentry id="protocol-replication-replication-slot-snapshot">
+     <term><literal>REPLICATION_SLOT_SNAPSHOT</literal> <replaceable class="parameter">slot_name</replaceable> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
+      <indexterm><primary>REPLICATION_SLOT_SNAPSHOT</primary></indexterm>
+     </term>
+     <listitem>
+      <para>
+       Creates a snapshot including all the changes from the replication slot until
+       the point at which the replication slot becomes consistent. Then the snapshot
+       is used in the currenct transaction. This command is currently only supported
+       for logical replication.
+       slots.
+      </para>
+
+      <para>
+       In response to this command, the server will return a one-row result set,
+       containing the following field:
+       <variablelist>
+        <varlistentry>
+         <term><literal>consistent_point</literal> (<type>text</type>)</term>
+         <listitem>
+          <para>
+           The WAL location at which the slot became consistent.  This is the
+           earliest location from which streaming can start on this replication
+           slot.
+          </para>
+         </listitem>
+        </varlistentry>
+       </variablelist>
+      </para>
+     </listitem>
+    </varlistentry>
+
     <varlistentry id="protocol-replication-base-backup" xreflabel="BASE_BACKUP">
      <term><literal>BASE_BACKUP</literal> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
       <indexterm><primary>BASE_BACKUP</primary></indexterm>
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index 7f697b0f29..e367c5f757 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -80,6 +80,8 @@ static WalRcvExecResult *libpqrcv_exec(WalReceiverConn *conn,
 									   const int nRetTypes,
 									   const Oid *retTypes);
 static void libpqrcv_disconnect(WalReceiverConn *conn);
+static void libpqrcv_slot_snapshot(WalReceiverConn *conn, char *slotname,
+								   const WalRcvStreamOptions *options, XLogRecPtr *lsn);
 
 static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_connect = libpqrcv_connect,
@@ -96,7 +98,8 @@ static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_create_slot = libpqrcv_create_slot,
 	.walrcv_get_backend_pid = libpqrcv_get_backend_pid,
 	.walrcv_exec = libpqrcv_exec,
-	.walrcv_disconnect = libpqrcv_disconnect
+	.walrcv_disconnect = libpqrcv_disconnect,
+	.walrcv_slot_snapshot = libpqrcv_slot_snapshot
 };
 
 /* Prototypes for private functions */
@@ -968,6 +971,70 @@ libpqrcv_create_slot(WalReceiverConn *conn, const char *slotname,
 	return snapshot;
 }
 
+/*
+ * TODO
+ */
+static void
+libpqrcv_slot_snapshot(WalReceiverConn *conn,
+					   char *slotname,
+					   const WalRcvStreamOptions *options,
+					   XLogRecPtr *lsn)
+{
+	StringInfoData cmd;
+	PGresult   *res;
+	char	   *pubnames_str;
+	List	   *pubnames;
+	char	   *pubnames_literal;
+
+	initStringInfo(&cmd);
+
+	/* Build the command. */
+	appendStringInfo(&cmd, "REPLICATION_SLOT_SNAPSHOT \"%s\"", slotname);
+	appendStringInfoString(&cmd, " (");
+	appendStringInfo(&cmd, " proto_version '%u'",
+					 options->proto.logical.proto_version);
+
+	/* Add publication names. */
+	pubnames = options->proto.logical.publication_names;
+	pubnames_str = stringlist_to_identifierstr(conn->streamConn, pubnames);
+	if (!pubnames_str)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	pubnames_literal = PQescapeLiteral(conn->streamConn, pubnames_str,
+									   strlen(pubnames_str));
+	if (!pubnames_literal)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	appendStringInfo(&cmd, ", publication_names %s", pubnames_literal);
+	PQfreemem(pubnames_literal);
+	pfree(pubnames_str);
+
+	appendStringInfoString(&cmd, " )");
+
+	/* Execute the command. */
+	res = libpqrcv_PQexec(conn->streamConn, cmd.data);
+	pfree(cmd.data);
+
+	if (PQresultStatus(res) != PGRES_TUPLES_OK)
+	{
+		PQclear(res);
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg("Could not create a snapshot by replication slot \"%s\": %s",
+						slotname, pchomp(PQerrorMessage(conn->streamConn)))));
+	}
+
+	if (lsn)
+		*lsn = DatumGetLSN(DirectFunctionCall1Coll(pg_lsn_in, InvalidOid,
+												   CStringGetDatum(PQgetvalue(res, 0, 0))));
+
+	PQclear(res);
+}
+
 /*
  * Return PID of remote backend process.
  */
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 625a7f4273..16c0a27ba5 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -461,6 +461,10 @@ CreateInitDecodingContext(const char *plugin,
  * fast_forward
  *		bypass the generation of logical changes.
  *
+ * need_full_snapshot
+ * 		if true, create a snapshot able to read all tables,
+ * 		otherwise do not create any snapshot.
+ *
  * xl_routine
  *		XLogReaderRoutine used by underlying xlogreader
  *
@@ -479,6 +483,7 @@ LogicalDecodingContext *
 CreateDecodingContext(XLogRecPtr start_lsn,
 					  List *output_plugin_options,
 					  bool fast_forward,
+					  bool need_full_snapshot,
 					  XLogReaderRoutine *xl_routine,
 					  LogicalOutputPluginWriterPrepareWrite prepare_write,
 					  LogicalOutputPluginWriterWrite do_write,
@@ -487,6 +492,7 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 	LogicalDecodingContext *ctx;
 	ReplicationSlot *slot;
 	MemoryContext old_context;
+	TransactionId xmin_horizon = InvalidTransactionId;
 
 	/* shorter lines... */
 	slot = MyReplicationSlot;
@@ -533,8 +539,39 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 		start_lsn = slot->data.confirmed_flush;
 	}
 
+
+	/*
+	 * We need to determine a safe xmin horizon to start decoding from if we
+	 * want to create a snapshot too. Otherwise we would end up with a
+	 * snapshot that cannot be imported since xmin value from the snapshot may
+	 * be less than the oldest safe xmin. To avoid this call
+	 * GetOldestSafeDecodingTransactionId() to return a safe xmin value, which
+	 * can be used while exporting/importing the snapshot.
+	 *
+	 * So we have to acquire the ProcArrayLock to prevent computation of new
+	 * xmin horizons by other backends, get the safe decoding xid, and inform
+	 * the slot machinery about the new limit. Once that's done the
+	 * ProcArrayLock can be released as the slot machinery now is protecting
+	 * against vacuum.
+	 */
+	if (need_full_snapshot)
+	{
+		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+		SpinLockAcquire(&slot->mutex);
+		slot->effective_catalog_xmin = xmin_horizon;
+		slot->data.catalog_xmin = xmin_horizon;
+		slot->effective_xmin = xmin_horizon;
+		SpinLockRelease(&slot->mutex);
+
+		xmin_horizon = GetOldestSafeDecodingTransactionId(!need_full_snapshot);
+		ReplicationSlotsComputeRequiredXmin(true);
+
+		LWLockRelease(ProcArrayLock);
+	}
+
 	ctx = StartupDecodingContext(output_plugin_options,
-								 start_lsn, InvalidTransactionId, false,
+								 start_lsn, xmin_horizon, need_full_snapshot,
 								 fast_forward, xl_routine, prepare_write,
 								 do_write, update_progress);
 
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index 5c23178570..e4b80f2764 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -208,6 +208,7 @@ pg_logical_slot_get_changes_guts(FunctionCallInfo fcinfo, bool confirm, bool bin
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									options,
 									false,
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/repl_gram.y b/src/backend/replication/repl_gram.y
index 7ba33d6672..f70b2e087a 100644
--- a/src/backend/replication/repl_gram.y
+++ b/src/backend/replication/repl_gram.y
@@ -65,6 +65,7 @@ Node *replication_parse_result;
 %token K_CREATE_REPLICATION_SLOT
 %token K_DROP_REPLICATION_SLOT
 %token K_TIMELINE_HISTORY
+%token K_REPLICATION_SLOT_SNAPSHOT
 %token K_WAIT
 %token K_TIMELINE
 %token K_PHYSICAL
@@ -80,7 +81,7 @@ Node *replication_parse_result;
 %type <node>	command
 %type <node>	base_backup start_replication start_logical_replication
 				create_replication_slot drop_replication_slot identify_system
-				read_replication_slot timeline_history show
+				read_replication_slot timeline_history show replication_slot_snapshot
 %type <list>	generic_option_list
 %type <defelt>	generic_option
 %type <uintval>	opt_timeline
@@ -114,6 +115,7 @@ command:
 			| read_replication_slot
 			| timeline_history
 			| show
+			| replication_slot_snapshot
 			;
 
 /*
@@ -307,6 +309,19 @@ timeline_history:
 				}
 			;
 
+/*
+ * REPLICATION_SLOT_SNAPSHOT %s options
+ */
+replication_slot_snapshot:
+			K_REPLICATION_SLOT_SNAPSHOT var_name plugin_options
+				{
+					ReplicationSlotSnapshotCmd *n = makeNode(ReplicationSlotSnapshotCmd);
+					n->slotname = $2;
+					n->options = $3;
+					$$ = (Node *) n;
+				}
+			;
+
 opt_physical:
 			K_PHYSICAL
 			| /* EMPTY */
@@ -400,6 +415,7 @@ ident_or_keyword:
 			| K_CREATE_REPLICATION_SLOT	{ $$ = "create_replication_slot"; }
 			| K_DROP_REPLICATION_SLOT		{ $$ = "drop_replication_slot"; }
 			| K_TIMELINE_HISTORY			{ $$ = "timeline_history"; }
+			| K_REPLICATION_SLOT_SNAPSHOT	{ $$ = "replication_slot_snapshot"; }
 			| K_WAIT						{ $$ = "wait"; }
 			| K_TIMELINE					{ $$ = "timeline"; }
 			| K_PHYSICAL					{ $$ = "physical"; }
diff --git a/src/backend/replication/repl_scanner.l b/src/backend/replication/repl_scanner.l
index 72ef12225e..e33030da61 100644
--- a/src/backend/replication/repl_scanner.l
+++ b/src/backend/replication/repl_scanner.l
@@ -126,6 +126,7 @@ START_REPLICATION	{ return K_START_REPLICATION; }
 CREATE_REPLICATION_SLOT		{ return K_CREATE_REPLICATION_SLOT; }
 DROP_REPLICATION_SLOT		{ return K_DROP_REPLICATION_SLOT; }
 TIMELINE_HISTORY	{ return K_TIMELINE_HISTORY; }
+REPLICATION_SLOT_SNAPSHOT	{ return K_REPLICATION_SLOT_SNAPSHOT; }
 PHYSICAL			{ return K_PHYSICAL; }
 RESERVE_WAL			{ return K_RESERVE_WAL; }
 LOGICAL				{ return K_LOGICAL; }
@@ -303,6 +304,7 @@ replication_scanner_is_replication_command(void)
 		case K_DROP_REPLICATION_SLOT:
 		case K_READ_REPLICATION_SLOT:
 		case K_TIMELINE_HISTORY:
+		case K_REPLICATION_SLOT_SNAPSHOT:
 		case K_SHOW:
 			/* Yes; push back the first token so we can parse later. */
 			repl_pushed_back_token = first_token;
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 79d7f1372d..13d099fe06 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -478,6 +478,7 @@ pg_logical_replication_slot_advance(XLogRecPtr moveto)
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									NIL,
 									true,	/* fast_forward */
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index c11bb3716f..f9a22c8b16 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -238,6 +238,7 @@ static void CreateReplicationSlot(CreateReplicationSlotCmd *cmd);
 static void DropReplicationSlot(DropReplicationSlotCmd *cmd);
 static void StartReplication(StartReplicationCmd *cmd);
 static void StartLogicalReplication(StartReplicationCmd *cmd);
+static void ReplicationSlotSnapshot(ReplicationSlotSnapshotCmd *cmd);
 static void ProcessStandbyMessage(void);
 static void ProcessStandbyReplyMessage(void);
 static void ProcessStandbyHSFeedbackMessage(void);
@@ -1280,7 +1281,7 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	 * are reported early.
 	 */
 	logical_decoding_ctx =
-		CreateDecodingContext(cmd->startpoint, cmd->options, false,
+		CreateDecodingContext(cmd->startpoint, cmd->options, false, false,
 							  XL_ROUTINE(.page_read = logical_read_xlog_page,
 										 .segment_open = WalSndSegmentOpen,
 										 .segment_close = wal_segment_close),
@@ -1332,6 +1333,91 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	EndCommand(&qc, DestRemote, false);
 }
 
+/*
+ * Create a snapshot from an existing replication slot.
+ */
+static void
+ReplicationSlotSnapshot(ReplicationSlotSnapshotCmd *cmd)
+{
+	Snapshot	snap;
+	LogicalDecodingContext *ctx;
+	char		xloc[MAXFNAMELEN];
+	DestReceiver *dest;
+	TupOutputState *tstate;
+	TupleDesc	tupdesc;
+	Datum		values[1];
+	bool		nulls[1] = {0};
+
+	Assert(!MyReplicationSlot);
+
+	if (!IsTransactionBlock())
+		ereport(ERROR,
+				(errmsg("%s must be called inside a transaction",
+						"REPLICATION_SLOT_SNAPSHOT ...")));
+
+	if (XactIsoLevel != XACT_REPEATABLE_READ)
+		ereport(ERROR,
+				(errmsg("%s must be called in REPEATABLE READ isolation mode transaction",
+						"REPLICATION_SLOT_SNAPSHOT ...")));
+
+	if (FirstSnapshotSet)
+		ereport(ERROR,
+				(errmsg("%s must be called before any query",
+						"REPLICATION_SLOT_SNAPSHOT ...")));
+
+	if (IsSubTransaction())
+		ereport(ERROR,
+				(errmsg("%s must not be called in a subtransaction",
+						"REPLICATION_SLOT_SNAPSHOT ...")));
+
+	ReplicationSlotAcquire(cmd->slotname, false);
+
+	ctx = CreateDecodingContext(MyReplicationSlot->data.restart_lsn,
+								cmd->options,
+								false,
+								true,
+								XL_ROUTINE(.page_read = logical_read_xlog_page,
+										   .segment_open = WalSndSegmentOpen,
+										   .segment_close = wal_segment_close),
+								WalSndPrepareWrite, WalSndWriteData,
+								WalSndUpdateProgress);
+
+	/*
+	 * Signal that we don't need the timeout mechanism. We're just creating
+	 * the replication slot and don't yet accept feedback messages or send
+	 * keepalives. As we possibly need to wait for further WAL the walsender
+	 * would otherwise possibly be killed too soon.
+	 */
+	last_reply_timestamp = 0;
+
+	/* build initial snapshot, might take a while */
+	DecodingContextFindStartpoint(ctx);
+
+	snap = SnapBuildInitialSnapshot(ctx->snapshot_builder);
+	RestoreTransactionSnapshot(snap, MyProc);
+
+	/* Don't need the decoding context anymore */
+	FreeDecodingContext(ctx);
+
+	/* Create a tuple to send consisten WAL location */
+	snprintf(xloc, sizeof(xloc), "%X/%X",
+			 LSN_FORMAT_ARGS(MyReplicationSlot->data.confirmed_flush));
+
+	dest = CreateDestReceiver(DestRemoteSimple);
+	tupdesc = CreateTemplateTupleDesc(1);
+	TupleDescInitBuiltinEntry(tupdesc, (AttrNumber) 1, "consistent_point",
+							  TEXTOID, -1, 0);
+	tstate = begin_tup_output_tupdesc(dest, tupdesc, &TTSOpsVirtual);
+
+	/* consistent wal location */
+	values[0] = CStringGetTextDatum(xloc);
+
+	do_tup_output(tstate, values, nulls);
+	end_tup_output(tstate);
+
+	ReplicationSlotRelease();
+}
+
 /*
  * LogicalDecodingContext 'prepare_write' callback.
  *
@@ -1860,6 +1946,15 @@ exec_replication_command(const char *cmd_string)
 			}
 			break;
 
+		case T_ReplicationSlotSnapshotCmd:
+			{
+				cmdtag = "REPLICATION_SLOT_SNAPSHOT";
+				set_ps_display(cmdtag);
+				ReplicationSlotSnapshot((ReplicationSlotSnapshotCmd *) cmd_node);
+				EndReplicationCommand(cmdtag);
+				break;
+			}
+
 		default:
 			elog(ERROR, "unrecognized replication command node tag: %u",
 				 cmd_node->type);
diff --git a/src/include/nodes/replnodes.h b/src/include/nodes/replnodes.h
index 8ae9c517ff..94aa7d2253 100644
--- a/src/include/nodes/replnodes.h
+++ b/src/include/nodes/replnodes.h
@@ -108,4 +108,15 @@ typedef struct TimeLineHistoryCmd
 	TimeLineID	timeline;
 } TimeLineHistoryCmd;
 
+/* ----------------------
+ *		REPLICATION_SLOT_SNAPSHOT command
+ * ----------------------
+ */
+typedef struct ReplicationSlotSnapshotCmd
+{
+	NodeTag		type;
+	char	   *slotname;
+	List	   *options;
+} ReplicationSlotSnapshotCmd;
+
 #endif							/* REPLNODES_H */
diff --git a/src/include/replication/logical.h b/src/include/replication/logical.h
index 4ad019e25a..3875b32f9d 100644
--- a/src/include/replication/logical.h
+++ b/src/include/replication/logical.h
@@ -125,6 +125,7 @@ extern LogicalDecodingContext *CreateInitDecodingContext(const char *plugin,
 extern LogicalDecodingContext *CreateDecodingContext(XLogRecPtr start_lsn,
 													 List *output_plugin_options,
 													 bool fast_forward,
+													 bool need_full_snapshot,
 													 XLogReaderRoutine *xl_routine,
 													 LogicalOutputPluginWriterPrepareWrite prepare_write,
 													 LogicalOutputPluginWriterWrite do_write,
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 9339f29303..274a162b48 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -384,6 +384,16 @@ typedef WalRcvExecResult *(*walrcv_exec_fn) (WalReceiverConn *conn,
  */
 typedef void (*walrcv_disconnect_fn) (WalReceiverConn *conn);
 
+/*
+ * walrcv_slot_snapshot_fn
+ *
+ * Create a snapshot by an existing replication slot
+ */
+typedef void (*walrcv_slot_snapshot_fn) (WalReceiverConn *conn,
+										 char *slotname,
+										 const WalRcvStreamOptions *options,
+										 XLogRecPtr *lsn);
+
 typedef struct WalReceiverFunctionsType
 {
 	walrcv_connect_fn walrcv_connect;
@@ -401,6 +411,7 @@ typedef struct WalReceiverFunctionsType
 	walrcv_get_backend_pid_fn walrcv_get_backend_pid;
 	walrcv_exec_fn walrcv_exec;
 	walrcv_disconnect_fn walrcv_disconnect;
+	walrcv_slot_snapshot_fn walrcv_slot_snapshot;
 } WalReceiverFunctionsType;
 
 extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
@@ -435,6 +446,8 @@ extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
 	WalReceiverFunctions->walrcv_exec(conn, exec, nRetTypes, retTypes)
 #define walrcv_disconnect(conn) \
 	WalReceiverFunctions->walrcv_disconnect(conn)
+#define walrcv_slot_snapshot(conn, slotname, options, lsn) \
+	WalReceiverFunctions->walrcv_slot_snapshot(conn, slotname, options, lsn)
 
 static inline void
 walrcv_clear_result(WalRcvExecResult *walres)
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 60c71d05fe..6b90a4a693 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2315,6 +2315,7 @@ ReplicationSlotCtlData
 ReplicationSlotOnDisk
 ReplicationSlotPersistency
 ReplicationSlotPersistentData
+ReplicationSlotSnapshotCmd
 ReplicationState
 ReplicationStateCtl
 ReplicationStateOnDisk
-- 
2.25.1

v6-0002-Reuse-Logical-Replication-Background-worker.patchapplication/octet-stream; name=v6-0002-Reuse-Logical-Replication-Background-worker.patchDownload
From 901624f1d48af5a2766af9ba85fd28ccffd0daef Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH 2/2] Reuse Logical Replication Background worker

This commit allows tablesync workers to move to another table that needs synchronization,
when they're done with the current table in tablesync phase of Logical Replication.

Before this commit, tablesync workers were capable of syncing only one
relation. A new worker, replication slot and origin were needed for each
relation in the subscription.

Now, tablesync workers are not only limited with one relation and can move to another relation and reuse existing
replication slots and origins

This reduces the overhead of launching/killing a new background worker for each relation.
By reusing tablesync workers, replication slots and origins created for tablesync can be reused as well.
Removing the burden of creating/dropping replication slot/origin improves tablesync speed significantly especially for empty or small tables.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/catalogs.sgml                  |  29 ++
 src/backend/catalog/pg_subscription.c       | 284 ++++++++++++-
 src/backend/commands/subscriptioncmds.c     | 238 +++++++----
 src/backend/replication/logical/launcher.c  |   9 +-
 src/backend/replication/logical/tablesync.c | 434 +++++++++++++++-----
 src/backend/replication/logical/worker.c    | 370 +++++++++++------
 src/include/catalog/pg_subscription.h       |   6 +
 src/include/catalog/pg_subscription_rel.h   |  15 +-
 src/include/replication/slot.h              |   3 +-
 src/include/replication/worker_internal.h   |  32 +-
 src/test/regress/expected/misc_sanity.out   |  30 +-
 11 files changed, 1095 insertions(+), 355 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 9316b811ac..7d55db6354 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -7997,6 +7997,17 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        origin.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>sublastusedid</structfield> <type>int8</type>
+      </para>
+      <para>
+       The last used ID for tablesync workers. This ID is used to
+       create replication slots. The last used ID needs to be stored
+       to make logical replication can safely proceed after any interruption.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
@@ -8081,6 +8092,24 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        otherwise null
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srrelslotname</structfield> <type>text</type>
+      </para>
+      <para>
+       Replication slot name that is used for synchronization of relation
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srreloriginname</structfield> <type>text</type>
+      </para>
+      <para>
+       Origin name that is used for tracking synchronization of relation
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index a56ae311c3..98d627ff49 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -114,6 +114,14 @@ GetSubscription(Oid subid, bool missing_ok)
 	Assert(!isnull);
 	sub->origin = TextDatumGetCString(datum);
 
+	/* Get last used id */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
+							tup,
+							Anum_pg_subscription_sublastusedid,
+							&isnull);
+	Assert(!isnull);
+	sub->lastusedid = DatumGetInt64(datum);
+
 	ReleaseSysCache(tup);
 
 	return sub;
@@ -205,6 +213,44 @@ DisableSubscription(Oid subid)
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update the last used replication slot ID for the given subscription.
+ */
+void
+UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid)
+{
+	Relation	rel;
+	bool		nulls[Natts_pg_subscription];
+	bool		replaces[Natts_pg_subscription];
+	Datum		values[Natts_pg_subscription];
+	HeapTuple	tup;
+
+	/* Look up the subscription in the catalog */
+	rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+	tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	/* Form a new tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	replaces[Anum_pg_subscription_sublastusedid - 1] = true;
+	values[Anum_pg_subscription_sublastusedid- 1] = Int64GetDatum(lastusedid);
+
+	/* Update the catalog */
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+	heap_freetuple(tup);
+
+	table_close(rel, NoLock);
+}
+
 /*
  * Convert text array to list of strings.
  *
@@ -234,7 +280,7 @@ textarray_to_stringlist(ArrayType *textarray)
  */
 void
 AddSubscriptionRelState(Oid subid, Oid relid, char state,
-						XLogRecPtr sublsn)
+						XLogRecPtr sublsn, char *relslotname, char *reloriginname)
 {
 	Relation	rel;
 	HeapTuple	tup;
@@ -263,6 +309,14 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
 	else
 		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] = CStringGetTextDatum(relslotname);
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] = CStringGetTextDatum(reloriginname);
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -275,6 +329,58 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Internal function to modify columns for relation state update
+ */
+static void
+UpdateSubscriptionRelState_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char state,
+									XLogRecPtr sublsn)
+{
+	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
+	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+
+	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (sublsn != InvalidXLogRecPtr)
+		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
+	else
+		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication slot update
+ */
+static void
+UpdateSubscriptionRelReplicationSlot_internal(Datum *values,
+											bool *nulls,
+											bool *replaces,
+											char *relslotname)
+{
+	replaces[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] = CStringGetTextDatum(relslotname);
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication origin update
+ */
+static void
+UpdateSubscriptionRelOrigin_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char *reloriginname)
+{
+	replaces[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] = CStringGetTextDatum(reloriginname);
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+}
+
 /*
  * Update the state of a subscription table.
  */
@@ -305,14 +411,48 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	memset(nulls, false, sizeof(nulls));
 	memset(replaces, false, sizeof(replaces));
 
-	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
-	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
 
-	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
-	if (sublsn != InvalidXLogRecPtr)
-		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
-	else
-		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update the replication slot name of a subscription table.
+ */
+void
+UpdateSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *relslotname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
 
 	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
 							replaces);
@@ -324,6 +464,134 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update replication slot name, origin name and state of
+ * a subscription table in one transaction.
+ */
+void
+UpdateSubscriptionRel(Oid subid,
+					  Oid relid,
+					  char state,
+					  XLogRecPtr sublsn,
+					  char *relslotname,
+					  char *reloriginname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
+	UpdateSubscriptionRelOrigin_internal(values, nulls, replaces, reloriginname);
+
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Get origin name of subscription table.
+ *
+ * Returns null if the subscription table does not have a origin.
+ */
+void
+GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*originname;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srreloriginname, isnull);
+	if (!*isnull)
+	{
+		originname = TextDatumGetCString(d);
+		memcpy(reloriginname, originname, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
+/*
+ * Get replication slot name of subscription table.
+ *
+ * Returns null if the subscription table does not have a replication slot.
+ */
+void
+GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*relrepslot;
+	bool		isnull;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srrelslotname, &isnull);
+	if (!isnull)
+	{
+		relrepslot = TextDatumGetCString(d);
+		memcpy(slotname, relrepslot, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
 /*
  * Get state of subscription table.
  *
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index b9c5df796f..964b698e61 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -648,6 +648,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 		publicationListToArray(publications);
 	values[Anum_pg_subscription_suborigin - 1] =
 		CStringGetTextDatum(opts.origin);
+	values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(1);
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -657,7 +658,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 
 	recordDependencyOnOwner(SubscriptionRelationId, subid, owner);
 
-	ReplicationOriginNameForLogicalRep(subid, InvalidOid, originname, sizeof(originname));
+	ReplicationOriginNameForLogicalRep(subid, originname, sizeof(originname), false);
 	replorigin_create(originname);
 
 	/*
@@ -708,7 +709,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 										 rv->schemaname, rv->relname);
 
 				AddSubscriptionRelState(subid, relid, table_state,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 			}
 
 			/*
@@ -798,6 +799,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	} SubRemoveRels;
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -875,7 +878,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				AddSubscriptionRelState(sub->oid, relid,
 										copy_data ? SUBREL_STATE_INIT : SUBREL_STATE_READY,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 				ereport(DEBUG1,
 						(errmsg_internal("table \"%s.%s\" added to subscription \"%s\"",
 										 rv->schemaname, rv->relname, sub->name)));
@@ -899,6 +902,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				char		state;
 				XLogRecPtr	statelsn;
+				char		slotname[NAMEDATALEN] = {0};
 
 				/*
 				 * Lock pg_subscription_rel with AccessExclusiveLock to
@@ -925,7 +929,29 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/*
+				 * Find the logical replication sync worker if exists store
+				 * the slot number for dropping associated replication slots
+				 * later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot_name);
+				}
+				else
+				{
+					/*
+					 * Sync of this relation might be failed in an earlier
+					 * attempt, but the replication slot might still exist.
+					 */
+					GetSubscriptionRelReplicationSlot(sub->oid, relid, slotname);
+					if (strlen(slotname) > 0)
+						sub_remove_slots = lappend(sub_remove_slots, slotname);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
@@ -945,8 +971,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 					 * origin and by this time the origin might be already
 					 * removed. For these reasons, passing missing_ok = true.
 					 */
-					ReplicationOriginNameForLogicalRep(sub->oid, relid, originname,
-													   sizeof(originname));
+					ReplicationOriginNameForLogicalRep(sub->oid, originname,
+													   sizeof(originname), true);
 					replorigin_drop_by_name(originname, true, false);
 				}
 
@@ -959,31 +985,24 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
-		 * the database operations we won't be able to rollback dropped slots.
+		 * Drop the replication slots associated with tablesync workers for
+		 * removed tables. This has to be at the end because otherwise if
+		 * there is an error while doing the database operations we won't be
+		 * able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			memcpy(syncslotname, lfirst(lc), sizeof(NAMEDATALEN));
+
+			/*
+			 * There is no certainty, maybe the slot does not exist yet. Also,
+			 * if we fail after removing some of the slots, next time, it will
+			 * again try to drop already dropped slots and fail. For these
+			 * reasons, we allow missing_ok = true for the drop.
+			 */
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1319,8 +1338,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
 					char		originname[NAMEDATALEN];
 					XLogRecPtr	remote_lsn;
 
-					ReplicationOriginNameForLogicalRep(subid, InvalidOid,
-													   originname, sizeof(originname));
+					ReplicationOriginNameForLogicalRep(subid, originname,
+													   sizeof(originname), false);
 					originid = replorigin_by_name(originname, false);
 					remote_lsn = replorigin_get_progress(originid, false);
 
@@ -1380,6 +1399,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	char	   *subname;
 	char	   *conninfo;
 	char	   *slotname;
+	int64		lastusedid;
 	List	   *subworkers;
 	ListCell   *lc;
 	char		originname[NAMEDATALEN];
@@ -1451,6 +1471,14 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	else
 		slotname = NULL;
 
+	/* Get the last used identifier by the subscription */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID, tup,
+							Anum_pg_subscription_sublastusedid, &isnull);
+	if (!isnull)
+		lastusedid = DatumGetInt64(datum);
+	else
+		lastusedid = 0;
+
 	/*
 	 * Since dropping a replication slot is not transactional, the replication
 	 * slot stays dropped even if the transaction rolls back.  So we cannot
@@ -1500,37 +1528,29 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	}
 	list_free(subworkers);
 
+	rstates = GetSubscriptionRelations(subid, true);
+
 	/*
-	 * Cleanup of tablesync replication origins.
-	 *
-	 * Any READY-state relations would already have dealt with clean-ups.
+	 * Cleanup of tablesync replication origins associated with the
+	 * subscription, if exists. Try to drop origins by creating all origin
+	 * names created for this subscription.
 	 *
 	 * Note that the state can't change because we have already stopped both
 	 * the apply and tablesync workers and they can't restart because of
 	 * exclusive lock on the subscription.
+	 *
+	 * XXX: This can be handled better instead of looping through all possible
 	 */
-	rstates = GetSubscriptionRelations(subid, true);
-	foreach(lc, rstates)
+	for (int64 i = 1; i <= lastusedid; i++)
 	{
-		SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-		Oid			relid = rstate->relid;
-
-		/* Only cleanup resources of tablesync workers */
-		if (!OidIsValid(relid))
-			continue;
+		char		originname_to_drop[NAMEDATALEN] = {0};
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * It is possible that the origin is not yet created for tablesync
-		 * worker so passing missing_ok = true. This can happen for the states
-		 * before SUBREL_STATE_FINISHEDCOPY.
-		 */
-		ReplicationOriginNameForLogicalRep(subid, relid, originname,
-										   sizeof(originname));
-		replorigin_drop_by_name(originname, true, false);
+		snprintf(originname_to_drop, sizeof(originname_to_drop), "pg_%u_%lld", subid, (long long) i);
+		/* missin_ok = true, since the origin might be already dropped. */
+		replorigin_drop_by_name(originname_to_drop, true, false);
 	}
 
+
 	/* Clean up dependencies */
 	deleteSharedDependencyRecordsFor(SubscriptionRelationId, subid, 0);
 
@@ -1538,7 +1558,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	RemoveSubscriptionRel(subid, InvalidOid);
 
 	/* Remove the origin tracking if exists. */
-	ReplicationOriginNameForLogicalRep(subid, InvalidOid, originname, sizeof(originname));
+	ReplicationOriginNameForLogicalRep(subid, originname, sizeof(originname), false);
 	replorigin_drop_by_name(originname, true, false);
 
 	/*
@@ -1582,39 +1602,17 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
+		List	   *slots = NULL;
 
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
 
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char	   *syncslotname = (char *) lfirst(lc);
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1637,6 +1635,71 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * Get the replication slot names associated with the subscription.
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok)
+{
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+					 " FROM pg_replication_slots"
+					 " WHERE slot_name LIKE 'pg_%i_sync_%%';",
+					 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+					errmsg("not tuple returned."));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();
+	\
+		return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -1989,6 +2052,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -1999,18 +2063,20 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
-			char		syncslotname[NAMEDATALEN] = {0};
-
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
-				 syncslotname);
+				 worker->slot_name);
 		}
 	}
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index a69e371c05..0d33d3592b 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -264,7 +264,7 @@ logicalrep_workers_find(Oid subid, bool only_running)
  */
 void
 logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
-						 Oid relid)
+						 Oid relid, int64 slotid)
 {
 	BackgroundWorker bgw;
 	BackgroundWorkerHandle *bgw_handle;
@@ -371,7 +371,11 @@ retry:
 	/* Prepare the worker slot. */
 	worker->launch_time = now;
 	worker->in_use = true;
+	worker->is_first_run = true;
 	worker->generation++;
+	worker->created_slot = false;
+	worker->rep_slot_id = slotid;
+	worker->slot_name = (char *) palloc(NAMEDATALEN);
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -379,6 +383,7 @@ retry:
 	worker->relid = relid;
 	worker->relstate = SUBREL_STATE_UNKNOWN;
 	worker->relstate_lsn = InvalidXLogRecPtr;
+	worker->move_to_next_rel = false;
 	worker->stream_fileset = NULL;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
@@ -869,7 +874,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, 0);
 				}
 			}
 
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 2fdfeb5b4c..bb242e0c95 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -126,12 +126,8 @@ static bool FetchTableStates(bool *started_tx);
 
 static StringInfo copybuf = NULL;
 
-/*
- * Exit routine for synchronization worker.
- */
 static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+clean_sync_worker(void)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -143,18 +139,28 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
-	/* And flush all writes. */
-	XLogFlush(GetXLogWriteRecPtr());
-
-	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
-	CommitTransactionCommand();
+	/*
+	 * Disconnect from publisher. Otherwise reused sync workers causes
+	 * exceeding max_wal_senders
+	 */
+	walrcv_disconnect(LogRepWorkerWalRcvConn);
+	LogRepWorkerWalRcvConn = NULL;
 
 	/* Find the main apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
+}
+
+/*
+ * Exit routine for synchronization worker.
+ */
+static void
+pg_attribute_noreturn()
+finish_sync_worker(void)
+{
+	clean_sync_worker();
+
+	/* And flush all writes. */
+	XLogFlush(GetXLogWriteRecPtr());
 
 	/* Stop gracefully */
 	proc_exit(0);
@@ -284,6 +290,10 @@ invalidate_syncing_table_states(Datum arg, int cacheid, uint32 hashvalue)
 static void
 process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 {
+	List	   *rstates;
+	SubscriptionRelState *rstate;
+	ListCell   *lc;
+
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_CATCHUP &&
@@ -292,6 +302,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		TimeLineID	tli;
 		char		syncslotname[NAMEDATALEN] = {0};
 		char		originname[NAMEDATALEN] = {0};
+		bool		is_streaming_ended = false;
 
 		MyLogicalRepWorker->relstate = SUBREL_STATE_SYNCDONE;
 		MyLogicalRepWorker->relstate_lsn = current_lsn;
@@ -308,40 +319,29 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 								   MyLogicalRepWorker->relid,
 								   MyLogicalRepWorker->relstate,
 								   MyLogicalRepWorker->relstate_lsn);
+		CommitTransactionCommand();
 
 		/*
-		 * End streaming so that LogRepWorkerWalRcvConn can be used to drop
-		 * the slot.
-		 */
-		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
-
-		/*
-		 * Cleanup the tablesync slot.
+		 * Cleanup the tablesync slot. If the slot name used by this worker is
+		 * different from the default slot name for the worker, this means the
+		 * current table had started to being synchronized by another worker
+		 * and replication slot. And this worker is reusing a replication slot
+		 * from a previous attempt. We do not need that replication slot
+		 * anymore.
 		 *
 		 * This has to be done after updating the state because otherwise if
 		 * there is an error while doing the database operations we won't be
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
+										MyLogicalRepWorker->rep_slot_id,
 										syncslotname,
 										sizeof(syncslotname));
 
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
-		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
-
-		CommitTransactionCommand();
-		pgstat_report_stat(false);
-
-		/*
-		 * Start a new transaction to clean up the tablesync origin tracking.
-		 * This transaction will be ended within the finish_sync_worker().
-		 * Now, even, if we fail to remove this here, the apply worker will
-		 * ensure to clean it up afterward.
+		 * We are safe to drop the replication trackin origin after this
+		 * point. Now, even, if we fail to remove this here, the apply worker
+		 * will ensure to clean it up afterward.
 		 *
 		 * We need to do this after the table state is set to SYNCDONE.
 		 * Otherwise, if an error occurs while performing the database
@@ -350,34 +350,125 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 * have been cleared before restart. So, the restarted worker will use
 		 * invalid replication progress state resulting in replay of
 		 * transactions that have already been applied.
+		 *
+		 * Firstly reset the origin session to remove the ownership of the
+		 * slot. This is needed to allow the origin to be dropped or reused
+		 * later.
+		 */
+		replorigin_session_reset();
+		replorigin_session_origin = InvalidRepOriginId;
+		replorigin_session_origin_lsn = InvalidXLogRecPtr;
+		replorigin_session_origin_timestamp = 0;
+
+		StartTransactionCommand();
+		if (MyLogicalRepWorker->slot_name && strcmp(syncslotname, MyLogicalRepWorker->slot_name) != 0)
+		{
+			/*
+			 * End streaming so that LogRepWorkerWalRcvConn can be used to
+			 * drop the slot.
+			 */
+			walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+			is_streaming_ended = true;
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
+
+			ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+											   originname,
+											   sizeof(originname),
+											   true);
+
+			/* Drop replication origin */
+			replorigin_drop_by_name(originname, true, false);
+		}
+
+		/*
+		 * We are safe to remove persisted replication slot and origin data,
+		 * since it's already in SYNCDONE state. They will not be needed
+		 * anymore.
 		 */
+		UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+							  MyLogicalRepWorker->relid,
+							  MyLogicalRepWorker->relstate,
+							  MyLogicalRepWorker->relstate_lsn,
+							  NULL,
+							  NULL);
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid))));
+
+		CommitTransactionCommand();
+		pgstat_report_stat(false);
+
 		StartTransactionCommand();
 
+		/*
+		 * This should return the default origin name for the worker. Even if
+		 * the worker used a different origin for this table, it should be
+		 * dropped and removed from the catalog so far.
+		 */
 		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-										   MyLogicalRepWorker->relid,
 										   originname,
-										   sizeof(originname));
+										   sizeof(originname),
+										   true);
 
 		/*
-		 * Resetting the origin session removes the ownership of the slot.
-		 * This is needed to allow the origin to be dropped.
+		 * Check if any table whose relation state is still INIT. If a table
+		 * in INIT state is found, the worker will not be finished, it will be
+		 * reused instead.
 		 */
-		replorigin_session_reset();
-		replorigin_session_origin = InvalidRepOriginId;
-		replorigin_session_origin_lsn = InvalidXLogRecPtr;
-		replorigin_session_origin_timestamp = 0;
+		rstates = GetSubscriptionRelations(MySubscription->oid, true);
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * There is a chance that the user is concurrently performing refresh
-		 * for the subscription where we remove the table state and its origin
-		 * or the apply worker would have removed this origin. So passing
-		 * missing_ok = true.
-		 */
-		replorigin_drop_by_name(originname, true, false);
+		foreach(lc, rstates)
+		{
+			rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
+			memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
+
+			/*
+			 * Pick the table for the next run if there is not another worker
+			 * already picked that table.
+			 */
+			LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+			if (rstate->state != SUBREL_STATE_SYNCDONE &&
+				!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+			{
+				/* Update worker state for the next table */
+				MyLogicalRepWorker->is_first_run = false;
+				MyLogicalRepWorker->relid = rstate->relid;
+				MyLogicalRepWorker->relstate = rstate->state;
+				MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+				MyLogicalRepWorker->move_to_next_rel = true;
+				LWLockRelease(LogicalRepWorkerLock);
+				break;
+			}
+			LWLockRelease(LogicalRepWorkerLock);
+		}
+
+		/* Cleanup before next run or ending the worker. */
+		if (!MyLogicalRepWorker->move_to_next_rel)
+		{
+			/*
+			 * It is important to give an error if we are unable to drop the
+			 * slot, otherwise, it won't be dropped till the corresponding
+			 * subscription is dropped. So passing missing_ok = false.
+			 */
+			if (MyLogicalRepWorker->created_slot)
+			{
+				/* End streaming if it's not already ended. */
+				if (!is_streaming_ended)
+					walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+				ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
+			}
 
-		finish_sync_worker();
+			/* Drop replication origin before exiting. */
+			replorigin_drop_by_name(originname, true, false);
+
+			finish_sync_worker();
+		}
+		else
+		{
+			clean_sync_worker();
+		}
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -485,6 +576,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 			if (current_lsn >= rstate->lsn)
 			{
 				char		originname[NAMEDATALEN];
+				bool		is_origin_null = true;
 
 				rstate->state = SUBREL_STATE_READY;
 				rstate->lsn = current_lsn;
@@ -505,18 +597,27 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				 * error while dropping we won't restart it to drop the
 				 * origin. So passing missing_ok = true.
 				 */
-				ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-												   rstate->relid,
-												   originname,
-												   sizeof(originname));
-				replorigin_drop_by_name(originname, true, false);
+				GetSubscriptionRelOrigin(MyLogicalRepWorker->subid,
+										 rstate->relid, originname,
+										 &is_origin_null);
+
+				if (!is_origin_null)
+				{
+					replorigin_drop_by_name(originname, true, false);
+				}
 
 				/*
 				 * Update the state to READY only after the origin cleanup.
 				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-										   rstate->relid, rstate->state,
-										   rstate->lsn);
+				UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+									  rstate->relid,
+									  rstate->state,
+									  rstate->lsn,
+									  NULL,
+									  NULL);
+
+				CommitTransactionCommand();
+				started_tx = false;
 			}
 		}
 		else
@@ -605,11 +706,21 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						TimestampDifferenceExceeds(hentry->last_start_time, now,
 												   wal_retrieve_retry_interval))
 					{
+						if (IsTransactionState())
+							CommitTransactionCommand();
+						StartTransactionCommand();
+						started_tx = true;
+
+						MySubscription->lastusedid++;
+						UpdateSubscriptionLastSlotId(MyLogicalRepWorker->subid,
+													 MySubscription->lastusedid);
+
 						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
 												 MySubscription->oid,
 												 MySubscription->name,
 												 MyLogicalRepWorker->userid,
-												 rstate->relid);
+												 rstate->relid,
+												 MySubscription->lastusedid);
 						hentry->last_start_time = now;
 					}
 				}
@@ -802,6 +913,7 @@ fetch_remote_table_info(char *nspname, char *relname,
 		TupleTableSlot *tslot;
 		Oid			attrsRow[] = {INT2VECTOROID};
 		StringInfoData pub_names;
+
 		initStringInfo(&pub_names);
 		foreach(lc, MySubscription->publications)
 		{
@@ -1174,8 +1286,8 @@ copy_table(Relation rel)
  * The name must not exceed NAMEDATALEN - 1 because of remote node constraints
  * on slot name length. We append system_identifier to avoid slot_name
  * collision with subscriptions in other clusters. With the current scheme
- * pg_%u_sync_%u_UINT64_FORMAT (3 + 10 + 6 + 10 + 20 + '\0'), the maximum
- * length of slot_name will be 50.
+ * pg_%u_sync_%lu_UINT64_FORMAT (3 + 10 + 6 + 20 + 20 + '\0'), the maximum
+ * length of slot_name will be 45.
  *
  * The returned slot name is stored in the supplied buffer (syncslotname) with
  * the given size.
@@ -1186,11 +1298,11 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
+ReplicationSlotNameForTablesync(Oid suboid, int64 slotid,
 								char *syncslotname, Size szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%lld_" UINT64_FORMAT, suboid,
+			(long long) slotid, GetSystemIdentifier());
 }
 
 /*
@@ -1213,6 +1325,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	WalRcvExecResult *res;
 	char		originname[NAMEDATALEN];
 	RepOriginId originid;
+	char	   *prev_slotname;
 
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
@@ -1241,7 +1354,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
+									MyLogicalRepWorker->rep_slot_id,
 									slotname,
 									NAMEDATALEN);
 
@@ -1261,11 +1374,25 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC ||
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY);
 
+	/*
+	 * See if tablesync of the current relation has been started with another
+	 * replication slot.
+	 *
+	 * Read previous slot name from the catalog, if exists.
+	 */
+	prev_slotname = (char *) palloc0(NAMEDATALEN);
+	StartTransactionCommand();
+	GetSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+									  MyLogicalRepWorker->relid,
+									  prev_slotname);
+
 	/* Assign the origin tracking record name. */
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
-									   MyLogicalRepWorker->relid,
 									   originname,
-									   sizeof(originname));
+									   sizeof(originname),
+									   true);
+
+	CommitTransactionCommand();
 
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC)
 	{
@@ -1280,10 +1407,48 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		 * breakdown then it wouldn't have succeeded so trying it next time
 		 * seems like a better bet.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, slotname, true);
+		if (strlen(prev_slotname) > 0)
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);
+
+			StartTransactionCommand();
+			/* Replication drop might still exist. Try to drop */
+			replorigin_drop_by_name(originname, true, false);
+
+			/*
+			 * Remove replication slot and origin name from the relation's
+			 * catalog record
+			 */
+			UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+								  MyLogicalRepWorker->relid,
+								  MyLogicalRepWorker->relstate,
+								  MyLogicalRepWorker->relstate_lsn,
+								  NULL,
+								  NULL);
+			CommitTransactionCommand();
+		}
 	}
 	else if (MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY)
 	{
+		/*
+		 * At this point, the table that is currently being synchronized
+		 * should have its replication slot name filled in the catalog. The
+		 * tablesync process was started with another sync worker and
+		 * replication slot. We need to continue using the same replication
+		 * slot in this worker too.
+		 */
+		if (strlen(prev_slotname) == 0)
+		{
+			elog(ERROR, "Replication slot could not be found for relation %u",
+				 MyLogicalRepWorker->relid);
+		}
+
+		/*
+		 * Proceed with the correct replication slot. Use previously created
+		 * replication slot to sync this table.
+		 */
+		slotname = prev_slotname;
+
 		/*
 		 * The COPY phase was previously done, but tablesync then crashed
 		 * before it was able to finish normally.
@@ -1304,6 +1469,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		goto copy_table_done;
 	}
 
+	/* Preparing for table copy operation */
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 	MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
 	MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
@@ -1311,10 +1477,12 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   MyLogicalRepWorker->relstate,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  MyLogicalRepWorker->relstate,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 	CommitTransactionCommand();
 	pgstat_report_stat(true);
 
@@ -1353,6 +1521,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						GetUserNameFromId(GetUserId(), true),
 						RelationGetRelationName(rel))));
 
+
 	/*
 	 * Start a transaction in the remote node in REPEATABLE READ mode.  This
 	 * ensures that both the replication slot we create (see below) and the
@@ -1368,55 +1537,100 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						res->err)));
 	walrcv_clear_result(res);
 
+	originid = replorigin_by_name(originname, true);
+
 	/*
 	 * Create a new permanent logical decoding slot. This slot will be used
 	 * for the catchup phase after COPY is done, so tell it to use the
 	 * snapshot to make the final data consistent.
 	 *
+	 * Replication slot will only be created if either this is the first run
+	 * of the worker or we're not using a previous replication slot.
+	 *
 	 * Prevent cancel/die interrupts while creating slot here because it is
 	 * possible that before the server finishes this command, a concurrent
 	 * drop subscription happens which would complete without removing this
 	 * slot leading to a dangling slot on the server.
+	 *
 	 */
-	HOLD_INTERRUPTS();
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-	RESUME_INTERRUPTS();
-
-	/*
-	 * Setup replication origin tracking. The purpose of doing this before the
-	 * copy is to avoid doing the copy again due to any error in setting up
-	 * origin tracking.
-	 */
-	originid = replorigin_by_name(originname, true);
-	if (!OidIsValid(originid))
+	if (!MyLogicalRepWorker->created_slot)
 	{
+		HOLD_INTERRUPTS();
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						   slotname, false /* permanent */ , false /* two_phase */ ,
+						   CRS_USE_SNAPSHOT, origin_startpos);
+		RESUME_INTERRUPTS();
+
 		/*
-		 * Origin tracking does not exist, so create it now.
-		 *
-		 * Then advance to the LSN got from walrcv_create_slot. This is WAL
-		 * logged for the purpose of recovery. Locks are to prevent the
-		 * replication origin from vanishing while advancing.
+		 * Remember that we created the slot so that we will not try to create
+		 * it again.
 		 */
-		originid = replorigin_create(originname);
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->created_slot = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 
-		LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
-		replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
-						   true /* go backward */ , true /* WAL log */ );
-		UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+		/*
+		 * Setup replication origin tracking. The purpose of doing this before
+		 * the copy is to avoid doing the copy again due to any error in
+		 * setting up origin tracking.
+		 */
 
-		replorigin_session_setup(originid);
-		replorigin_session_origin = originid;
+		if (!OidIsValid(originid))
+		{
+			/*
+			 * Origin tracking does not exist, so create it now.
+			 */
+			originid = replorigin_create(originname);
+		}
+		else
+		{
+			/*
+			 * At this point, there shouldn't be any existing replication
+			 * origin wit the same name.
+			 */
+			ereport(ERROR,
+					(errcode(ERRCODE_DUPLICATE_OBJECT),
+					 errmsg("replication origin \"%s\" already exists",
+							originname)));
+		}
 	}
 	else
 	{
-		ereport(ERROR,
-				(errcode(ERRCODE_DUPLICATE_OBJECT),
-				 errmsg("replication origin \"%s\" already exists",
-						originname)));
+		/*
+		 * Do not create a new replication slot, reuse the existing one
+		 * instead. Use a new snapshot for the replication slot to ensure that
+		 * tablesync and apply proceses are consistent with each other.
+		 */
+		WalRcvStreamOptions options;
+		int			server_version;
+
+		server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+		options.proto.logical.proto_version =
+			server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+			server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+			LOGICALREP_PROTO_VERSION_NUM;
+		options.proto.logical.publication_names = MySubscription->publications;
+
+		HOLD_INTERRUPTS();
+		walrcv_slot_snapshot(LogRepWorkerWalRcvConn, slotname, &options, origin_startpos);
+		RESUME_INTERRUPTS();
 	}
 
+	/*
+	 * Advance to the LSN got from walrcv_create_slot. This is WAL
+	 * logged for the purpose of recovery. Locks are to prevent the
+	 * replication origin from vanishing while advancing.
+	 *
+	 * Then setup replication origin tracking.
+	 */
+	LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+	replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
+					   true /* go backward */ , true /* WAL log */ );
+	UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+
+	replorigin_session_setup(originid);
+	replorigin_session_origin = originid;
+
 	/* Now do the initial data copy */
 	PushActiveSnapshot(GetTransactionSnapshot());
 	copy_table(rel);
@@ -1439,10 +1653,12 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	 * Update the persisted state to indicate the COPY phase is done; make it
 	 * visible to others.
 	 */
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   SUBREL_STATE_FINISHEDCOPY,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  SUBREL_STATE_FINISHEDCOPY,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 
 	CommitTransactionCommand();
 
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 446f84fa97..b10660384a 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -317,6 +317,7 @@ static void stream_open_file(Oid subid, TransactionId xid,
 							 bool first_segment);
 static void stream_write_change(char action, StringInfo s);
 static void stream_close_file(void);
+static void stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
@@ -369,18 +370,26 @@ static inline void reset_apply_error_context_info(void);
  * Form the origin name for the subscription.
  *
  * This is a common function for tablesync and other workers. Tablesync workers
- * must pass a valid relid. Other callers must pass relid = InvalidOid.
+ * must pass is_tablesync true so that origin name includes slot id.
  *
  * Return the name in the supplied buffer.
  */
 void
-ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
-								   char *originname, Size szoriginname)
+ReplicationOriginNameForLogicalRep(Oid suboid, char *originname,
+								   Size szoriginname, bool is_tablesync)
 {
-	if (OidIsValid(relid))
+	if (is_tablesync)
 	{
-		/* Replication origin name for tablesync workers. */
-		snprintf(originname, szoriginname, "pg_%u_%u", suboid, relid);
+		bool		is_null = true;
+
+		/*
+		 * Replication origin name for tablesync workers. First, look into the
+		 * catalog. If originname does not exist, then use the default name.
+		 */
+		GetSubscriptionRelOrigin(suboid, MyLogicalRepWorker->relid,
+								 originname, &is_null);
+		if (is_null)
+			snprintf(originname, szoriginname, "pg_%u_%lld", suboid, (long long) MyLogicalRepWorker->rep_slot_id);
 	}
 	else
 	{
@@ -2839,6 +2848,23 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can move to next table while running
+				 * process_syncing_tables_for_sync. Before we were able to
+				 * reuse tablesync workers, that
+				 * process_syncing_tables_for_sync call would exit the worker
+				 * instead of moving to next table. Now that tablesync workers
+				 * can be reused, we need to take care of memory contexts here
+				 * before moving to sync a table.
+				 */
+				if (MyLogicalRepWorker->move_to_next_rel)
+				{
+					MemoryContextResetAndDeleteChildren(ApplyMessageContext);
+					MemoryContextSwitchTo(TopMemoryContext);
+					return;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -2858,6 +2884,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+			if (MyLogicalRepWorker->move_to_next_rel)
+			{
+				endofstream = true;
+			}
 		}
 
 		/* Cleanup the memory. */
@@ -2959,8 +2989,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	/* Pop the error context stack */
 	error_context_stack = errcallback.previous;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * If it's moving to next relation, this is a sync worker. Sync workers
+	 * end the streaming during process_syncing_tables_for_sync. Calling
+	 * endstreaming twice causes "no COPY in progress" errors.
+	 */
+	if (!MyLogicalRepWorker->move_to_next_rel)
+	{
+		/* All done */
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	}
 }
 
 /*
@@ -3502,6 +3540,35 @@ stream_write_change(char action, StringInfo s)
 	BufFileWrite(stream_fd, &s->data[s->cursor], len);
 }
 
+/*
+ * stream_build_options_replication
+ * 		Build logical replication streaming options.
+ *
+ * This function sets streaming options including replication slot name
+ * and origin start position. Workers need these options for logical replication.
+ */
+static void
+stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+	options->proto.logical.streaming = MySubscription->stream;
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -3576,6 +3643,9 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 
 	/* allocate slot name in long-lived context */
 	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+
+	/* Keep the replication slot name used for this sync. */
+	MyLogicalRepWorker->slot_name = *myslotname;
 	pfree(syncslotname);
 }
 
@@ -3613,6 +3683,140 @@ start_apply(XLogRecPtr origin_startpos)
 	PG_END_TRY();
 }
 
+/*
+ * Runs the tablesync worker.
+ * It starts table sync. After successful sync,
+ * builds streaming options and starts streaming.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Set this to false for safety, in case we're already reusing the worker */
+	MyLogicalRepWorker->move_to_next_rel = false;
+
+	start_table_sync(origin_startpos, &slotname);
+
+	/*
+	 * Allocate the origin name in long-lived context for error context
+	 * message.
+	 */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   originname,
+									   originname_size,
+									   true);
+	CommitTransactionCommand();
+
+	apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
+															   originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/* Start normal logical streaming replication. */
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+}
+
+/*
+ * Runs the apply worker.
+ * It sets up replication origin, the streaming options
+ * and then starts streaming.
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
+{
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+
+	slotname = MySubscription->slotname;
+
+	/*
+	 * This shouldn't happen if the subscription is enabled, but guard against
+	 * DDL bugs or manual catalog changes.  (libpqwalreceiver will crash if
+	 * slot is NULL.)
+	 */
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+				 errmsg("subscription has no replication slot set")));
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, originname,
+									   originname_size, false);
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											MySubscription->name, &err);
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+				 errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+	 * We don't really use the output identify_system for anything but it does
+	 * some initializations on the upstream so let's still call it.
+	 */
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	/*
+	 * Allocate the origin name in long-lived context for error context
+	 * message.
+	 */
+	apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
+															   originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+	{
+		/* Start streaming with two_phase enabled */
+		options->proto.logical.twophase = true;
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
+	}
+	else
+	{
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+	}
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+}
+
 /* Logical Replication Apply worker entry point */
 void
 ApplyWorkerMain(Datum main_arg)
@@ -3623,7 +3827,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	/* Attach to slot */
 	logicalrep_worker_attach(worker_slot);
@@ -3714,143 +3917,48 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		/*
-		 * Allocate the origin name in long-lived context for error context
-		 * message.
-		 */
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
-																   originname);
-	}
-	else
-	{
-		/* This is main apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		/*
-		 * Allocate the origin name in long-lived context for error context
-		 * message.
-		 */
-		apply_error_callback_arg.origin_name = MemoryContextStrdup(ApplyContext,
-																   originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
-	 * the subscription relation state.
+	 * the subscription relation state. Do this outside the loop to avoid
+	 * exceeding MAX_SYSCACHE_CALLBACKS
 	 */
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
-
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
-
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-	options.proto.logical.streaming = MySubscription->stream;
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
-
-	if (!am_tablesync_worker())
+	/*
+	 * The loop where worker does its job. It loops until the worker is not
+	 * reused.
+	 */
+	while (MyLogicalRepWorker->is_first_run ||
+		   MyLogicalRepWorker->move_to_next_rel)
 	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
+		if (am_tablesync_worker())
 		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
+			/*
+			 * This is a tablesync worker. Start syncing tables before
+			 * starting the apply loop.
+			 */
+			run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 		}
 		else
 		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+			/* This is main apply worker */
+			run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 		}
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-						MySubscription->name,
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-						"?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
-
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+		/* Run the main loop. */
+		start_apply(origin_startpos);
 
+		if (MyLogicalRepWorker->move_to_next_rel)
+		{
+			StartTransactionCommand();
+			ereport(LOG,
+					(errmsg("logical replication table synchronization worker for subscription \"%s\" has moved to sync table \"%s\".",
+							MySubscription->name, get_rel_name(MyLogicalRepWorker->relid))));
+			CommitTransactionCommand();
+		}
+	}
 	proc_exit(0);
 }
 
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index eb4eeb448b..326be6cd5b 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -102,6 +102,9 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
 
 	/* Only publish data originating from the specified origin */
 	text		suborigin BKI_DEFAULT(LOGICALREP_ORIGIN_ANY);
+
+	/* The last used ID to create a replication slot for tablesync */
+	int64		sublastusedid BKI_DEFAULT(0);
 #endif
 } FormData_pg_subscription;
 
@@ -135,11 +138,14 @@ typedef struct Subscription
 	List	   *publications;	/* List of publication names to subscribe to */
 	char	   *origin;			/* Only publish data originating from the
 								 * specified origin */
+	int64		lastusedid;		/* Last used unique ID to create replication
+								 * slots in tablesync */
 } Subscription;
 
 extern Subscription *GetSubscription(Oid subid, bool missing_ok);
 extern void FreeSubscription(Subscription *sub);
 extern void DisableSubscription(Oid subid);
+extern void UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid);
 
 extern int	CountDBSubscriptions(Oid dbid);
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 60a2bcca23..a35d04cccd 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -44,6 +44,12 @@ CATALOG(pg_subscription_rel,6102,SubscriptionRelRelationId)
 											 * used for synchronization
 											 * coordination, or NULL if not
 											 * valid */
+	text		srrelslotname BKI_FORCE_NULL;	/* name of the replication
+												 * slot for relation in
+												 * subscription */
+	text		srreloriginname BKI_FORCE_NULL; /* origin name for relation in
+												 * subscription */
+
 #endif
 } FormData_pg_subscription_rel;
 
@@ -81,10 +87,17 @@ typedef struct SubscriptionRelState
 } SubscriptionRelState;
 
 extern void AddSubscriptionRelState(Oid subid, Oid relid, char state,
-									XLogRecPtr sublsn);
+									XLogRecPtr sublsn, char *relslotname, char *reloriginname);
 extern void UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 									   XLogRecPtr sublsn);
+extern void UpdateSubscriptionRel(Oid subid, Oid relid, char state,
+								  XLogRecPtr sublsn, char *relslotname, char *reloriginname);
+extern void UpdateSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *relslotname);
+
 extern char GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn);
+extern void GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname);
+extern void GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull);
+
 extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 
 extern bool HasSubscriptionRelations(Oid subid);
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 8872c80cdf..3547daaaec 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -219,8 +219,9 @@ extern bool InvalidateObsoleteReplicationSlots(XLogSegNo oldestSegno);
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
 extern bool ReplicationSlotName(int index, Name name);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, Size szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, int64 slotid, char *syncslotname, Size szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List *GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 2a3ec5c2d8..563cc9637a 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -30,6 +30,26 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/*
+	 * Indicates if worker is running for the first time or in reuse
+	 */
+	bool		is_first_run;
+
+	/*
+	 * Indicates if the sync worker created a replication slot or it reuses an
+	 * existing one created by another worker.
+	 */
+	bool		created_slot;
+
+	/*
+	 * Unique identifier for replication slot to be created by tablesnync
+	 * workers, if needed.
+	 */
+	int64		rep_slot_id;
+
+	/* Replication slot name used by the worker. */
+	char	   *slot_name;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
@@ -51,6 +71,12 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/*
+	 * Used to indicate whether sync worker will be reused for another
+	 * relation
+	 */
+	bool		move_to_next_rel;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction, the
@@ -85,15 +111,15 @@ 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);
+									 Oid userid, Oid relid, int64 slotid);
 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);
 
 extern int	logicalrep_sync_worker_count(Oid subid);
 
-extern void ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
-											   char *originname, Size szoriginname);
+extern void ReplicationOriginNameForLogicalRep(Oid suboid, char *originname,
+											   Size szoriginname, bool is_tablesync);
 extern char *LogicalRepSyncTableStart(XLogRecPtr *origin_startpos);
 
 extern bool AllTablesyncsReady(void);
diff --git a/src/test/regress/expected/misc_sanity.out b/src/test/regress/expected/misc_sanity.out
index a57fd142a9..3d34a21421 100644
--- a/src/test/regress/expected/misc_sanity.out
+++ b/src/test/regress/expected/misc_sanity.out
@@ -47,20 +47,22 @@ WHERE c.oid < 16384 AND
       relkind = 'r' AND
       attstorage != 'p'
 ORDER BY 1, 2;
-         relname         |    attname    |   atttypid   
--------------------------+---------------+--------------
- pg_attribute            | attacl        | aclitem[]
- pg_attribute            | attfdwoptions | text[]
- pg_attribute            | attmissingval | anyarray
- pg_attribute            | attoptions    | text[]
- pg_class                | relacl        | aclitem[]
- pg_class                | reloptions    | text[]
- pg_class                | relpartbound  | pg_node_tree
- pg_index                | indexprs      | pg_node_tree
- pg_index                | indpred       | pg_node_tree
- pg_largeobject          | data          | bytea
- pg_largeobject_metadata | lomacl        | aclitem[]
-(11 rows)
+         relname         |     attname     |   atttypid   
+-------------------------+-----------------+--------------
+ pg_attribute            | attacl          | aclitem[]
+ pg_attribute            | attfdwoptions   | text[]
+ pg_attribute            | attmissingval   | anyarray
+ pg_attribute            | attoptions      | text[]
+ pg_class                | relacl          | aclitem[]
+ pg_class                | reloptions      | text[]
+ pg_class                | relpartbound    | pg_node_tree
+ pg_index                | indexprs        | pg_node_tree
+ pg_index                | indpred         | pg_node_tree
+ pg_largeobject          | data            | bytea
+ pg_largeobject_metadata | lomacl          | aclitem[]
+ pg_subscription_rel     | srreloriginname | text
+ pg_subscription_rel     | srrelslotname   | text
+(13 rows)
 
 -- system catalogs without primary keys
 --
-- 
2.25.1

#25Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Melih Mutlu (#24)
2 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi hackers,

Rebased the patch to resolve conflicts.

Best,
--
Melih Mutlu
Microsoft

Attachments:

v3-0001-Add-replication-protocol-cmd-to-create-a-snapshot.patchapplication/octet-stream; name=v3-0001-Add-replication-protocol-cmd-to-create-a-snapshot.patchDownload
From 0392f3e3f5c371e0b11c853a419f48b5e402660b Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 13 Oct 2022 17:05:45 +0300
Subject: [PATCH 1/2] Add replication protocol cmd to create a snapshot

Introduced REPLICATION_SLOT_SNAPSHOT to be able to create and use a
snapshot without creating a new replication slot, but by using an
existing slot.

REPLICATION_SLOT_SNAPSHOT simply does what CREATE_REPLICATION_SLOT does
without creating a new replication slot.

REPLICATION_SLOT_SNAPSHOT command imports the snapshot into the current
transaction and returns consistent_point. The changes earlier than the
consistent_point will be applied by importing the snapshot. All changes
later than the consistent_point will be available to be consumed from
the replication slot.

This is useful for reusing replication slots in logical replication.
Otherwise, tablesync workers cannot start from a consistent point to copy a relation and then apply changes by consuming from replication slot.
---
 doc/src/sgml/protocol.sgml                    | 32 ++++++
 .../libpqwalreceiver/libpqwalreceiver.c       | 69 ++++++++++++-
 src/backend/replication/logical/logical.c     | 39 +++++++-
 .../replication/logical/logicalfuncs.c        |  1 +
 src/backend/replication/repl_gram.y           | 18 +++-
 src/backend/replication/repl_scanner.l        |  2 +
 src/backend/replication/slotfuncs.c           |  1 +
 src/backend/replication/walsender.c           | 97 ++++++++++++++++++-
 src/include/nodes/replnodes.h                 | 11 +++
 src/include/replication/logical.h             |  1 +
 src/include/replication/walreceiver.h         | 13 +++
 src/tools/pgindent/typedefs.list              |  1 +
 12 files changed, 281 insertions(+), 4 deletions(-)

diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 93fc7167d4..93a3867996 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2613,6 +2613,38 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
      </listitem>
     </varlistentry>
 
+    <varlistentry id="protocol-replication-replication-slot-snapshot">
+     <term><literal>REPLICATION_SLOT_SNAPSHOT</literal> <replaceable class="parameter">slot_name</replaceable> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
+      <indexterm><primary>REPLICATION_SLOT_SNAPSHOT</primary></indexterm>
+     </term>
+     <listitem>
+      <para>
+       Creates a snapshot including all the changes from the replication slot until
+       the point at which the replication slot becomes consistent. Then the snapshot
+       is used in the currenct transaction. This command is currently only supported
+       for logical replication.
+       slots.
+      </para>
+
+      <para>
+       In response to this command, the server will return a one-row result set,
+       containing the following field:
+       <variablelist>
+        <varlistentry>
+         <term><literal>consistent_point</literal> (<type>text</type>)</term>
+         <listitem>
+          <para>
+           The WAL location at which the slot became consistent.  This is the
+           earliest location from which streaming can start on this replication
+           slot.
+          </para>
+         </listitem>
+        </varlistentry>
+       </variablelist>
+      </para>
+     </listitem>
+    </varlistentry>
+
     <varlistentry id="protocol-replication-base-backup" xreflabel="BASE_BACKUP">
      <term><literal>BASE_BACKUP</literal> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
       <indexterm><primary>BASE_BACKUP</primary></indexterm>
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index c40c6220db..9213fd2b1b 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -80,6 +80,8 @@ static WalRcvExecResult *libpqrcv_exec(WalReceiverConn *conn,
 									   const int nRetTypes,
 									   const Oid *retTypes);
 static void libpqrcv_disconnect(WalReceiverConn *conn);
+static void libpqrcv_slot_snapshot(WalReceiverConn *conn, char *slotname,
+								   const WalRcvStreamOptions *options, XLogRecPtr *lsn);
 
 static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_connect = libpqrcv_connect,
@@ -96,7 +98,8 @@ static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_create_slot = libpqrcv_create_slot,
 	.walrcv_get_backend_pid = libpqrcv_get_backend_pid,
 	.walrcv_exec = libpqrcv_exec,
-	.walrcv_disconnect = libpqrcv_disconnect
+	.walrcv_disconnect = libpqrcv_disconnect,
+	.walrcv_slot_snapshot = libpqrcv_slot_snapshot
 };
 
 /* Prototypes for private functions */
@@ -968,6 +971,70 @@ libpqrcv_create_slot(WalReceiverConn *conn, const char *slotname,
 	return snapshot;
 }
 
+/*
+ * TODO
+ */
+static void
+libpqrcv_slot_snapshot(WalReceiverConn *conn,
+					   char *slotname,
+					   const WalRcvStreamOptions *options,
+					   XLogRecPtr *lsn)
+{
+	StringInfoData cmd;
+	PGresult   *res;
+	char	   *pubnames_str;
+	List	   *pubnames;
+	char	   *pubnames_literal;
+
+	initStringInfo(&cmd);
+
+	/* Build the command. */
+	appendStringInfo(&cmd, "REPLICATION_SLOT_SNAPSHOT \"%s\"", slotname);
+	appendStringInfoString(&cmd, " (");
+	appendStringInfo(&cmd, " proto_version '%u'",
+					 options->proto.logical.proto_version);
+
+	/* Add publication names. */
+	pubnames = options->proto.logical.publication_names;
+	pubnames_str = stringlist_to_identifierstr(conn->streamConn, pubnames);
+	if (!pubnames_str)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	pubnames_literal = PQescapeLiteral(conn->streamConn, pubnames_str,
+									   strlen(pubnames_str));
+	if (!pubnames_literal)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	appendStringInfo(&cmd, ", publication_names %s", pubnames_literal);
+	PQfreemem(pubnames_literal);
+	pfree(pubnames_str);
+
+	appendStringInfoString(&cmd, " )");
+
+	/* Execute the command. */
+	res = libpqrcv_PQexec(conn->streamConn, cmd.data);
+	pfree(cmd.data);
+
+	if (PQresultStatus(res) != PGRES_TUPLES_OK)
+	{
+		PQclear(res);
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg("Could not create a snapshot by replication slot \"%s\": %s",
+						slotname, pchomp(PQerrorMessage(conn->streamConn)))));
+	}
+
+	if (lsn)
+		*lsn = DatumGetLSN(DirectFunctionCall1Coll(pg_lsn_in, InvalidOid,
+												   CStringGetDatum(PQgetvalue(res, 0, 0))));
+
+	PQclear(res);
+}
+
 /*
  * Return PID of remote backend process.
  */
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 52d1fe6269..b62babe359 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -461,6 +461,10 @@ CreateInitDecodingContext(const char *plugin,
  * fast_forward
  *		bypass the generation of logical changes.
  *
+ * need_full_snapshot
+ * 		if true, create a snapshot able to read all tables,
+ * 		otherwise do not create any snapshot.
+ *
  * xl_routine
  *		XLogReaderRoutine used by underlying xlogreader
  *
@@ -479,6 +483,7 @@ LogicalDecodingContext *
 CreateDecodingContext(XLogRecPtr start_lsn,
 					  List *output_plugin_options,
 					  bool fast_forward,
+					  bool need_full_snapshot,
 					  XLogReaderRoutine *xl_routine,
 					  LogicalOutputPluginWriterPrepareWrite prepare_write,
 					  LogicalOutputPluginWriterWrite do_write,
@@ -487,6 +492,7 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 	LogicalDecodingContext *ctx;
 	ReplicationSlot *slot;
 	MemoryContext old_context;
+	TransactionId xmin_horizon = InvalidTransactionId;
 
 	/* shorter lines... */
 	slot = MyReplicationSlot;
@@ -533,8 +539,39 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 		start_lsn = slot->data.confirmed_flush;
 	}
 
+
+	/*
+	 * We need to determine a safe xmin horizon to start decoding from if we
+	 * want to create a snapshot too. Otherwise we would end up with a
+	 * snapshot that cannot be imported since xmin value from the snapshot may
+	 * be less than the oldest safe xmin. To avoid this call
+	 * GetOldestSafeDecodingTransactionId() to return a safe xmin value, which
+	 * can be used while exporting/importing the snapshot.
+	 *
+	 * So we have to acquire the ProcArrayLock to prevent computation of new
+	 * xmin horizons by other backends, get the safe decoding xid, and inform
+	 * the slot machinery about the new limit. Once that's done the
+	 * ProcArrayLock can be released as the slot machinery now is protecting
+	 * against vacuum.
+	 */
+	if (need_full_snapshot)
+	{
+		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+		SpinLockAcquire(&slot->mutex);
+		slot->effective_catalog_xmin = xmin_horizon;
+		slot->data.catalog_xmin = xmin_horizon;
+		slot->effective_xmin = xmin_horizon;
+		SpinLockRelease(&slot->mutex);
+
+		xmin_horizon = GetOldestSafeDecodingTransactionId(!need_full_snapshot);
+		ReplicationSlotsComputeRequiredXmin(true);
+
+		LWLockRelease(ProcArrayLock);
+	}
+
 	ctx = StartupDecodingContext(output_plugin_options,
-								 start_lsn, InvalidTransactionId, false,
+								 start_lsn, xmin_horizon, need_full_snapshot,
 								 fast_forward, xl_routine, prepare_write,
 								 do_write, update_progress);
 
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index fa1b641a2b..1191c70eb0 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -208,6 +208,7 @@ pg_logical_slot_get_changes_guts(FunctionCallInfo fcinfo, bool confirm, bool bin
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									options,
 									false,
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/repl_gram.y b/src/backend/replication/repl_gram.y
index 0c874e33cf..e5f0235d1e 100644
--- a/src/backend/replication/repl_gram.y
+++ b/src/backend/replication/repl_gram.y
@@ -65,6 +65,7 @@ Node *replication_parse_result;
 %token K_CREATE_REPLICATION_SLOT
 %token K_DROP_REPLICATION_SLOT
 %token K_TIMELINE_HISTORY
+%token K_REPLICATION_SLOT_SNAPSHOT
 %token K_WAIT
 %token K_TIMELINE
 %token K_PHYSICAL
@@ -80,7 +81,7 @@ Node *replication_parse_result;
 %type <node>	command
 %type <node>	base_backup start_replication start_logical_replication
 				create_replication_slot drop_replication_slot identify_system
-				read_replication_slot timeline_history show
+				read_replication_slot timeline_history show replication_slot_snapshot
 %type <list>	generic_option_list
 %type <defelt>	generic_option
 %type <uintval>	opt_timeline
@@ -114,6 +115,7 @@ command:
 			| read_replication_slot
 			| timeline_history
 			| show
+			| replication_slot_snapshot
 			;
 
 /*
@@ -307,6 +309,19 @@ timeline_history:
 				}
 			;
 
+/*
+ * REPLICATION_SLOT_SNAPSHOT %s options
+ */
+replication_slot_snapshot:
+			K_REPLICATION_SLOT_SNAPSHOT var_name plugin_options
+				{
+					ReplicationSlotSnapshotCmd *n = makeNode(ReplicationSlotSnapshotCmd);
+					n->slotname = $2;
+					n->options = $3;
+					$$ = (Node *) n;
+				}
+			;
+
 opt_physical:
 			K_PHYSICAL
 			| /* EMPTY */
@@ -400,6 +415,7 @@ ident_or_keyword:
 			| K_CREATE_REPLICATION_SLOT	{ $$ = "create_replication_slot"; }
 			| K_DROP_REPLICATION_SLOT		{ $$ = "drop_replication_slot"; }
 			| K_TIMELINE_HISTORY			{ $$ = "timeline_history"; }
+			| K_REPLICATION_SLOT_SNAPSHOT	{ $$ = "replication_slot_snapshot"; }
 			| K_WAIT						{ $$ = "wait"; }
 			| K_TIMELINE					{ $$ = "timeline"; }
 			| K_PHYSICAL					{ $$ = "physical"; }
diff --git a/src/backend/replication/repl_scanner.l b/src/backend/replication/repl_scanner.l
index cb467ca46f..1988a6203b 100644
--- a/src/backend/replication/repl_scanner.l
+++ b/src/backend/replication/repl_scanner.l
@@ -126,6 +126,7 @@ START_REPLICATION	{ return K_START_REPLICATION; }
 CREATE_REPLICATION_SLOT		{ return K_CREATE_REPLICATION_SLOT; }
 DROP_REPLICATION_SLOT		{ return K_DROP_REPLICATION_SLOT; }
 TIMELINE_HISTORY	{ return K_TIMELINE_HISTORY; }
+REPLICATION_SLOT_SNAPSHOT	{ return K_REPLICATION_SLOT_SNAPSHOT; }
 PHYSICAL			{ return K_PHYSICAL; }
 RESERVE_WAL			{ return K_RESERVE_WAL; }
 LOGICAL				{ return K_LOGICAL; }
@@ -303,6 +304,7 @@ replication_scanner_is_replication_command(void)
 		case K_DROP_REPLICATION_SLOT:
 		case K_READ_REPLICATION_SLOT:
 		case K_TIMELINE_HISTORY:
+		case K_REPLICATION_SLOT_SNAPSHOT:
 		case K_SHOW:
 			/* Yes; push back the first token so we can parse later. */
 			repl_pushed_back_token = first_token;
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 2f3c964824..b3ae11b2c8 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -478,6 +478,7 @@ pg_logical_replication_slot_advance(XLogRecPtr moveto)
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									NIL,
 									true,	/* fast_forward */
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 015ae2995d..70d926f4f0 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -238,6 +238,7 @@ static void CreateReplicationSlot(CreateReplicationSlotCmd *cmd);
 static void DropReplicationSlot(DropReplicationSlotCmd *cmd);
 static void StartReplication(StartReplicationCmd *cmd);
 static void StartLogicalReplication(StartReplicationCmd *cmd);
+static void ReplicationSlotSnapshot(ReplicationSlotSnapshotCmd *cmd);
 static void ProcessStandbyMessage(void);
 static void ProcessStandbyReplyMessage(void);
 static void ProcessStandbyHSFeedbackMessage(void);
@@ -1280,7 +1281,7 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	 * are reported early.
 	 */
 	logical_decoding_ctx =
-		CreateDecodingContext(cmd->startpoint, cmd->options, false,
+		CreateDecodingContext(cmd->startpoint, cmd->options, false, false,
 							  XL_ROUTINE(.page_read = logical_read_xlog_page,
 										 .segment_open = WalSndSegmentOpen,
 										 .segment_close = wal_segment_close),
@@ -1332,6 +1333,91 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	EndCommand(&qc, DestRemote, false);
 }
 
+/*
+ * Create a snapshot from an existing replication slot.
+ */
+static void
+ReplicationSlotSnapshot(ReplicationSlotSnapshotCmd *cmd)
+{
+	Snapshot	snap;
+	LogicalDecodingContext *ctx;
+	char		xloc[MAXFNAMELEN];
+	DestReceiver *dest;
+	TupOutputState *tstate;
+	TupleDesc	tupdesc;
+	Datum		values[1];
+	bool		nulls[1] = {0};
+
+	Assert(!MyReplicationSlot);
+
+	if (!IsTransactionBlock())
+		ereport(ERROR,
+				(errmsg("%s must be called inside a transaction",
+						"REPLICATION_SLOT_SNAPSHOT ...")));
+
+	if (XactIsoLevel != XACT_REPEATABLE_READ)
+		ereport(ERROR,
+				(errmsg("%s must be called in REPEATABLE READ isolation mode transaction",
+						"REPLICATION_SLOT_SNAPSHOT ...")));
+
+	if (FirstSnapshotSet)
+		ereport(ERROR,
+				(errmsg("%s must be called before any query",
+						"REPLICATION_SLOT_SNAPSHOT ...")));
+
+	if (IsSubTransaction())
+		ereport(ERROR,
+				(errmsg("%s must not be called in a subtransaction",
+						"REPLICATION_SLOT_SNAPSHOT ...")));
+
+	ReplicationSlotAcquire(cmd->slotname, false);
+
+	ctx = CreateDecodingContext(MyReplicationSlot->data.restart_lsn,
+								cmd->options,
+								false,
+								true,
+								XL_ROUTINE(.page_read = logical_read_xlog_page,
+										   .segment_open = WalSndSegmentOpen,
+										   .segment_close = wal_segment_close),
+								WalSndPrepareWrite, WalSndWriteData,
+								WalSndUpdateProgress);
+
+	/*
+	 * Signal that we don't need the timeout mechanism. We're just creating
+	 * the replication slot and don't yet accept feedback messages or send
+	 * keepalives. As we possibly need to wait for further WAL the walsender
+	 * would otherwise possibly be killed too soon.
+	 */
+	last_reply_timestamp = 0;
+
+	/* build initial snapshot, might take a while */
+	DecodingContextFindStartpoint(ctx);
+
+	snap = SnapBuildInitialSnapshot(ctx->snapshot_builder);
+	RestoreTransactionSnapshot(snap, MyProc);
+
+	/* Don't need the decoding context anymore */
+	FreeDecodingContext(ctx);
+
+	/* Create a tuple to send consisten WAL location */
+	snprintf(xloc, sizeof(xloc), "%X/%X",
+			 LSN_FORMAT_ARGS(MyReplicationSlot->data.confirmed_flush));
+
+	dest = CreateDestReceiver(DestRemoteSimple);
+	tupdesc = CreateTemplateTupleDesc(1);
+	TupleDescInitBuiltinEntry(tupdesc, (AttrNumber) 1, "consistent_point",
+							  TEXTOID, -1, 0);
+	tstate = begin_tup_output_tupdesc(dest, tupdesc, &TTSOpsVirtual);
+
+	/* consistent wal location */
+	values[0] = CStringGetTextDatum(xloc);
+
+	do_tup_output(tstate, values, nulls);
+	end_tup_output(tstate);
+
+	ReplicationSlotRelease();
+}
+
 /*
  * LogicalDecodingContext 'prepare_write' callback.
  *
@@ -1860,6 +1946,15 @@ exec_replication_command(const char *cmd_string)
 			}
 			break;
 
+		case T_ReplicationSlotSnapshotCmd:
+			{
+				cmdtag = "REPLICATION_SLOT_SNAPSHOT";
+				set_ps_display(cmdtag);
+				ReplicationSlotSnapshot((ReplicationSlotSnapshotCmd *) cmd_node);
+				EndReplicationCommand(cmdtag);
+				break;
+			}
+
 		default:
 			elog(ERROR, "unrecognized replication command node tag: %u",
 				 cmd_node->type);
diff --git a/src/include/nodes/replnodes.h b/src/include/nodes/replnodes.h
index 4321ba8f86..44a4580671 100644
--- a/src/include/nodes/replnodes.h
+++ b/src/include/nodes/replnodes.h
@@ -108,4 +108,15 @@ typedef struct TimeLineHistoryCmd
 	TimeLineID	timeline;
 } TimeLineHistoryCmd;
 
+/* ----------------------
+ *		REPLICATION_SLOT_SNAPSHOT command
+ * ----------------------
+ */
+typedef struct ReplicationSlotSnapshotCmd
+{
+	NodeTag		type;
+	char	   *slotname;
+	List	   *options;
+} ReplicationSlotSnapshotCmd;
+
 #endif							/* REPLNODES_H */
diff --git a/src/include/replication/logical.h b/src/include/replication/logical.h
index 5f49554ea0..6535786a0e 100644
--- a/src/include/replication/logical.h
+++ b/src/include/replication/logical.h
@@ -125,6 +125,7 @@ extern LogicalDecodingContext *CreateInitDecodingContext(const char *plugin,
 extern LogicalDecodingContext *CreateDecodingContext(XLogRecPtr start_lsn,
 													 List *output_plugin_options,
 													 bool fast_forward,
+													 bool need_full_snapshot,
 													 XLogReaderRoutine *xl_routine,
 													 LogicalOutputPluginWriterPrepareWrite prepare_write,
 													 LogicalOutputPluginWriterWrite do_write,
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index decffe352d..bd11f9f31e 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -384,6 +384,16 @@ typedef WalRcvExecResult *(*walrcv_exec_fn) (WalReceiverConn *conn,
  */
 typedef void (*walrcv_disconnect_fn) (WalReceiverConn *conn);
 
+/*
+ * walrcv_slot_snapshot_fn
+ *
+ * Create a snapshot by an existing replication slot
+ */
+typedef void (*walrcv_slot_snapshot_fn) (WalReceiverConn *conn,
+										 char *slotname,
+										 const WalRcvStreamOptions *options,
+										 XLogRecPtr *lsn);
+
 typedef struct WalReceiverFunctionsType
 {
 	walrcv_connect_fn walrcv_connect;
@@ -401,6 +411,7 @@ typedef struct WalReceiverFunctionsType
 	walrcv_get_backend_pid_fn walrcv_get_backend_pid;
 	walrcv_exec_fn walrcv_exec;
 	walrcv_disconnect_fn walrcv_disconnect;
+	walrcv_slot_snapshot_fn walrcv_slot_snapshot;
 } WalReceiverFunctionsType;
 
 extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
@@ -435,6 +446,8 @@ extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
 	WalReceiverFunctions->walrcv_exec(conn, exec, nRetTypes, retTypes)
 #define walrcv_disconnect(conn) \
 	WalReceiverFunctions->walrcv_disconnect(conn)
+#define walrcv_slot_snapshot(conn, slotname, options, lsn) \
+	WalReceiverFunctions->walrcv_slot_snapshot(conn, slotname, options, lsn)
 
 static inline void
 walrcv_clear_result(WalRcvExecResult *walres)
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 23bafec5f7..209918c380 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2322,6 +2322,7 @@ ReplicationSlotCtlData
 ReplicationSlotOnDisk
 ReplicationSlotPersistency
 ReplicationSlotPersistentData
+ReplicationSlotSnapshotCmd
 ReplicationState
 ReplicationStateCtl
 ReplicationStateOnDisk
-- 
2.25.1

v7-0002-Reuse-Logical-Replication-Background-worker.patchapplication/octet-stream; name=v7-0002-Reuse-Logical-Replication-Background-worker.patchDownload
From fcac0624791d965687d284898237a2721afec4ab Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH 2/2] Reuse Logical Replication Background worker

This commit allows tablesync workers to move to another table that needs synchronization,
when they're done with the current table in tablesync phase of Logical Replication.

Before this commit, tablesync workers were capable of syncing only one
relation. A new worker, replication slot and origin were needed for each
relation in the subscription.

Now, tablesync workers are not only limited with one relation and can move to another relation and reuse existing
replication slots and origins

This reduces the overhead of launching/killing a new background worker for each relation.
By reusing tablesync workers, replication slots and origins created for tablesync can be reused as well.
Removing the burden of creating/dropping replication slot/origin improves tablesync speed significantly especially for empty or small tables.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/catalogs.sgml                    |  29 ++
 src/backend/catalog/pg_subscription.c         | 284 +++++++++++-
 src/backend/commands/subscriptioncmds.c       | 238 ++++++----
 .../replication/logical/applyparallelworker.c |   7 +-
 src/backend/replication/logical/launcher.c    |   9 +-
 src/backend/replication/logical/tablesync.c   | 434 +++++++++++++-----
 src/backend/replication/logical/worker.c      | 395 ++++++++++------
 src/include/catalog/pg_subscription.h         |   6 +
 src/include/catalog/pg_subscription_rel.h     |  15 +-
 src/include/replication/slot.h                |   3 +-
 src/include/replication/worker_internal.h     |  33 +-
 src/test/regress/expected/misc_sanity.out     |  30 +-
 12 files changed, 1113 insertions(+), 370 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index c1e4048054..e34c129f0f 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8002,6 +8002,17 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        origin.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>sublastusedid</structfield> <type>int8</type>
+      </para>
+      <para>
+       The last used ID for tablesync workers. This ID is used to
+       create replication slots. The last used ID needs to be stored
+       to make logical replication can safely proceed after any interruption.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
@@ -8086,6 +8097,24 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        otherwise null
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srrelslotname</structfield> <type>text</type>
+      </para>
+      <para>
+       Replication slot name that is used for synchronization of relation
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srreloriginname</structfield> <type>text</type>
+      </para>
+      <para>
+       Origin name that is used for tracking synchronization of relation
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index a56ae311c3..98d627ff49 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -114,6 +114,14 @@ GetSubscription(Oid subid, bool missing_ok)
 	Assert(!isnull);
 	sub->origin = TextDatumGetCString(datum);
 
+	/* Get last used id */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
+							tup,
+							Anum_pg_subscription_sublastusedid,
+							&isnull);
+	Assert(!isnull);
+	sub->lastusedid = DatumGetInt64(datum);
+
 	ReleaseSysCache(tup);
 
 	return sub;
@@ -205,6 +213,44 @@ DisableSubscription(Oid subid)
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update the last used replication slot ID for the given subscription.
+ */
+void
+UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid)
+{
+	Relation	rel;
+	bool		nulls[Natts_pg_subscription];
+	bool		replaces[Natts_pg_subscription];
+	Datum		values[Natts_pg_subscription];
+	HeapTuple	tup;
+
+	/* Look up the subscription in the catalog */
+	rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+	tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	/* Form a new tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	replaces[Anum_pg_subscription_sublastusedid - 1] = true;
+	values[Anum_pg_subscription_sublastusedid- 1] = Int64GetDatum(lastusedid);
+
+	/* Update the catalog */
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+	heap_freetuple(tup);
+
+	table_close(rel, NoLock);
+}
+
 /*
  * Convert text array to list of strings.
  *
@@ -234,7 +280,7 @@ textarray_to_stringlist(ArrayType *textarray)
  */
 void
 AddSubscriptionRelState(Oid subid, Oid relid, char state,
-						XLogRecPtr sublsn)
+						XLogRecPtr sublsn, char *relslotname, char *reloriginname)
 {
 	Relation	rel;
 	HeapTuple	tup;
@@ -263,6 +309,14 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
 	else
 		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] = CStringGetTextDatum(relslotname);
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] = CStringGetTextDatum(reloriginname);
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -275,6 +329,58 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Internal function to modify columns for relation state update
+ */
+static void
+UpdateSubscriptionRelState_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char state,
+									XLogRecPtr sublsn)
+{
+	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
+	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+
+	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (sublsn != InvalidXLogRecPtr)
+		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
+	else
+		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication slot update
+ */
+static void
+UpdateSubscriptionRelReplicationSlot_internal(Datum *values,
+											bool *nulls,
+											bool *replaces,
+											char *relslotname)
+{
+	replaces[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] = CStringGetTextDatum(relslotname);
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication origin update
+ */
+static void
+UpdateSubscriptionRelOrigin_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char *reloriginname)
+{
+	replaces[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] = CStringGetTextDatum(reloriginname);
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+}
+
 /*
  * Update the state of a subscription table.
  */
@@ -305,14 +411,48 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	memset(nulls, false, sizeof(nulls));
 	memset(replaces, false, sizeof(replaces));
 
-	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
-	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
 
-	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
-	if (sublsn != InvalidXLogRecPtr)
-		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
-	else
-		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update the replication slot name of a subscription table.
+ */
+void
+UpdateSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *relslotname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
 
 	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
 							replaces);
@@ -324,6 +464,134 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update replication slot name, origin name and state of
+ * a subscription table in one transaction.
+ */
+void
+UpdateSubscriptionRel(Oid subid,
+					  Oid relid,
+					  char state,
+					  XLogRecPtr sublsn,
+					  char *relslotname,
+					  char *reloriginname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
+	UpdateSubscriptionRelOrigin_internal(values, nulls, replaces, reloriginname);
+
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Get origin name of subscription table.
+ *
+ * Returns null if the subscription table does not have a origin.
+ */
+void
+GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*originname;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srreloriginname, isnull);
+	if (!*isnull)
+	{
+		originname = TextDatumGetCString(d);
+		memcpy(reloriginname, originname, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
+/*
+ * Get replication slot name of subscription table.
+ *
+ * Returns null if the subscription table does not have a replication slot.
+ */
+void
+GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*relrepslot;
+	bool		isnull;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srrelslotname, &isnull);
+	if (!isnull)
+	{
+		relrepslot = TextDatumGetCString(d);
+		memcpy(slotname, relrepslot, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
 /*
  * Get state of subscription table.
  *
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index baff00dd74..2a68648a2b 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -649,6 +649,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 		publicationListToArray(publications);
 	values[Anum_pg_subscription_suborigin - 1] =
 		CStringGetTextDatum(opts.origin);
+	values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(1);
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -658,7 +659,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 
 	recordDependencyOnOwner(SubscriptionRelationId, subid, owner);
 
-	ReplicationOriginNameForLogicalRep(subid, InvalidOid, originname, sizeof(originname));
+	ReplicationOriginNameForLogicalRep(subid, originname, sizeof(originname), false);
 	replorigin_create(originname);
 
 	/*
@@ -709,7 +710,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 										 rv->schemaname, rv->relname);
 
 				AddSubscriptionRelState(subid, relid, table_state,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 			}
 
 			/*
@@ -799,6 +800,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	} SubRemoveRels;
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -876,7 +879,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				AddSubscriptionRelState(sub->oid, relid,
 										copy_data ? SUBREL_STATE_INIT : SUBREL_STATE_READY,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 				ereport(DEBUG1,
 						(errmsg_internal("table \"%s.%s\" added to subscription \"%s\"",
 										 rv->schemaname, rv->relname, sub->name)));
@@ -900,6 +903,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				char		state;
 				XLogRecPtr	statelsn;
+				char		slotname[NAMEDATALEN] = {0};
 
 				/*
 				 * Lock pg_subscription_rel with AccessExclusiveLock to
@@ -926,7 +930,29 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/*
+				 * Find the logical replication sync worker if exists store
+				 * the slot number for dropping associated replication slots
+				 * later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot_name);
+				}
+				else
+				{
+					/*
+					 * Sync of this relation might be failed in an earlier
+					 * attempt, but the replication slot might still exist.
+					 */
+					GetSubscriptionRelReplicationSlot(sub->oid, relid, slotname);
+					if (strlen(slotname) > 0)
+						sub_remove_slots = lappend(sub_remove_slots, slotname);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
@@ -946,8 +972,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 					 * origin and by this time the origin might be already
 					 * removed. For these reasons, passing missing_ok = true.
 					 */
-					ReplicationOriginNameForLogicalRep(sub->oid, relid, originname,
-													   sizeof(originname));
+					ReplicationOriginNameForLogicalRep(sub->oid, originname,
+													   sizeof(originname), true);
 					replorigin_drop_by_name(originname, true, false);
 				}
 
@@ -960,31 +986,24 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
-		 * the database operations we won't be able to rollback dropped slots.
+		 * Drop the replication slots associated with tablesync workers for
+		 * removed tables. This has to be at the end because otherwise if
+		 * there is an error while doing the database operations we won't be
+		 * able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			memcpy(syncslotname, lfirst(lc), sizeof(NAMEDATALEN));
+
+			/*
+			 * There is no certainty, maybe the slot does not exist yet. Also,
+			 * if we fail after removing some of the slots, next time, it will
+			 * again try to drop already dropped slots and fail. For these
+			 * reasons, we allow missing_ok = true for the drop.
+			 */
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1320,8 +1339,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
 					char		originname[NAMEDATALEN];
 					XLogRecPtr	remote_lsn;
 
-					ReplicationOriginNameForLogicalRep(subid, InvalidOid,
-													   originname, sizeof(originname));
+					ReplicationOriginNameForLogicalRep(subid, originname,
+													   sizeof(originname), false);
 					originid = replorigin_by_name(originname, false);
 					remote_lsn = replorigin_get_progress(originid, false);
 
@@ -1384,6 +1403,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	char	   *subname;
 	char	   *conninfo;
 	char	   *slotname;
+	int64		lastusedid;
 	List	   *subworkers;
 	ListCell   *lc;
 	char		originname[NAMEDATALEN];
@@ -1455,6 +1475,14 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	else
 		slotname = NULL;
 
+	/* Get the last used identifier by the subscription */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID, tup,
+							Anum_pg_subscription_sublastusedid, &isnull);
+	if (!isnull)
+		lastusedid = DatumGetInt64(datum);
+	else
+		lastusedid = 0;
+
 	/*
 	 * Since dropping a replication slot is not transactional, the replication
 	 * slot stays dropped even if the transaction rolls back.  So we cannot
@@ -1504,37 +1532,29 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	}
 	list_free(subworkers);
 
+	rstates = GetSubscriptionRelations(subid, true);
+
 	/*
-	 * Cleanup of tablesync replication origins.
-	 *
-	 * Any READY-state relations would already have dealt with clean-ups.
+	 * Cleanup of tablesync replication origins associated with the
+	 * subscription, if exists. Try to drop origins by creating all origin
+	 * names created for this subscription.
 	 *
 	 * Note that the state can't change because we have already stopped both
 	 * the apply and tablesync workers and they can't restart because of
 	 * exclusive lock on the subscription.
+	 *
+	 * XXX: This can be handled better instead of looping through all possible
 	 */
-	rstates = GetSubscriptionRelations(subid, true);
-	foreach(lc, rstates)
+	for (int64 i = 1; i <= lastusedid; i++)
 	{
-		SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-		Oid			relid = rstate->relid;
-
-		/* Only cleanup resources of tablesync workers */
-		if (!OidIsValid(relid))
-			continue;
+		char		originname_to_drop[NAMEDATALEN] = {0};
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * It is possible that the origin is not yet created for tablesync
-		 * worker so passing missing_ok = true. This can happen for the states
-		 * before SUBREL_STATE_FINISHEDCOPY.
-		 */
-		ReplicationOriginNameForLogicalRep(subid, relid, originname,
-										   sizeof(originname));
-		replorigin_drop_by_name(originname, true, false);
+		snprintf(originname_to_drop, sizeof(originname_to_drop), "pg_%u_%lld", subid, (long long) i);
+		/* missin_ok = true, since the origin might be already dropped. */
+		replorigin_drop_by_name(originname_to_drop, true, false);
 	}
 
+
 	/* Clean up dependencies */
 	deleteSharedDependencyRecordsFor(SubscriptionRelationId, subid, 0);
 
@@ -1542,7 +1562,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	RemoveSubscriptionRel(subid, InvalidOid);
 
 	/* Remove the origin tracking if exists. */
-	ReplicationOriginNameForLogicalRep(subid, InvalidOid, originname, sizeof(originname));
+	ReplicationOriginNameForLogicalRep(subid, originname, sizeof(originname), false);
 	replorigin_drop_by_name(originname, true, false);
 
 	/*
@@ -1586,39 +1606,17 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
+		List	   *slots = NULL;
 
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
 
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char	   *syncslotname = (char *) lfirst(lc);
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1641,6 +1639,71 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * Get the replication slot names associated with the subscription.
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok)
+{
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+					 " FROM pg_replication_slots"
+					 " WHERE slot_name LIKE 'pg_%i_sync_%%';",
+					 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+					errmsg("not tuple returned."));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();
+	\
+		return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -1995,6 +2058,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -2005,18 +2069,20 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
-			char		syncslotname[NAMEDATALEN] = {0};
-
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
-				 syncslotname);
+				 worker->slot_name);
 		}
 	}
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 2e5914d5d9..8df3a12c19 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -440,7 +440,8 @@ pa_launch_parallel_worker(void)
 										MySubscription->name,
 										MyLogicalRepWorker->userid,
 										InvalidOid,
-										dsm_segment_handle(winfo->dsm_seg));
+										dsm_segment_handle(winfo->dsm_seg),
+										0);
 
 	if (launched)
 	{
@@ -942,8 +943,8 @@ ParallelApplyWorkerMain(Datum main_arg)
 
 	/* Setup replication origin tracking. */
 	StartTransactionCommand();
-	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-									   originname, sizeof(originname));
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, originname,
+									   sizeof(originname), false);
 	originid = replorigin_by_name(originname, false);
 
 	/*
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index afb7acddaa..f817037479 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -276,7 +276,7 @@ logicalrep_workers_find(Oid subid, bool only_running)
  */
 bool
 logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
-						 Oid relid, dsm_handle subworker_dsm)
+						 Oid relid, dsm_handle subworker_dsm, int64 slotid)
 {
 	BackgroundWorker bgw;
 	BackgroundWorkerHandle *bgw_handle;
@@ -401,7 +401,11 @@ retry:
 	/* Prepare the worker slot. */
 	worker->launch_time = now;
 	worker->in_use = true;
+	worker->is_first_run = true;
 	worker->generation++;
+	worker->created_slot = false;
+	worker->rep_slot_id = slotid;
+	worker->slot_name = (char *) palloc(NAMEDATALEN);
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -409,6 +413,7 @@ retry:
 	worker->relid = relid;
 	worker->relstate = SUBREL_STATE_UNKNOWN;
 	worker->relstate_lsn = InvalidXLogRecPtr;
+	worker->move_to_next_rel = false;
 	worker->stream_fileset = NULL;
 	worker->apply_leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
@@ -1015,7 +1020,7 @@ ApplyLauncherMain(Datum main_arg)
 					wait_time = wal_retrieve_retry_interval;
 
 					logicalrep_worker_launch(sub->dbid, sub->oid, sub->name,
-											 sub->owner, InvalidOid, DSM_HANDLE_INVALID);
+											 sub->owner, InvalidOid, DSM_HANDLE_INVALID, 0);
 				}
 			}
 
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 38dfce7129..01925d4e99 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -126,12 +126,8 @@ static bool FetchTableStates(bool *started_tx);
 
 static StringInfo copybuf = NULL;
 
-/*
- * Exit routine for synchronization worker.
- */
 static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+clean_sync_worker(void)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -143,18 +139,28 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
-	/* And flush all writes. */
-	XLogFlush(GetXLogWriteRecPtr());
-
-	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
-	CommitTransactionCommand();
+	/*
+	 * Disconnect from publisher. Otherwise reused sync workers causes
+	 * exceeding max_wal_senders
+	 */
+	walrcv_disconnect(LogRepWorkerWalRcvConn);
+	LogRepWorkerWalRcvConn = NULL;
 
 	/* Find the leader apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
+}
+
+/*
+ * Exit routine for synchronization worker.
+ */
+static void
+pg_attribute_noreturn()
+finish_sync_worker(void)
+{
+	clean_sync_worker();
+
+	/* And flush all writes. */
+	XLogFlush(GetXLogWriteRecPtr());
 
 	/* Stop gracefully */
 	proc_exit(0);
@@ -284,6 +290,10 @@ invalidate_syncing_table_states(Datum arg, int cacheid, uint32 hashvalue)
 static void
 process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 {
+	List	   *rstates;
+	SubscriptionRelState *rstate;
+	ListCell   *lc;
+
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_CATCHUP &&
@@ -292,6 +302,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		TimeLineID	tli;
 		char		syncslotname[NAMEDATALEN] = {0};
 		char		originname[NAMEDATALEN] = {0};
+		bool		is_streaming_ended = false;
 
 		MyLogicalRepWorker->relstate = SUBREL_STATE_SYNCDONE;
 		MyLogicalRepWorker->relstate_lsn = current_lsn;
@@ -308,40 +319,29 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 								   MyLogicalRepWorker->relid,
 								   MyLogicalRepWorker->relstate,
 								   MyLogicalRepWorker->relstate_lsn);
+		CommitTransactionCommand();
 
 		/*
-		 * End streaming so that LogRepWorkerWalRcvConn can be used to drop
-		 * the slot.
-		 */
-		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
-
-		/*
-		 * Cleanup the tablesync slot.
+		 * Cleanup the tablesync slot. If the slot name used by this worker is
+		 * different from the default slot name for the worker, this means the
+		 * current table had started to being synchronized by another worker
+		 * and replication slot. And this worker is reusing a replication slot
+		 * from a previous attempt. We do not need that replication slot
+		 * anymore.
 		 *
 		 * This has to be done after updating the state because otherwise if
 		 * there is an error while doing the database operations we won't be
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
+										MyLogicalRepWorker->rep_slot_id,
 										syncslotname,
 										sizeof(syncslotname));
 
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
-		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
-
-		CommitTransactionCommand();
-		pgstat_report_stat(false);
-
-		/*
-		 * Start a new transaction to clean up the tablesync origin tracking.
-		 * This transaction will be ended within the finish_sync_worker().
-		 * Now, even, if we fail to remove this here, the apply worker will
-		 * ensure to clean it up afterward.
+		 * We are safe to drop the replication trackin origin after this
+		 * point. Now, even, if we fail to remove this here, the apply worker
+		 * will ensure to clean it up afterward.
 		 *
 		 * We need to do this after the table state is set to SYNCDONE.
 		 * Otherwise, if an error occurs while performing the database
@@ -350,34 +350,125 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 * have been cleared before restart. So, the restarted worker will use
 		 * invalid replication progress state resulting in replay of
 		 * transactions that have already been applied.
+		 *
+		 * Firstly reset the origin session to remove the ownership of the
+		 * slot. This is needed to allow the origin to be dropped or reused
+		 * later.
 		 */
+		replorigin_session_reset();
+		replorigin_session_origin = InvalidRepOriginId;
+		replorigin_session_origin_lsn = InvalidXLogRecPtr;
+		replorigin_session_origin_timestamp = 0;
+
 		StartTransactionCommand();
+		if (MyLogicalRepWorker->slot_name && strcmp(syncslotname, MyLogicalRepWorker->slot_name) != 0)
+		{
+			/*
+			 * End streaming so that LogRepWorkerWalRcvConn can be used to
+			 * drop the slot.
+			 */
+			walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+			is_streaming_ended = true;
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
 
-		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
+			ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+											   originname,
+											   sizeof(originname),
+											   true);
+
+			/* Drop replication origin */
+			replorigin_drop_by_name(originname, true, false);
+		}
 
 		/*
-		 * Resetting the origin session removes the ownership of the slot.
-		 * This is needed to allow the origin to be dropped.
+		 * We are safe to remove persisted replication slot and origin data,
+		 * since it's already in SYNCDONE state. They will not be needed
+		 * anymore.
 		 */
-		replorigin_session_reset();
-		replorigin_session_origin = InvalidRepOriginId;
-		replorigin_session_origin_lsn = InvalidXLogRecPtr;
-		replorigin_session_origin_timestamp = 0;
+		UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+							  MyLogicalRepWorker->relid,
+							  MyLogicalRepWorker->relstate,
+							  MyLogicalRepWorker->relstate_lsn,
+							  NULL,
+							  NULL);
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid))));
+
+		CommitTransactionCommand();
+		pgstat_report_stat(false);
+
+		StartTransactionCommand();
 
 		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * There is a chance that the user is concurrently performing refresh
-		 * for the subscription where we remove the table state and its origin
-		 * or the apply worker would have removed this origin. So passing
-		 * missing_ok = true.
+		 * This should return the default origin name for the worker. Even if
+		 * the worker used a different origin for this table, it should be
+		 * dropped and removed from the catalog so far.
+		 */
+		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+										   originname,
+										   sizeof(originname),
+										   true);
+
+		/*
+		 * Check if any table whose relation state is still INIT. If a table
+		 * in INIT state is found, the worker will not be finished, it will be
+		 * reused instead.
 		 */
-		replorigin_drop_by_name(originname, true, false);
+		rstates = GetSubscriptionRelations(MySubscription->oid, true);
+
+		foreach(lc, rstates)
+		{
+			rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
+			memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
+
+			/*
+			 * Pick the table for the next run if there is not another worker
+			 * already picked that table.
+			 */
+			LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+			if (rstate->state != SUBREL_STATE_SYNCDONE &&
+				!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+			{
+				/* Update worker state for the next table */
+				MyLogicalRepWorker->is_first_run = false;
+				MyLogicalRepWorker->relid = rstate->relid;
+				MyLogicalRepWorker->relstate = rstate->state;
+				MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+				MyLogicalRepWorker->move_to_next_rel = true;
+				LWLockRelease(LogicalRepWorkerLock);
+				break;
+			}
+			LWLockRelease(LogicalRepWorkerLock);
+		}
+
+		/* Cleanup before next run or ending the worker. */
+		if (!MyLogicalRepWorker->move_to_next_rel)
+		{
+			/*
+			 * It is important to give an error if we are unable to drop the
+			 * slot, otherwise, it won't be dropped till the corresponding
+			 * subscription is dropped. So passing missing_ok = false.
+			 */
+			if (MyLogicalRepWorker->created_slot)
+			{
+				/* End streaming if it's not already ended. */
+				if (!is_streaming_ended)
+					walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+				ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
+			}
+
+			/* Drop replication origin before exiting. */
+			replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
+			finish_sync_worker();
+		}
+		else
+		{
+			clean_sync_worker();
+		}
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -464,6 +555,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 			if (current_lsn >= rstate->lsn)
 			{
 				char		originname[NAMEDATALEN];
+				bool		is_origin_null = true;
 
 				rstate->state = SUBREL_STATE_READY;
 				rstate->lsn = current_lsn;
@@ -484,18 +576,27 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				 * error while dropping we won't restart it to drop the
 				 * origin. So passing missing_ok = true.
 				 */
-				ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-												   rstate->relid,
-												   originname,
-												   sizeof(originname));
-				replorigin_drop_by_name(originname, true, false);
+				GetSubscriptionRelOrigin(MyLogicalRepWorker->subid,
+										 rstate->relid, originname,
+										 &is_origin_null);
+
+				if (!is_origin_null)
+				{
+					replorigin_drop_by_name(originname, true, false);
+				}
 
 				/*
 				 * Update the state to READY only after the origin cleanup.
 				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-										   rstate->relid, rstate->state,
-										   rstate->lsn);
+				UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+									  rstate->relid,
+									  rstate->state,
+									  rstate->lsn,
+									  NULL,
+									  NULL);
+
+				CommitTransactionCommand();
+				started_tx = false;
 			}
 		}
 		else
@@ -584,12 +685,22 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						TimestampDifferenceExceeds(hentry->last_start_time, now,
 												   wal_retrieve_retry_interval))
 					{
+						if (IsTransactionState())
+							CommitTransactionCommand();
+						StartTransactionCommand();
+						started_tx = true;
+
+						MySubscription->lastusedid++;
+						UpdateSubscriptionLastSlotId(MyLogicalRepWorker->subid,
+													 MySubscription->lastusedid);
+
 						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
 												 MySubscription->oid,
 												 MySubscription->name,
 												 MyLogicalRepWorker->userid,
 												 rstate->relid,
-												 DSM_HANDLE_INVALID);
+												 DSM_HANDLE_INVALID,
+												 MySubscription->lastusedid);
 						hentry->last_start_time = now;
 					}
 				}
@@ -1190,8 +1301,8 @@ copy_table(Relation rel)
  * The name must not exceed NAMEDATALEN - 1 because of remote node constraints
  * on slot name length. We append system_identifier to avoid slot_name
  * collision with subscriptions in other clusters. With the current scheme
- * pg_%u_sync_%u_UINT64_FORMAT (3 + 10 + 6 + 10 + 20 + '\0'), the maximum
- * length of slot_name will be 50.
+ * pg_%u_sync_%lu_UINT64_FORMAT (3 + 10 + 6 + 20 + 20 + '\0'), the maximum
+ * length of slot_name will be 45.
  *
  * The returned slot name is stored in the supplied buffer (syncslotname) with
  * the given size.
@@ -1202,11 +1313,11 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
+ReplicationSlotNameForTablesync(Oid suboid, int64 slotid,
 								char *syncslotname, Size szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%lld_" UINT64_FORMAT, suboid,
+			(long long) slotid, GetSystemIdentifier());
 }
 
 /*
@@ -1229,6 +1340,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	WalRcvExecResult *res;
 	char		originname[NAMEDATALEN];
 	RepOriginId originid;
+	char	   *prev_slotname;
 
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
@@ -1257,7 +1369,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
+									MyLogicalRepWorker->rep_slot_id,
 									slotname,
 									NAMEDATALEN);
 
@@ -1277,11 +1389,25 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC ||
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY);
 
+	/*
+	 * See if tablesync of the current relation has been started with another
+	 * replication slot.
+	 *
+	 * Read previous slot name from the catalog, if exists.
+	 */
+	prev_slotname = (char *) palloc0(NAMEDATALEN);
+	StartTransactionCommand();
+	GetSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+									  MyLogicalRepWorker->relid,
+									  prev_slotname);
+
 	/* Assign the origin tracking record name. */
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
-									   MyLogicalRepWorker->relid,
 									   originname,
-									   sizeof(originname));
+									   sizeof(originname),
+									   true);
+
+	CommitTransactionCommand();
 
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC)
 	{
@@ -1296,10 +1422,48 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		 * breakdown then it wouldn't have succeeded so trying it next time
 		 * seems like a better bet.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, slotname, true);
+		if (strlen(prev_slotname) > 0)
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);
+
+			StartTransactionCommand();
+			/* Replication drop might still exist. Try to drop */
+			replorigin_drop_by_name(originname, true, false);
+
+			/*
+			 * Remove replication slot and origin name from the relation's
+			 * catalog record
+			 */
+			UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+								  MyLogicalRepWorker->relid,
+								  MyLogicalRepWorker->relstate,
+								  MyLogicalRepWorker->relstate_lsn,
+								  NULL,
+								  NULL);
+			CommitTransactionCommand();
+		}
 	}
 	else if (MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY)
 	{
+		/*
+		 * At this point, the table that is currently being synchronized
+		 * should have its replication slot name filled in the catalog. The
+		 * tablesync process was started with another sync worker and
+		 * replication slot. We need to continue using the same replication
+		 * slot in this worker too.
+		 */
+		if (strlen(prev_slotname) == 0)
+		{
+			elog(ERROR, "Replication slot could not be found for relation %u",
+				 MyLogicalRepWorker->relid);
+		}
+
+		/*
+		 * Proceed with the correct replication slot. Use previously created
+		 * replication slot to sync this table.
+		 */
+		slotname = prev_slotname;
+
 		/*
 		 * The COPY phase was previously done, but tablesync then crashed
 		 * before it was able to finish normally.
@@ -1320,6 +1484,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		goto copy_table_done;
 	}
 
+	/* Preparing for table copy operation */
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 	MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
 	MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
@@ -1327,10 +1492,12 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   MyLogicalRepWorker->relstate,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  MyLogicalRepWorker->relstate,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 	CommitTransactionCommand();
 	pgstat_report_stat(true);
 
@@ -1369,6 +1536,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						GetUserNameFromId(GetUserId(), true),
 						RelationGetRelationName(rel))));
 
+
 	/*
 	 * Start a transaction in the remote node in REPEATABLE READ mode.  This
 	 * ensures that both the replication slot we create (see below) and the
@@ -1384,55 +1552,99 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						res->err)));
 	walrcv_clear_result(res);
 
+	originid = replorigin_by_name(originname, true);
+
 	/*
 	 * Create a new permanent logical decoding slot. This slot will be used
 	 * for the catchup phase after COPY is done, so tell it to use the
 	 * snapshot to make the final data consistent.
 	 *
+	 * Replication slot will only be created if either this is the first run
+	 * of the worker or we're not using a previous replication slot.
+	 *
 	 * Prevent cancel/die interrupts while creating slot here because it is
 	 * possible that before the server finishes this command, a concurrent
 	 * drop subscription happens which would complete without removing this
 	 * slot leading to a dangling slot on the server.
+	 *
 	 */
-	HOLD_INTERRUPTS();
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-	RESUME_INTERRUPTS();
-
-	/*
-	 * Setup replication origin tracking. The purpose of doing this before the
-	 * copy is to avoid doing the copy again due to any error in setting up
-	 * origin tracking.
-	 */
-	originid = replorigin_by_name(originname, true);
-	if (!OidIsValid(originid))
+	if (!MyLogicalRepWorker->created_slot)
 	{
+		HOLD_INTERRUPTS();
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						   slotname, false /* permanent */ , false /* two_phase */ ,
+						   CRS_USE_SNAPSHOT, origin_startpos);
+		RESUME_INTERRUPTS();
+
 		/*
-		 * Origin tracking does not exist, so create it now.
-		 *
-		 * Then advance to the LSN got from walrcv_create_slot. This is WAL
-		 * logged for the purpose of recovery. Locks are to prevent the
-		 * replication origin from vanishing while advancing.
+		 * Remember that we created the slot so that we will not try to create
+		 * it again.
 		 */
-		originid = replorigin_create(originname);
-
-		LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
-		replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
-						   true /* go backward */ , true /* WAL log */ );
-		UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->created_slot = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
+		/*
+		 * Setup replication origin tracking. The purpose of doing this before
+		 * the copy is to avoid doing the copy again due to any error in
+		 * setting up origin tracking.
+		 */
+		if (!OidIsValid(originid))
+		{
+			/*
+			 * Origin tracking does not exist, so create it now.
+			 */
+			originid = replorigin_create(originname);
+		}
+		else
+		{
+			/*
+			 * At this point, there shouldn't be any existing replication
+			 * origin wit the same name.
+			 */
+			ereport(ERROR,
+					(errcode(ERRCODE_DUPLICATE_OBJECT),
+					 errmsg("replication origin \"%s\" already exists",
+							originname)));
+		}
 	}
 	else
 	{
-		ereport(ERROR,
-				(errcode(ERRCODE_DUPLICATE_OBJECT),
-				 errmsg("replication origin \"%s\" already exists",
-						originname)));
+		/*
+		 * Do not create a new replication slot, reuse the existing one
+		 * instead. Use a new snapshot for the replication slot to ensure that
+		 * tablesync and apply proceses are consistent with each other.
+		 */
+		WalRcvStreamOptions options;
+		int			server_version;
+
+		server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+		options.proto.logical.proto_version =
+			server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+			server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+			LOGICALREP_PROTO_VERSION_NUM;
+		options.proto.logical.publication_names = MySubscription->publications;
+
+		HOLD_INTERRUPTS();
+		walrcv_slot_snapshot(LogRepWorkerWalRcvConn, slotname, &options, origin_startpos);
+		RESUME_INTERRUPTS();
 	}
 
+	/*
+	 * Advance to the LSN got from walrcv_create_slot. This is WAL
+	 * logged for the purpose of recovery. Locks are to prevent the
+	 * replication origin from vanishing while advancing.
+	 *
+	 * Then setup replication origin tracking.
+	 */
+	LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+	replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
+					   true /* go backward */ , true /* WAL log */ );
+	UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+
 	/* Now do the initial data copy */
 	PushActiveSnapshot(GetTransactionSnapshot());
 	copy_table(rel);
@@ -1455,10 +1667,12 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	 * Update the persisted state to indicate the COPY phase is done; make it
 	 * visible to others.
 	 */
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   SUBREL_STATE_FINISHEDCOPY,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  SUBREL_STATE_FINISHEDCOPY,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 
 	CommitTransactionCommand();
 
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 79cda39445..032f402aed 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -385,6 +385,7 @@ static void stream_open_file(Oid subid, TransactionId xid,
 static void stream_write_change(char action, StringInfo s);
 static void stream_open_and_write_change(TransactionId xid, char action, StringInfo s);
 static void stream_close_file(void);
+static void stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
@@ -443,18 +444,26 @@ get_worker_name(void)
  * Form the origin name for the subscription.
  *
  * This is a common function for tablesync and other workers. Tablesync workers
- * must pass a valid relid. Other callers must pass relid = InvalidOid.
+ * must pass is_tablesync true so that origin name includes slot id.
  *
  * Return the name in the supplied buffer.
  */
 void
-ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
-								   char *originname, Size szoriginname)
+ReplicationOriginNameForLogicalRep(Oid suboid, char *originname,
+								   Size szoriginname, bool is_tablesync)
 {
-	if (OidIsValid(relid))
+	if (is_tablesync)
 	{
-		/* Replication origin name for tablesync workers. */
-		snprintf(originname, szoriginname, "pg_%u_%u", suboid, relid);
+		bool		is_null = true;
+
+		/*
+		 * Replication origin name for tablesync workers. First, look into the
+		 * catalog. If originname does not exist, then use the default name.
+		 */
+		GetSubscriptionRelOrigin(suboid, MyLogicalRepWorker->relid,
+								 originname, &is_null);
+		if (is_null)
+			snprintf(originname, szoriginname, "pg_%u_%lld", suboid, (long long) MyLogicalRepWorker->rep_slot_id);
 	}
 	else
 	{
@@ -3573,6 +3582,23 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can move to next table while running
+				 * process_syncing_tables_for_sync. Before we were able to
+				 * reuse tablesync workers, that
+				 * process_syncing_tables_for_sync call would exit the worker
+				 * instead of moving to next table. Now that tablesync workers
+				 * can be reused, we need to take care of memory contexts here
+				 * before moving to sync a table.
+				 */
+				if (MyLogicalRepWorker->move_to_next_rel)
+				{
+					MemoryContextResetAndDeleteChildren(ApplyMessageContext);
+					MemoryContextSwitchTo(TopMemoryContext);
+					return;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -3592,6 +3618,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+			if (MyLogicalRepWorker->move_to_next_rel)
+			{
+				endofstream = true;
+			}
 		}
 
 		/* Cleanup the memory. */
@@ -3694,8 +3724,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * If it's moving to next relation, this is a sync worker. Sync workers
+	 * end the streaming during process_syncing_tables_for_sync. Calling
+	 * endstreaming twice causes "no COPY in progress" errors.
+	 */
+	if (!MyLogicalRepWorker->move_to_next_rel)
+	{
+		/* All done */
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	}
 }
 
 /*
@@ -4277,6 +4315,56 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+ /* stream_build_options
+  * 	Build logical replication streaming options.
+  *
+  * This function sets streaming options including replication slot name
+  * and origin start position. Workers need these options for logical replication.
+  */
+static void
+stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4351,6 +4439,9 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 
 	/* allocate slot name in long-lived context */
 	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+
+	/* Keep the replication slot name used for this sync. */
+	MyLogicalRepWorker->slot_name = *myslotname;
 	pfree(syncslotname);
 }
 
@@ -4388,6 +4479,135 @@ start_apply(XLogRecPtr origin_startpos)
 	PG_END_TRY();
 }
 
+/*
+ * Runs the tablesync worker.
+ * It starts table sync. After successful sync,
+ * builds streaming options and starts streaming.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Set this to false for safety, in case we're already reusing the worker */
+	MyLogicalRepWorker->move_to_next_rel = false;
+
+	start_table_sync(origin_startpos, &slotname);
+
+	/*
+	 * Allocate the origin name in long-lived context for error context
+	 * message.
+	 */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   originname,
+									   originname_size,
+									   true);
+	CommitTransactionCommand();
+
+	set_apply_error_context_origin(originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/* Start normal logical streaming replication. */
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+}
+
+/*
+ * Runs the apply worker.
+ * It sets up replication origin, the streaming options
+ * and then starts streaming.
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
+{
+	/* This is the leader apply worker */
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+
+	slotname = MySubscription->slotname;
+
+	/*
+		* This shouldn't happen if the subscription is enabled, but guard
+		* against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+		* crash if slot is NULL.)
+		*/
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, originname,
+										originname_size, false);
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											MySubscription->name, &err);
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+		* We don't really use the output identify_system for anything but it
+		* does some initializations on the upstream so let's still call it.
+		*/
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+	{
+		/* Start streaming with two_phase enabled */
+		options->proto.logical.twophase = true;
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
+	}
+	else
+	{
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+	}
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+}
+
 /*
  * Common initialization for leader apply worker and parallel apply worker.
  *
@@ -4477,7 +4697,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	/* Attach to slot */
 	logicalrep_worker_attach(worker_slot);
@@ -4505,156 +4724,48 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
-	 * the subscription relation state.
+	 * the subscription relation state. Do this outside the loop to avoid
+	 * exceeding MAX_SYSCACHE_CALLBACKS
 	 */
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
-
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
-
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-
 	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 * The loop where worker does its job. It loops until the worker is not
+	 * reused.
 	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
-
-	if (!am_tablesync_worker())
+	while (MyLogicalRepWorker->is_first_run ||
+		   MyLogicalRepWorker->move_to_next_rel)
 	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
+		if (am_tablesync_worker())
 		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
+			/*
+			 * This is a tablesync worker. Start syncing tables before
+			 * starting the apply loop.
+			 */
+			run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 		}
 		else
 		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+			/* This is main apply worker */
+			run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 		}
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-						MySubscription->name,
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-						"?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
-
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+		/* Run the main loop. */
+		start_apply(origin_startpos);
 
+		if (MyLogicalRepWorker->move_to_next_rel)
+		{
+			StartTransactionCommand();
+			ereport(LOG,
+					(errmsg("logical replication table synchronization worker for subscription \"%s\" has moved to sync table \"%s\".",
+							MySubscription->name, get_rel_name(MyLogicalRepWorker->relid))));
+			CommitTransactionCommand();
+		}
+	}
 	proc_exit(0);
 }
 
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index b0f2a1705d..a0ee12e259 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -103,6 +103,9 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
 
 	/* Only publish data originating from the specified origin */
 	text		suborigin BKI_DEFAULT(LOGICALREP_ORIGIN_ANY);
+
+	/* The last used ID to create a replication slot for tablesync */
+	int64		sublastusedid BKI_DEFAULT(0);
 #endif
 } FormData_pg_subscription;
 
@@ -137,6 +140,8 @@ typedef struct Subscription
 	List	   *publications;	/* List of publication names to subscribe to */
 	char	   *origin;			/* Only publish data originating from the
 								 * specified origin */
+	int64		lastusedid;		/* Last used unique ID to create replication
+								 * slots in tablesync */
 } Subscription;
 
 /* Disallow streaming in-progress transactions. */
@@ -157,6 +162,7 @@ typedef struct Subscription
 extern Subscription *GetSubscription(Oid subid, bool missing_ok);
 extern void FreeSubscription(Subscription *sub);
 extern void DisableSubscription(Oid subid);
+extern void UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid);
 
 extern int	CountDBSubscriptions(Oid dbid);
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 60a2bcca23..a35d04cccd 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -44,6 +44,12 @@ CATALOG(pg_subscription_rel,6102,SubscriptionRelRelationId)
 											 * used for synchronization
 											 * coordination, or NULL if not
 											 * valid */
+	text		srrelslotname BKI_FORCE_NULL;	/* name of the replication
+												 * slot for relation in
+												 * subscription */
+	text		srreloriginname BKI_FORCE_NULL; /* origin name for relation in
+												 * subscription */
+
 #endif
 } FormData_pg_subscription_rel;
 
@@ -81,10 +87,17 @@ typedef struct SubscriptionRelState
 } SubscriptionRelState;
 
 extern void AddSubscriptionRelState(Oid subid, Oid relid, char state,
-									XLogRecPtr sublsn);
+									XLogRecPtr sublsn, char *relslotname, char *reloriginname);
 extern void UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 									   XLogRecPtr sublsn);
+extern void UpdateSubscriptionRel(Oid subid, Oid relid, char state,
+								  XLogRecPtr sublsn, char *relslotname, char *reloriginname);
+extern void UpdateSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *relslotname);
+
 extern char GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn);
+extern void GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname);
+extern void GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull);
+
 extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 
 extern bool HasSubscriptionRelations(Oid subid);
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 8872c80cdf..3547daaaec 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -219,8 +219,9 @@ extern bool InvalidateObsoleteReplicationSlots(XLogSegNo oldestSegno);
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
 extern bool ReplicationSlotName(int index, Name name);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, Size szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, int64 slotid, char *syncslotname, Size szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List *GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index db891eea8a..aee0d85f1d 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -35,6 +35,26 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/*
+	 * Indicates if worker is running for the first time or in reuse
+	 */
+	bool		is_first_run;
+
+	/*
+	 * Indicates if the sync worker created a replication slot or it reuses an
+	 * existing one created by another worker.
+	 */
+	bool		created_slot;
+
+	/*
+	 * Unique identifier for replication slot to be created by tablesnync
+	 * workers, if needed.
+	 */
+	int64		rep_slot_id;
+
+	/* Replication slot name used by the worker. */
+	char	   *slot_name;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
@@ -56,6 +76,12 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/*
+	 * Used to indicate whether sync worker will be reused for another
+	 * relation
+	 */
+	bool		move_to_next_rel;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction or
@@ -231,7 +257,8 @@ extern LogicalRepWorker *logicalrep_worker_find(Oid subid, Oid relid,
 extern List *logicalrep_workers_find(Oid subid, bool only_running);
 extern bool logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname,
 									 Oid userid, Oid relid,
-									 dsm_handle subworker_dsm);
+									 dsm_handle subworker_dsm,
+									 int64 slotid);
 extern void logicalrep_worker_stop(Oid subid, Oid relid);
 extern void logicalrep_pa_worker_stop(int slot_no, uint16 generation);
 extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
@@ -239,8 +266,8 @@ extern void logicalrep_worker_wakeup_ptr(LogicalRepWorker *worker);
 
 extern int	logicalrep_sync_worker_count(Oid subid);
 
-extern void ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
-											   char *originname, Size szoriginname);
+extern void ReplicationOriginNameForLogicalRep(Oid suboid, char *originname,
+											   Size szoriginname, bool is_tablesync);
 extern char *LogicalRepSyncTableStart(XLogRecPtr *origin_startpos);
 
 extern bool AllTablesyncsReady(void);
diff --git a/src/test/regress/expected/misc_sanity.out b/src/test/regress/expected/misc_sanity.out
index a57fd142a9..3d34a21421 100644
--- a/src/test/regress/expected/misc_sanity.out
+++ b/src/test/regress/expected/misc_sanity.out
@@ -47,20 +47,22 @@ WHERE c.oid < 16384 AND
       relkind = 'r' AND
       attstorage != 'p'
 ORDER BY 1, 2;
-         relname         |    attname    |   atttypid   
--------------------------+---------------+--------------
- pg_attribute            | attacl        | aclitem[]
- pg_attribute            | attfdwoptions | text[]
- pg_attribute            | attmissingval | anyarray
- pg_attribute            | attoptions    | text[]
- pg_class                | relacl        | aclitem[]
- pg_class                | reloptions    | text[]
- pg_class                | relpartbound  | pg_node_tree
- pg_index                | indexprs      | pg_node_tree
- pg_index                | indpred       | pg_node_tree
- pg_largeobject          | data          | bytea
- pg_largeobject_metadata | lomacl        | aclitem[]
-(11 rows)
+         relname         |     attname     |   atttypid   
+-------------------------+-----------------+--------------
+ pg_attribute            | attacl          | aclitem[]
+ pg_attribute            | attfdwoptions   | text[]
+ pg_attribute            | attmissingval   | anyarray
+ pg_attribute            | attoptions      | text[]
+ pg_class                | relacl          | aclitem[]
+ pg_class                | reloptions      | text[]
+ pg_class                | relpartbound    | pg_node_tree
+ pg_index                | indexprs        | pg_node_tree
+ pg_index                | indpred         | pg_node_tree
+ pg_largeobject          | data            | bytea
+ pg_largeobject_metadata | lomacl          | aclitem[]
+ pg_subscription_rel     | srreloriginname | text
+ pg_subscription_rel     | srrelslotname   | text
+(13 rows)
 
 -- system catalogs without primary keys
 --
-- 
2.25.1

#26shiy.fnst@fujitsu.com
shiy.fnst@fujitsu.com
In reply to: Melih Mutlu (#25)
1 attachment(s)
RE: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Wed, Jan 11, 2023 4:31 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi hackers,

Rebased the patch to resolve conflicts.

Thanks for your patch. Here are some comments.

0001 patch
============
1. walsender.c
+ /* Create a tuple to send consisten WAL location */

"consisten" should be "consistent" I think.

2. logical.c
+	if (need_full_snapshot)
+	{
+		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+		SpinLockAcquire(&slot->mutex);
+		slot->effective_catalog_xmin = xmin_horizon;
+		slot->data.catalog_xmin = xmin_horizon;
+		slot->effective_xmin = xmin_horizon;
+		SpinLockRelease(&slot->mutex);
+
+		xmin_horizon = GetOldestSafeDecodingTransactionId(!need_full_snapshot);
+		ReplicationSlotsComputeRequiredXmin(true);
+
+		LWLockRelease(ProcArrayLock);
+	}

It seems that we should first get the safe decoding xid, then inform the slot
machinery about the new limit, right? Otherwise the limit will be
InvalidTransactionId and that seems inconsistent with the comment.

3. doc/src/sgml/protocol.sgml
+       is used in the currenct transaction. This command is currently only supported
+       for logical replication.
+       slots.

We don't need to put "slots" in a new line.

0002 patch
============
1.
In pg_subscription_rel.h, I think the type of "srrelslotname" can be changed to
NameData, see "subslotname" in pg_subscription.h.

2.
+				 * Find the logical replication sync worker if exists store
+				 * the slot number for dropping associated replication slots
+				 * later.

Should we add comma after "if exists"?

3.
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();
+	\
+		return tablelist;
+}

Do we need the backslash?

4.
+	/*
+	 * Advance to the LSN got from walrcv_create_slot. This is WAL
+	 * logged for the purpose of recovery. Locks are to prevent the
+	 * replication origin from vanishing while advancing.

"walrcv_create_slot" should be changed to
"walrcv_create_slot/walrcv_slot_snapshot" I think.

5.
+			/* Replication drop might still exist. Try to drop */
+			replorigin_drop_by_name(originname, true, false);

Should "Replication drop" be "Replication origin"?

6.
I saw an assertion failure in the following case, could you please look into it?
The backtrace is attached.

-- pub
CREATE TABLE tbl1 (a int, b text);
CREATE TABLE tbl2 (a int primary key, b text);
create publication pub for table tbl1, tbl2;
insert into tbl1 values (1, 'a');
insert into tbl1 values (1, 'a');

-- sub
CREATE TABLE tbl1 (a int primary key, b text);
CREATE TABLE tbl2 (a int primary key, b text);
create subscription sub connection 'dbname=postgres port=5432' publication pub;

Subscriber log:
2023-01-17 14:47:10.054 CST [1980841] LOG: logical replication apply worker for subscription "sub" has started
2023-01-17 14:47:10.060 CST [1980843] LOG: logical replication table synchronization worker for subscription "sub", table "tbl1" has started
2023-01-17 14:47:10.070 CST [1980845] LOG: logical replication table synchronization worker for subscription "sub", table "tbl2" has started
2023-01-17 14:47:10.073 CST [1980843] ERROR: duplicate key value violates unique constraint "tbl1_pkey"
2023-01-17 14:47:10.073 CST [1980843] DETAIL: Key (a)=(1) already exists.
2023-01-17 14:47:10.073 CST [1980843] CONTEXT: COPY tbl1, line 2
2023-01-17 14:47:10.074 CST [1980821] LOG: background worker "logical replication worker" (PID 1980843) exited with exit code 1
2023-01-17 14:47:10.083 CST [1980845] LOG: logical replication table synchronization worker for subscription "sub", table "tbl2" has finished
2023-01-17 14:47:10.083 CST [1980845] LOG: logical replication table synchronization worker for subscription "sub" has moved to sync table "tbl1".
TRAP: failed Assert("node != InvalidRepOriginId"), File: "origin.c", Line: 892, PID: 1980845

Regards,
Shi yu

Attachments:

backtrace.txttext/plain; name=backtrace.txtDownload
#27wangw.fnst@fujitsu.com
wangw.fnst@fujitsu.com
In reply to: Melih Mutlu (#25)
RE: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Wed, Jan 11, 2023 4:31 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Rebased the patch to resolve conflicts.

Thanks for your patch set.

Here are some comments:

v3-0001* patch
===============

1. typedefs.list
I think we also need to add "walrcv_slot_snapshot_fn" to this file.

v7-0002* patch
===============
1. About function ReplicationOriginNameForLogicalRep()
Do we need to modify the API of this function? I think the original API could
also meet the current needs. Since this is not a static function, I think it
seems better to keep the original API if there is no reason. Please let me know
if I'm missing something.

-----

2. Comment atop the function GetSubscriptionRelReplicationSlot
+/*
+ * Get replication slot name of subscription table.
+ *
+ * Returns null if the subscription table does not have a replication slot.
+ */

Since this function always returns NULL, I think it would be better to say the
value in "slotname" here instead of the function's return value.

If you agree with this, please also kindly modify the comment atop the function
GetSubscriptionRelOrigin.

-----

3. typo
+			 * At this point, there shouldn't be any existing replication
+			 * origin wit the same name.

wit -> with

-----

4. In function CreateSubscription
+ values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(1);

I think it might be better to initialize this field to NULL or 0 here.
Because in the patch, we always ignore the initialized value when launching
the sync worker in the function process_syncing_tables_for_apply. And I think
we could document in pg-doc that this value means that no tables have been
synced yet.

-----

5. New member "created_slot" in structure LogicalRepWorker
+	/*
+	 * Indicates if the sync worker created a replication slot or it reuses an
+	 * existing one created by another worker.
+	 */
+	bool		created_slot;

I think the second half of the sentence looks inaccurate.
Because I think this flag could be false even when we reuse an existing slot
created by another worker. Assuming the first run for the worker tries to sync
a table which is synced by another sync worker before, and the relstate is set
to SUBREL_STATE_FINISHEDCOPY by another sync worker, I think this flag will not
be set to true. (see function LogicalRepSyncTableStart)

So, what if we simplify the description here and just say that this worker
already has it's default slot?

If I'm not missing something and you agree with this, please also kindly modify
the relevant comment atop the if-statement (!MyLogicalRepWorker->created_slot)
in the function LogicalRepSyncTableStart.

Regards,
Wang Wei

#28Melih Mutlu
m.melihmutlu@gmail.com
In reply to: wangw.fnst@fujitsu.com (#27)
2 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi,

Thanks for your review.
Attached updated versions of the patches.

wangw.fnst@fujitsu.com <wangw.fnst@fujitsu.com>, 17 Oca 2023 Sal, 14:15
tarihinde şunu yazdı:

On Wed, Jan 11, 2023 4:31 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:
v3-0001* patch
===============

1. typedefs.list
I think we also need to add "walrcv_slot_snapshot_fn" to this file.

Done.

v7-0002* patch
===============
1. About function ReplicationOriginNameForLogicalRep()
Do we need to modify the API of this function? I think the original API
could
also meet the current needs. Since this is not a static function, I think
it
seems better to keep the original API if there is no reason. Please let me
know
if I'm missing something.

You're right.
I still need to modify ReplicationOriginNameForLogicalRep. Origin names are
not tied to relations anymore, so their name doesn't need to
include relation id.
But I didn't really need to change the function signature. I reverted that
part of the change in the updated version of the patch.

2. Comment atop the function GetSubscriptionRelReplicationSlot

Done

3. typo
+                        * At this point, there shouldn't be any existing
replication
+                        * origin wit the same name.

Done.

4. In function CreateSubscription
+ values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(1);

I think it might be better to initialize this field to NULL or 0 here.
Because in the patch, we always ignore the initialized value when launching
the sync worker in the function process_syncing_tables_for_apply. And I
think
we could document in pg-doc that this value means that no tables have been
synced yet.

I changed it to start from 0 and added a line into the related doc to
indicate that 0 means that no table has been synced yet.

5. New member "created_slot" in structure LogicalRepWorker
+       /*
+        * Indicates if the sync worker created a replication slot or it
reuses an
+        * existing one created by another worker.
+        */
+       bool            created_slot;

I think the second half of the sentence looks inaccurate.
Because I think this flag could be false even when we reuse an existing
slot
created by another worker. Assuming the first run for the worker tries to
sync
a table which is synced by another sync worker before, and the relstate is
set
to SUBREL_STATE_FINISHEDCOPY by another sync worker, I think this flag
will not
be set to true. (see function LogicalRepSyncTableStart)

So, what if we simplify the description here and just say that this worker
already has it's default slot?

If I'm not missing something and you agree with this, please also kindly
modify
the relevant comment atop the if-statement
(!MyLogicalRepWorker->created_slot)
in the function LogicalRepSyncTableStart.

This "created_slot" indicates whether the current worker has created a
replication slot for its own use. If so, created_slot will be true,
otherwise false.
Let's say the tablesync worker has not created its own slot yet in its
previous runs or this is its first run. And the worker decides to reuse an
existing replication slot (which created by another tablesync worker). Then
created_slot is expected to be false. Because this particular
tablesync worker has not created its own slot yet in either of its runs.

In your example, the worker is in its first run and begin to sync a table
whose state is FINISHEDCOPY. If the table's state is FINISHEDCOPY then the
table should already have a replication slot created for its own sync
process. The worker will want to reuse that existing replication slot for
this particular table and it will not create a new replication slot. So
created_slot will be false, because the worker has not actually created any
replication slot yet.

Basically, created_slot is set to true only if "walrcv_create_slot" is
called by the tablesync worker any time during its lifetime. Otherwise,
it's possible that the worker can use existing replication slots for each
table it syncs. (e.g. if all the tables that the worker has synced were in
FINISHEDCOPY state, then the worker will not need to create a new slot).

Does it make sense now? Maybe I need to improve comments to make it clearer.

Best,
--
Melih Mutlu
Microsoft

Attachments:

v4-0001-Add-replication-protocol-cmd-to-create-a-snapshot.patchapplication/octet-stream; name=v4-0001-Add-replication-protocol-cmd-to-create-a-snapshot.patchDownload
From 1ce1967e48fd7a40a1344067ab14caa751b625fe Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 13 Oct 2022 17:05:45 +0300
Subject: [PATCH 1/2] Add replication protocol cmd to create a snapshot

Introduced REPLICATION_SLOT_SNAPSHOT to be able to create and use a
snapshot without creating a new replication slot, but by using an
existing slot.

REPLICATION_SLOT_SNAPSHOT simply does what CREATE_REPLICATION_SLOT does
without creating a new replication slot.

REPLICATION_SLOT_SNAPSHOT command imports the snapshot into the current
transaction and returns consistent_point. The changes earlier than the
consistent_point will be applied by importing the snapshot. All changes
later than the consistent_point will be available to be consumed from
the replication slot.

This is useful for reusing replication slots in logical replication.
Otherwise, tablesync workers cannot start from a consistent point to copy a relation and then apply changes by consuming from replication slot.
---
 doc/src/sgml/protocol.sgml                    | 32 ++++++
 .../libpqwalreceiver/libpqwalreceiver.c       | 69 ++++++++++++-
 src/backend/replication/logical/logical.c     | 39 +++++++-
 .../replication/logical/logicalfuncs.c        |  1 +
 src/backend/replication/repl_gram.y           | 18 +++-
 src/backend/replication/repl_scanner.l        |  2 +
 src/backend/replication/slotfuncs.c           |  1 +
 src/backend/replication/walsender.c           | 97 ++++++++++++++++++-
 src/include/nodes/replnodes.h                 | 11 +++
 src/include/replication/logical.h             |  1 +
 src/include/replication/walreceiver.h         | 13 +++
 src/tools/pgindent/typedefs.list              |  2 +
 12 files changed, 282 insertions(+), 4 deletions(-)

diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 93fc7167d4..93a3867996 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2613,6 +2613,38 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
      </listitem>
     </varlistentry>
 
+    <varlistentry id="protocol-replication-replication-slot-snapshot">
+     <term><literal>REPLICATION_SLOT_SNAPSHOT</literal> <replaceable class="parameter">slot_name</replaceable> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
+      <indexterm><primary>REPLICATION_SLOT_SNAPSHOT</primary></indexterm>
+     </term>
+     <listitem>
+      <para>
+       Creates a snapshot including all the changes from the replication slot until
+       the point at which the replication slot becomes consistent. Then the snapshot
+       is used in the currenct transaction. This command is currently only supported
+       for logical replication.
+       slots.
+      </para>
+
+      <para>
+       In response to this command, the server will return a one-row result set,
+       containing the following field:
+       <variablelist>
+        <varlistentry>
+         <term><literal>consistent_point</literal> (<type>text</type>)</term>
+         <listitem>
+          <para>
+           The WAL location at which the slot became consistent.  This is the
+           earliest location from which streaming can start on this replication
+           slot.
+          </para>
+         </listitem>
+        </varlistentry>
+       </variablelist>
+      </para>
+     </listitem>
+    </varlistentry>
+
     <varlistentry id="protocol-replication-base-backup" xreflabel="BASE_BACKUP">
      <term><literal>BASE_BACKUP</literal> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
       <indexterm><primary>BASE_BACKUP</primary></indexterm>
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index c40c6220db..9213fd2b1b 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -80,6 +80,8 @@ static WalRcvExecResult *libpqrcv_exec(WalReceiverConn *conn,
 									   const int nRetTypes,
 									   const Oid *retTypes);
 static void libpqrcv_disconnect(WalReceiverConn *conn);
+static void libpqrcv_slot_snapshot(WalReceiverConn *conn, char *slotname,
+								   const WalRcvStreamOptions *options, XLogRecPtr *lsn);
 
 static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_connect = libpqrcv_connect,
@@ -96,7 +98,8 @@ static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_create_slot = libpqrcv_create_slot,
 	.walrcv_get_backend_pid = libpqrcv_get_backend_pid,
 	.walrcv_exec = libpqrcv_exec,
-	.walrcv_disconnect = libpqrcv_disconnect
+	.walrcv_disconnect = libpqrcv_disconnect,
+	.walrcv_slot_snapshot = libpqrcv_slot_snapshot
 };
 
 /* Prototypes for private functions */
@@ -968,6 +971,70 @@ libpqrcv_create_slot(WalReceiverConn *conn, const char *slotname,
 	return snapshot;
 }
 
+/*
+ * TODO
+ */
+static void
+libpqrcv_slot_snapshot(WalReceiverConn *conn,
+					   char *slotname,
+					   const WalRcvStreamOptions *options,
+					   XLogRecPtr *lsn)
+{
+	StringInfoData cmd;
+	PGresult   *res;
+	char	   *pubnames_str;
+	List	   *pubnames;
+	char	   *pubnames_literal;
+
+	initStringInfo(&cmd);
+
+	/* Build the command. */
+	appendStringInfo(&cmd, "REPLICATION_SLOT_SNAPSHOT \"%s\"", slotname);
+	appendStringInfoString(&cmd, " (");
+	appendStringInfo(&cmd, " proto_version '%u'",
+					 options->proto.logical.proto_version);
+
+	/* Add publication names. */
+	pubnames = options->proto.logical.publication_names;
+	pubnames_str = stringlist_to_identifierstr(conn->streamConn, pubnames);
+	if (!pubnames_str)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	pubnames_literal = PQescapeLiteral(conn->streamConn, pubnames_str,
+									   strlen(pubnames_str));
+	if (!pubnames_literal)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	appendStringInfo(&cmd, ", publication_names %s", pubnames_literal);
+	PQfreemem(pubnames_literal);
+	pfree(pubnames_str);
+
+	appendStringInfoString(&cmd, " )");
+
+	/* Execute the command. */
+	res = libpqrcv_PQexec(conn->streamConn, cmd.data);
+	pfree(cmd.data);
+
+	if (PQresultStatus(res) != PGRES_TUPLES_OK)
+	{
+		PQclear(res);
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg("Could not create a snapshot by replication slot \"%s\": %s",
+						slotname, pchomp(PQerrorMessage(conn->streamConn)))));
+	}
+
+	if (lsn)
+		*lsn = DatumGetLSN(DirectFunctionCall1Coll(pg_lsn_in, InvalidOid,
+												   CStringGetDatum(PQgetvalue(res, 0, 0))));
+
+	PQclear(res);
+}
+
 /*
  * Return PID of remote backend process.
  */
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 1a58dd7649..d3bffd4cb2 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -461,6 +461,10 @@ CreateInitDecodingContext(const char *plugin,
  * fast_forward
  *		bypass the generation of logical changes.
  *
+ * need_full_snapshot
+ * 		if true, create a snapshot able to read all tables,
+ * 		otherwise do not create any snapshot.
+ *
  * xl_routine
  *		XLogReaderRoutine used by underlying xlogreader
  *
@@ -479,6 +483,7 @@ LogicalDecodingContext *
 CreateDecodingContext(XLogRecPtr start_lsn,
 					  List *output_plugin_options,
 					  bool fast_forward,
+					  bool need_full_snapshot,
 					  XLogReaderRoutine *xl_routine,
 					  LogicalOutputPluginWriterPrepareWrite prepare_write,
 					  LogicalOutputPluginWriterWrite do_write,
@@ -487,6 +492,7 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 	LogicalDecodingContext *ctx;
 	ReplicationSlot *slot;
 	MemoryContext old_context;
+	TransactionId xmin_horizon = InvalidTransactionId;
 
 	/* shorter lines... */
 	slot = MyReplicationSlot;
@@ -533,8 +539,39 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 		start_lsn = slot->data.confirmed_flush;
 	}
 
+
+	/*
+	 * We need to determine a safe xmin horizon to start decoding from if we
+	 * want to create a snapshot too. Otherwise we would end up with a
+	 * snapshot that cannot be imported since xmin value from the snapshot may
+	 * be less than the oldest safe xmin. To avoid this call
+	 * GetOldestSafeDecodingTransactionId() to return a safe xmin value, which
+	 * can be used while exporting/importing the snapshot.
+	 *
+	 * So we have to acquire the ProcArrayLock to prevent computation of new
+	 * xmin horizons by other backends, get the safe decoding xid, and inform
+	 * the slot machinery about the new limit. Once that's done the
+	 * ProcArrayLock can be released as the slot machinery now is protecting
+	 * against vacuum.
+	 */
+	if (need_full_snapshot)
+	{
+		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+		SpinLockAcquire(&slot->mutex);
+		slot->effective_catalog_xmin = xmin_horizon;
+		slot->data.catalog_xmin = xmin_horizon;
+		slot->effective_xmin = xmin_horizon;
+		SpinLockRelease(&slot->mutex);
+
+		xmin_horizon = GetOldestSafeDecodingTransactionId(!need_full_snapshot);
+		ReplicationSlotsComputeRequiredXmin(true);
+
+		LWLockRelease(ProcArrayLock);
+	}
+
 	ctx = StartupDecodingContext(output_plugin_options,
-								 start_lsn, InvalidTransactionId, false,
+								 start_lsn, xmin_horizon, need_full_snapshot,
 								 fast_forward, xl_routine, prepare_write,
 								 do_write, update_progress);
 
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index fa1b641a2b..1191c70eb0 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -208,6 +208,7 @@ pg_logical_slot_get_changes_guts(FunctionCallInfo fcinfo, bool confirm, bool bin
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									options,
 									false,
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/repl_gram.y b/src/backend/replication/repl_gram.y
index 0c874e33cf..e5f0235d1e 100644
--- a/src/backend/replication/repl_gram.y
+++ b/src/backend/replication/repl_gram.y
@@ -65,6 +65,7 @@ Node *replication_parse_result;
 %token K_CREATE_REPLICATION_SLOT
 %token K_DROP_REPLICATION_SLOT
 %token K_TIMELINE_HISTORY
+%token K_REPLICATION_SLOT_SNAPSHOT
 %token K_WAIT
 %token K_TIMELINE
 %token K_PHYSICAL
@@ -80,7 +81,7 @@ Node *replication_parse_result;
 %type <node>	command
 %type <node>	base_backup start_replication start_logical_replication
 				create_replication_slot drop_replication_slot identify_system
-				read_replication_slot timeline_history show
+				read_replication_slot timeline_history show replication_slot_snapshot
 %type <list>	generic_option_list
 %type <defelt>	generic_option
 %type <uintval>	opt_timeline
@@ -114,6 +115,7 @@ command:
 			| read_replication_slot
 			| timeline_history
 			| show
+			| replication_slot_snapshot
 			;
 
 /*
@@ -307,6 +309,19 @@ timeline_history:
 				}
 			;
 
+/*
+ * REPLICATION_SLOT_SNAPSHOT %s options
+ */
+replication_slot_snapshot:
+			K_REPLICATION_SLOT_SNAPSHOT var_name plugin_options
+				{
+					ReplicationSlotSnapshotCmd *n = makeNode(ReplicationSlotSnapshotCmd);
+					n->slotname = $2;
+					n->options = $3;
+					$$ = (Node *) n;
+				}
+			;
+
 opt_physical:
 			K_PHYSICAL
 			| /* EMPTY */
@@ -400,6 +415,7 @@ ident_or_keyword:
 			| K_CREATE_REPLICATION_SLOT	{ $$ = "create_replication_slot"; }
 			| K_DROP_REPLICATION_SLOT		{ $$ = "drop_replication_slot"; }
 			| K_TIMELINE_HISTORY			{ $$ = "timeline_history"; }
+			| K_REPLICATION_SLOT_SNAPSHOT	{ $$ = "replication_slot_snapshot"; }
 			| K_WAIT						{ $$ = "wait"; }
 			| K_TIMELINE					{ $$ = "timeline"; }
 			| K_PHYSICAL					{ $$ = "physical"; }
diff --git a/src/backend/replication/repl_scanner.l b/src/backend/replication/repl_scanner.l
index cb467ca46f..1988a6203b 100644
--- a/src/backend/replication/repl_scanner.l
+++ b/src/backend/replication/repl_scanner.l
@@ -126,6 +126,7 @@ START_REPLICATION	{ return K_START_REPLICATION; }
 CREATE_REPLICATION_SLOT		{ return K_CREATE_REPLICATION_SLOT; }
 DROP_REPLICATION_SLOT		{ return K_DROP_REPLICATION_SLOT; }
 TIMELINE_HISTORY	{ return K_TIMELINE_HISTORY; }
+REPLICATION_SLOT_SNAPSHOT	{ return K_REPLICATION_SLOT_SNAPSHOT; }
 PHYSICAL			{ return K_PHYSICAL; }
 RESERVE_WAL			{ return K_RESERVE_WAL; }
 LOGICAL				{ return K_LOGICAL; }
@@ -303,6 +304,7 @@ replication_scanner_is_replication_command(void)
 		case K_DROP_REPLICATION_SLOT:
 		case K_READ_REPLICATION_SLOT:
 		case K_TIMELINE_HISTORY:
+		case K_REPLICATION_SLOT_SNAPSHOT:
 		case K_SHOW:
 			/* Yes; push back the first token so we can parse later. */
 			repl_pushed_back_token = first_token;
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 2f3c964824..b3ae11b2c8 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -478,6 +478,7 @@ pg_logical_replication_slot_advance(XLogRecPtr moveto)
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									NIL,
 									true,	/* fast_forward */
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 4ed3747e3f..7acfa88fad 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -238,6 +238,7 @@ static void CreateReplicationSlot(CreateReplicationSlotCmd *cmd);
 static void DropReplicationSlot(DropReplicationSlotCmd *cmd);
 static void StartReplication(StartReplicationCmd *cmd);
 static void StartLogicalReplication(StartReplicationCmd *cmd);
+static void ReplicationSlotSnapshot(ReplicationSlotSnapshotCmd *cmd);
 static void ProcessStandbyMessage(void);
 static void ProcessStandbyReplyMessage(void);
 static void ProcessStandbyHSFeedbackMessage(void);
@@ -1280,7 +1281,7 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	 * are reported early.
 	 */
 	logical_decoding_ctx =
-		CreateDecodingContext(cmd->startpoint, cmd->options, false,
+		CreateDecodingContext(cmd->startpoint, cmd->options, false, false,
 							  XL_ROUTINE(.page_read = logical_read_xlog_page,
 										 .segment_open = WalSndSegmentOpen,
 										 .segment_close = wal_segment_close),
@@ -1332,6 +1333,91 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	EndCommand(&qc, DestRemote, false);
 }
 
+/*
+ * Create a snapshot from an existing replication slot.
+ */
+static void
+ReplicationSlotSnapshot(ReplicationSlotSnapshotCmd *cmd)
+{
+	Snapshot	snap;
+	LogicalDecodingContext *ctx;
+	char		xloc[MAXFNAMELEN];
+	DestReceiver *dest;
+	TupOutputState *tstate;
+	TupleDesc	tupdesc;
+	Datum		values[1];
+	bool		nulls[1] = {0};
+
+	Assert(!MyReplicationSlot);
+
+	if (!IsTransactionBlock())
+		ereport(ERROR,
+				(errmsg("%s must be called inside a transaction",
+						"REPLICATION_SLOT_SNAPSHOT ...")));
+
+	if (XactIsoLevel != XACT_REPEATABLE_READ)
+		ereport(ERROR,
+				(errmsg("%s must be called in REPEATABLE READ isolation mode transaction",
+						"REPLICATION_SLOT_SNAPSHOT ...")));
+
+	if (FirstSnapshotSet)
+		ereport(ERROR,
+				(errmsg("%s must be called before any query",
+						"REPLICATION_SLOT_SNAPSHOT ...")));
+
+	if (IsSubTransaction())
+		ereport(ERROR,
+				(errmsg("%s must not be called in a subtransaction",
+						"REPLICATION_SLOT_SNAPSHOT ...")));
+
+	ReplicationSlotAcquire(cmd->slotname, false);
+
+	ctx = CreateDecodingContext(MyReplicationSlot->data.restart_lsn,
+								cmd->options,
+								false,
+								true,
+								XL_ROUTINE(.page_read = logical_read_xlog_page,
+										   .segment_open = WalSndSegmentOpen,
+										   .segment_close = wal_segment_close),
+								WalSndPrepareWrite, WalSndWriteData,
+								WalSndUpdateProgress);
+
+	/*
+	 * Signal that we don't need the timeout mechanism. We're just creating
+	 * the replication slot and don't yet accept feedback messages or send
+	 * keepalives. As we possibly need to wait for further WAL the walsender
+	 * would otherwise possibly be killed too soon.
+	 */
+	last_reply_timestamp = 0;
+
+	/* build initial snapshot, might take a while */
+	DecodingContextFindStartpoint(ctx);
+
+	snap = SnapBuildInitialSnapshot(ctx->snapshot_builder);
+	RestoreTransactionSnapshot(snap, MyProc);
+
+	/* Don't need the decoding context anymore */
+	FreeDecodingContext(ctx);
+
+	/* Create a tuple to send consisten WAL location */
+	snprintf(xloc, sizeof(xloc), "%X/%X",
+			 LSN_FORMAT_ARGS(MyReplicationSlot->data.confirmed_flush));
+
+	dest = CreateDestReceiver(DestRemoteSimple);
+	tupdesc = CreateTemplateTupleDesc(1);
+	TupleDescInitBuiltinEntry(tupdesc, (AttrNumber) 1, "consistent_point",
+							  TEXTOID, -1, 0);
+	tstate = begin_tup_output_tupdesc(dest, tupdesc, &TTSOpsVirtual);
+
+	/* consistent wal location */
+	values[0] = CStringGetTextDatum(xloc);
+
+	do_tup_output(tstate, values, nulls);
+	end_tup_output(tstate);
+
+	ReplicationSlotRelease();
+}
+
 /*
  * LogicalDecodingContext 'prepare_write' callback.
  *
@@ -1860,6 +1946,15 @@ exec_replication_command(const char *cmd_string)
 			}
 			break;
 
+		case T_ReplicationSlotSnapshotCmd:
+			{
+				cmdtag = "REPLICATION_SLOT_SNAPSHOT";
+				set_ps_display(cmdtag);
+				ReplicationSlotSnapshot((ReplicationSlotSnapshotCmd *) cmd_node);
+				EndReplicationCommand(cmdtag);
+				break;
+			}
+
 		default:
 			elog(ERROR, "unrecognized replication command node tag: %u",
 				 cmd_node->type);
diff --git a/src/include/nodes/replnodes.h b/src/include/nodes/replnodes.h
index 4321ba8f86..44a4580671 100644
--- a/src/include/nodes/replnodes.h
+++ b/src/include/nodes/replnodes.h
@@ -108,4 +108,15 @@ typedef struct TimeLineHistoryCmd
 	TimeLineID	timeline;
 } TimeLineHistoryCmd;
 
+/* ----------------------
+ *		REPLICATION_SLOT_SNAPSHOT command
+ * ----------------------
+ */
+typedef struct ReplicationSlotSnapshotCmd
+{
+	NodeTag		type;
+	char	   *slotname;
+	List	   *options;
+} ReplicationSlotSnapshotCmd;
+
 #endif							/* REPLNODES_H */
diff --git a/src/include/replication/logical.h b/src/include/replication/logical.h
index 5f49554ea0..6535786a0e 100644
--- a/src/include/replication/logical.h
+++ b/src/include/replication/logical.h
@@ -125,6 +125,7 @@ extern LogicalDecodingContext *CreateInitDecodingContext(const char *plugin,
 extern LogicalDecodingContext *CreateDecodingContext(XLogRecPtr start_lsn,
 													 List *output_plugin_options,
 													 bool fast_forward,
+													 bool need_full_snapshot,
 													 XLogReaderRoutine *xl_routine,
 													 LogicalOutputPluginWriterPrepareWrite prepare_write,
 													 LogicalOutputPluginWriterWrite do_write,
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index decffe352d..bd11f9f31e 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -384,6 +384,16 @@ typedef WalRcvExecResult *(*walrcv_exec_fn) (WalReceiverConn *conn,
  */
 typedef void (*walrcv_disconnect_fn) (WalReceiverConn *conn);
 
+/*
+ * walrcv_slot_snapshot_fn
+ *
+ * Create a snapshot by an existing replication slot
+ */
+typedef void (*walrcv_slot_snapshot_fn) (WalReceiverConn *conn,
+										 char *slotname,
+										 const WalRcvStreamOptions *options,
+										 XLogRecPtr *lsn);
+
 typedef struct WalReceiverFunctionsType
 {
 	walrcv_connect_fn walrcv_connect;
@@ -401,6 +411,7 @@ typedef struct WalReceiverFunctionsType
 	walrcv_get_backend_pid_fn walrcv_get_backend_pid;
 	walrcv_exec_fn walrcv_exec;
 	walrcv_disconnect_fn walrcv_disconnect;
+	walrcv_slot_snapshot_fn walrcv_slot_snapshot;
 } WalReceiverFunctionsType;
 
 extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
@@ -435,6 +446,8 @@ extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
 	WalReceiverFunctions->walrcv_exec(conn, exec, nRetTypes, retTypes)
 #define walrcv_disconnect(conn) \
 	WalReceiverFunctions->walrcv_disconnect(conn)
+#define walrcv_slot_snapshot(conn, slotname, options, lsn) \
+	WalReceiverFunctions->walrcv_slot_snapshot(conn, slotname, options, lsn)
 
 static inline void
 walrcv_clear_result(WalRcvExecResult *walres)
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 24510ac29e..fa38ea91e2 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2321,6 +2321,7 @@ ReplicationSlotCtlData
 ReplicationSlotOnDisk
 ReplicationSlotPersistency
 ReplicationSlotPersistentData
+ReplicationSlotSnapshotCmd
 ReplicationState
 ReplicationStateCtl
 ReplicationStateOnDisk
@@ -3794,6 +3795,7 @@ walrcv_receive_fn
 walrcv_send_fn
 walrcv_server_version_fn
 walrcv_startstreaming_fn
+walrcv_slot_snapshot_fn
 wchar2mb_with_len_converter
 wchar_t
 win32_deadchild_waitinfo
-- 
2.25.1

v8-0002-Reuse-Logical-Replication-Background-worker.patchapplication/octet-stream; name=v8-0002-Reuse-Logical-Replication-Background-worker.patchDownload
From 6b26b1f21e410b2c649da401b33e8b1789a39ab7 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH 2/2] Reuse Logical Replication Background worker

This commit allows tablesync workers to move to another table that needs synchronization,
when they're done with the current table in tablesync phase of Logical Replication.

Before this commit, tablesync workers were capable of syncing only one
relation. A new worker, replication slot and origin were needed for each
relation in the subscription.

Now, tablesync workers are not only limited with one relation and can move to another relation and reuse existing
replication slots and origins

This reduces the overhead of launching/killing a new background worker for each relation.
By reusing tablesync workers, replication slots and origins created for tablesync can be reused as well.
Removing the burden of creating/dropping replication slot/origin improves tablesync speed significantly especially for empty or small tables.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/catalogs.sgml                    |  30 ++
 src/backend/catalog/pg_subscription.c         | 284 +++++++++++-
 src/backend/commands/subscriptioncmds.c       | 226 ++++++----
 .../replication/logical/applyparallelworker.c |   3 +-
 src/backend/replication/logical/launcher.c    |   9 +-
 src/backend/replication/logical/tablesync.c   | 426 +++++++++++++-----
 src/backend/replication/logical/worker.c      | 388 ++++++++++------
 src/include/catalog/pg_subscription.h         |   6 +
 src/include/catalog/pg_subscription_rel.h     |  15 +-
 src/include/replication/slot.h                |   3 +-
 src/include/replication/worker_internal.h     |  29 +-
 src/test/regress/expected/misc_sanity.out     |  30 +-
 12 files changed, 1097 insertions(+), 352 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index c1e4048054..eb8d4e1005 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8002,6 +8002,18 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        origin.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>sublastusedid</structfield> <type>int8</type>
+      </para>
+      <para>
+       The last used ID for tablesync workers. This ID is used to
+       create replication slots. The last used ID needs to be stored
+       to make logical replication can safely proceed after any interruption.
+       If sublastusedid is 0, then no table has been synced yet.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
@@ -8086,6 +8098,24 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        otherwise null
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srrelslotname</structfield> <type>text</type>
+      </para>
+      <para>
+       Replication slot name that is used for synchronization of relation
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srreloriginname</structfield> <type>text</type>
+      </para>
+      <para>
+       Origin name that is used for tracking synchronization of relation
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index a56ae311c3..f8dcad16ee 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -114,6 +114,14 @@ GetSubscription(Oid subid, bool missing_ok)
 	Assert(!isnull);
 	sub->origin = TextDatumGetCString(datum);
 
+	/* Get last used id */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
+							tup,
+							Anum_pg_subscription_sublastusedid,
+							&isnull);
+	Assert(!isnull);
+	sub->lastusedid = DatumGetInt64(datum);
+
 	ReleaseSysCache(tup);
 
 	return sub;
@@ -205,6 +213,44 @@ DisableSubscription(Oid subid)
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update the last used replication slot ID for the given subscription.
+ */
+void
+UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid)
+{
+	Relation	rel;
+	bool		nulls[Natts_pg_subscription];
+	bool		replaces[Natts_pg_subscription];
+	Datum		values[Natts_pg_subscription];
+	HeapTuple	tup;
+
+	/* Look up the subscription in the catalog */
+	rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+	tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	/* Form a new tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	replaces[Anum_pg_subscription_sublastusedid - 1] = true;
+	values[Anum_pg_subscription_sublastusedid- 1] = Int64GetDatum(lastusedid);
+
+	/* Update the catalog */
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+	heap_freetuple(tup);
+
+	table_close(rel, NoLock);
+}
+
 /*
  * Convert text array to list of strings.
  *
@@ -234,7 +280,7 @@ textarray_to_stringlist(ArrayType *textarray)
  */
 void
 AddSubscriptionRelState(Oid subid, Oid relid, char state,
-						XLogRecPtr sublsn)
+						XLogRecPtr sublsn, char *relslotname, char *reloriginname)
 {
 	Relation	rel;
 	HeapTuple	tup;
@@ -263,6 +309,14 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
 	else
 		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] = CStringGetTextDatum(relslotname);
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] = CStringGetTextDatum(reloriginname);
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -275,6 +329,58 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Internal function to modify columns for relation state update
+ */
+static void
+UpdateSubscriptionRelState_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char state,
+									XLogRecPtr sublsn)
+{
+	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
+	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+
+	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (sublsn != InvalidXLogRecPtr)
+		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
+	else
+		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication slot update
+ */
+static void
+UpdateSubscriptionRelReplicationSlot_internal(Datum *values,
+											bool *nulls,
+											bool *replaces,
+											char *relslotname)
+{
+	replaces[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] = CStringGetTextDatum(relslotname);
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication origin update
+ */
+static void
+UpdateSubscriptionRelOrigin_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char *reloriginname)
+{
+	replaces[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] = CStringGetTextDatum(reloriginname);
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+}
+
 /*
  * Update the state of a subscription table.
  */
@@ -305,14 +411,48 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	memset(nulls, false, sizeof(nulls));
 	memset(replaces, false, sizeof(replaces));
 
-	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
-	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
 
-	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
-	if (sublsn != InvalidXLogRecPtr)
-		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
-	else
-		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update the replication slot name of a subscription table.
+ */
+void
+UpdateSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *relslotname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
 
 	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
 							replaces);
@@ -324,6 +464,134 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update replication slot name, origin name and state of
+ * a subscription table in one transaction.
+ */
+void
+UpdateSubscriptionRel(Oid subid,
+					  Oid relid,
+					  char state,
+					  XLogRecPtr sublsn,
+					  char *relslotname,
+					  char *reloriginname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
+	UpdateSubscriptionRelOrigin_internal(values, nulls, replaces, reloriginname);
+
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Get origin name of subscription table.
+ *
+ * reloriginname's value has the replication origin name if the origin exists.
+ */
+void
+GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*originname;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srreloriginname, isnull);
+	if (!*isnull)
+	{
+		originname = TextDatumGetCString(d);
+		memcpy(reloriginname, originname, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
+/*
+ * Get replication slot name of subscription table.
+ *
+ * slotname's value has the replication slot name if the subscription has any.
+ */
+void
+GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*relrepslot;
+	bool		isnull;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srrelslotname, &isnull);
+	if (!isnull)
+	{
+		relrepslot = TextDatumGetCString(d);
+		memcpy(slotname, relrepslot, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
 /*
  * Get state of subscription table.
  *
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 464db6d247..1c4033fc79 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -649,6 +649,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 		publicationListToArray(publications);
 	values[Anum_pg_subscription_suborigin - 1] =
 		CStringGetTextDatum(opts.origin);
+	values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(0);
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -709,7 +710,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 										 rv->schemaname, rv->relname);
 
 				AddSubscriptionRelState(subid, relid, table_state,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 			}
 
 			/*
@@ -799,6 +800,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	} SubRemoveRels;
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -876,7 +879,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				AddSubscriptionRelState(sub->oid, relid,
 										copy_data ? SUBREL_STATE_INIT : SUBREL_STATE_READY,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 				ereport(DEBUG1,
 						(errmsg_internal("table \"%s.%s\" added to subscription \"%s\"",
 										 rv->schemaname, rv->relname, sub->name)));
@@ -900,6 +903,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				char		state;
 				XLogRecPtr	statelsn;
+				char		slotname[NAMEDATALEN] = {0};
 
 				/*
 				 * Lock pg_subscription_rel with AccessExclusiveLock to
@@ -926,7 +930,29 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/*
+				 * Find the logical replication sync worker if exists store
+				 * the slot number for dropping associated replication slots
+				 * later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot_name);
+				}
+				else
+				{
+					/*
+					 * Sync of this relation might be failed in an earlier
+					 * attempt, but the replication slot might still exist.
+					 */
+					GetSubscriptionRelReplicationSlot(sub->oid, relid, slotname);
+					if (strlen(slotname) > 0)
+						sub_remove_slots = lappend(sub_remove_slots, slotname);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
@@ -960,31 +986,24 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
-		 * the database operations we won't be able to rollback dropped slots.
+		 * Drop the replication slots associated with tablesync workers for
+		 * removed tables. This has to be at the end because otherwise if
+		 * there is an error while doing the database operations we won't be
+		 * able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			memcpy(syncslotname, lfirst(lc), sizeof(NAMEDATALEN));
+
+			/*
+			 * There is no certainty, maybe the slot does not exist yet. Also,
+			 * if we fail after removing some of the slots, next time, it will
+			 * again try to drop already dropped slots and fail. For these
+			 * reasons, we allow missing_ok = true for the drop.
+			 */
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1384,6 +1403,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	char	   *subname;
 	char	   *conninfo;
 	char	   *slotname;
+	int64		lastusedid;
 	List	   *subworkers;
 	ListCell   *lc;
 	char		originname[NAMEDATALEN];
@@ -1455,6 +1475,14 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	else
 		slotname = NULL;
 
+	/* Get the last used identifier by the subscription */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID, tup,
+							Anum_pg_subscription_sublastusedid, &isnull);
+	if (!isnull)
+		lastusedid = DatumGetInt64(datum);
+	else
+		lastusedid = 0;
+
 	/*
 	 * Since dropping a replication slot is not transactional, the replication
 	 * slot stays dropped even if the transaction rolls back.  So we cannot
@@ -1504,6 +1532,8 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	}
 	list_free(subworkers);
 
+	rstates = GetSubscriptionRelations(subid, true);
+
 	/*
 	 * Remove the no-longer-useful entry in the launcher's table of apply
 	 * worker start times.
@@ -1515,36 +1545,26 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	ApplyLauncherForgetWorkerStartTime(subid);
 
 	/*
-	 * Cleanup of tablesync replication origins.
-	 *
-	 * Any READY-state relations would already have dealt with clean-ups.
+	 * Cleanup of tablesync replication origins associated with the
+	 * subscription, if exists. Try to drop origins by creating all origin
+	 * names created for this subscription.
 	 *
 	 * Note that the state can't change because we have already stopped both
 	 * the apply and tablesync workers and they can't restart because of
 	 * exclusive lock on the subscription.
+	 *
+	 * XXX: This can be handled better instead of looping through all possible
 	 */
-	rstates = GetSubscriptionRelations(subid, true);
-	foreach(lc, rstates)
+	for (int64 i = 1; i <= lastusedid; i++)
 	{
-		SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-		Oid			relid = rstate->relid;
-
-		/* Only cleanup resources of tablesync workers */
-		if (!OidIsValid(relid))
-			continue;
+		char		originname_to_drop[NAMEDATALEN] = {0};
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * It is possible that the origin is not yet created for tablesync
-		 * worker so passing missing_ok = true. This can happen for the states
-		 * before SUBREL_STATE_FINISHEDCOPY.
-		 */
-		ReplicationOriginNameForLogicalRep(subid, relid, originname,
-										   sizeof(originname));
-		replorigin_drop_by_name(originname, true, false);
+		snprintf(originname_to_drop, sizeof(originname_to_drop), "pg_%u_%lld", subid, (long long) i);
+		/* missin_ok = true, since the origin might be already dropped. */
+		replorigin_drop_by_name(originname_to_drop, true, false);
 	}
 
+
 	/* Clean up dependencies */
 	deleteSharedDependencyRecordsFor(SubscriptionRelationId, subid, 0);
 
@@ -1596,39 +1616,17 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
+		List	   *slots = NULL;
 
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
 
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char	   *syncslotname = (char *) lfirst(lc);
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1651,6 +1649,71 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * Get the replication slot names associated with the subscription.
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok)
+{
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+					 " FROM pg_replication_slots"
+					 " WHERE slot_name LIKE 'pg_%i_sync_%%';",
+					 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+					errmsg("not tuple returned."));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();
+	\
+		return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -2005,6 +2068,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -2015,18 +2079,20 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
-			char		syncslotname[NAMEDATALEN] = {0};
-
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
-				 syncslotname);
+				 worker->slot_name);
 		}
 	}
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 3579e704fe..1cab625dfb 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -440,7 +440,8 @@ pa_launch_parallel_worker(void)
 										MySubscription->name,
 										MyLogicalRepWorker->userid,
 										InvalidOid,
-										dsm_segment_handle(winfo->dsm_seg));
+										dsm_segment_handle(winfo->dsm_seg),
+										0);
 
 	if (launched)
 	{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 564bffe5ca..b21951bc8f 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -304,7 +304,7 @@ logicalrep_workers_find(Oid subid, bool only_running)
  */
 bool
 logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
-						 Oid relid, dsm_handle subworker_dsm)
+						 Oid relid, dsm_handle subworker_dsm, int64 slotid)
 {
 	BackgroundWorker bgw;
 	BackgroundWorkerHandle *bgw_handle;
@@ -429,7 +429,11 @@ retry:
 	/* Prepare the worker slot. */
 	worker->launch_time = now;
 	worker->in_use = true;
+	worker->is_first_run = true;
 	worker->generation++;
+	worker->created_slot = false;
+	worker->rep_slot_id = slotid;
+	worker->slot_name = (char *) palloc(NAMEDATALEN);
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -437,6 +441,7 @@ retry:
 	worker->relid = relid;
 	worker->relstate = SUBREL_STATE_UNKNOWN;
 	worker->relstate_lsn = InvalidXLogRecPtr;
+	worker->move_to_next_rel = false;
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
@@ -1155,7 +1160,7 @@ ApplyLauncherMain(Datum main_arg)
 				ApplyLauncherSetWorkerStartTime(sub->oid, now);
 				logicalrep_worker_launch(sub->dbid, sub->oid, sub->name,
 										 sub->owner, InvalidOid,
-										 DSM_HANDLE_INVALID);
+										 DSM_HANDLE_INVALID, 0);
 			}
 			else
 			{
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 4647837b82..821351198d 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -126,12 +126,8 @@ static bool FetchTableStates(bool *started_tx);
 
 static StringInfo copybuf = NULL;
 
-/*
- * Exit routine for synchronization worker.
- */
 static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+clean_sync_worker(void)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -143,18 +139,28 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
-	/* And flush all writes. */
-	XLogFlush(GetXLogWriteRecPtr());
-
-	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
-	CommitTransactionCommand();
+	/*
+	 * Disconnect from publisher. Otherwise reused sync workers causes
+	 * exceeding max_wal_senders
+	 */
+	walrcv_disconnect(LogRepWorkerWalRcvConn);
+	LogRepWorkerWalRcvConn = NULL;
 
 	/* Find the leader apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
+}
+
+/*
+ * Exit routine for synchronization worker.
+ */
+static void
+pg_attribute_noreturn()
+finish_sync_worker(void)
+{
+	clean_sync_worker();
+
+	/* And flush all writes. */
+	XLogFlush(GetXLogWriteRecPtr());
 
 	/* Stop gracefully */
 	proc_exit(0);
@@ -284,6 +290,10 @@ invalidate_syncing_table_states(Datum arg, int cacheid, uint32 hashvalue)
 static void
 process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 {
+	List	   *rstates;
+	SubscriptionRelState *rstate;
+	ListCell   *lc;
+
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_CATCHUP &&
@@ -292,6 +302,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		TimeLineID	tli;
 		char		syncslotname[NAMEDATALEN] = {0};
 		char		originname[NAMEDATALEN] = {0};
+		bool		is_streaming_ended = false;
 
 		MyLogicalRepWorker->relstate = SUBREL_STATE_SYNCDONE;
 		MyLogicalRepWorker->relstate_lsn = current_lsn;
@@ -308,40 +319,29 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 								   MyLogicalRepWorker->relid,
 								   MyLogicalRepWorker->relstate,
 								   MyLogicalRepWorker->relstate_lsn);
+		CommitTransactionCommand();
 
 		/*
-		 * End streaming so that LogRepWorkerWalRcvConn can be used to drop
-		 * the slot.
-		 */
-		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
-
-		/*
-		 * Cleanup the tablesync slot.
+		 * Cleanup the tablesync slot. If the slot name used by this worker is
+		 * different from the default slot name for the worker, this means the
+		 * current table had started to being synchronized by another worker
+		 * and replication slot. And this worker is reusing a replication slot
+		 * from a previous attempt. We do not need that replication slot
+		 * anymore.
 		 *
 		 * This has to be done after updating the state because otherwise if
 		 * there is an error while doing the database operations we won't be
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
+										MyLogicalRepWorker->rep_slot_id,
 										syncslotname,
 										sizeof(syncslotname));
 
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
-		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
-
-		CommitTransactionCommand();
-		pgstat_report_stat(false);
-
-		/*
-		 * Start a new transaction to clean up the tablesync origin tracking.
-		 * This transaction will be ended within the finish_sync_worker().
-		 * Now, even, if we fail to remove this here, the apply worker will
-		 * ensure to clean it up afterward.
+		 * We are safe to drop the replication trackin origin after this
+		 * point. Now, even, if we fail to remove this here, the apply worker
+		 * will ensure to clean it up afterward.
 		 *
 		 * We need to do this after the table state is set to SYNCDONE.
 		 * Otherwise, if an error occurs while performing the database
@@ -350,34 +350,125 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 * have been cleared before restart. So, the restarted worker will use
 		 * invalid replication progress state resulting in replay of
 		 * transactions that have already been applied.
+		 *
+		 * Firstly reset the origin session to remove the ownership of the
+		 * slot. This is needed to allow the origin to be dropped or reused
+		 * later.
 		 */
+		replorigin_session_reset();
+		replorigin_session_origin = InvalidRepOriginId;
+		replorigin_session_origin_lsn = InvalidXLogRecPtr;
+		replorigin_session_origin_timestamp = 0;
+
 		StartTransactionCommand();
+		if (MyLogicalRepWorker->slot_name && strcmp(syncslotname, MyLogicalRepWorker->slot_name) != 0)
+		{
+			/*
+			 * End streaming so that LogRepWorkerWalRcvConn can be used to
+			 * drop the slot.
+			 */
+			walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+			is_streaming_ended = true;
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
+
+			ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+											   MyLogicalRepWorker->relid,
+											   originname,
+											   sizeof(originname));
 
+			/* Drop replication origin */
+			replorigin_drop_by_name(originname, true, false);
+		}
+
+		/*
+		 * We are safe to remove persisted replication slot and origin data,
+		 * since it's already in SYNCDONE state. They will not be needed
+		 * anymore.
+		 */
+		UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+							  MyLogicalRepWorker->relid,
+							  MyLogicalRepWorker->relstate,
+							  MyLogicalRepWorker->relstate_lsn,
+							  NULL,
+							  NULL);
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid))));
+
+		CommitTransactionCommand();
+		pgstat_report_stat(false);
+
+		StartTransactionCommand();
+
+		/*
+		 * This should return the default origin name for the worker. Even if
+		 * the worker used a different origin for this table, it should be
+		 * dropped and removed from the catalog so far.
+		 */
 		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
 										   MyLogicalRepWorker->relid,
 										   originname,
 										   sizeof(originname));
 
 		/*
-		 * Resetting the origin session removes the ownership of the slot.
-		 * This is needed to allow the origin to be dropped.
+		 * Check if any table whose relation state is still INIT. If a table
+		 * in INIT state is found, the worker will not be finished, it will be
+		 * reused instead.
 		 */
-		replorigin_session_reset();
-		replorigin_session_origin = InvalidRepOriginId;
-		replorigin_session_origin_lsn = InvalidXLogRecPtr;
-		replorigin_session_origin_timestamp = 0;
+		rstates = GetSubscriptionRelations(MySubscription->oid, true);
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * There is a chance that the user is concurrently performing refresh
-		 * for the subscription where we remove the table state and its origin
-		 * or the apply worker would have removed this origin. So passing
-		 * missing_ok = true.
-		 */
-		replorigin_drop_by_name(originname, true, false);
+		foreach(lc, rstates)
+		{
+			rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
+			memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
 
-		finish_sync_worker();
+			/*
+			 * Pick the table for the next run if there is not another worker
+			 * already picked that table.
+			 */
+			LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+			if (rstate->state != SUBREL_STATE_SYNCDONE &&
+				!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+			{
+				/* Update worker state for the next table */
+				MyLogicalRepWorker->is_first_run = false;
+				MyLogicalRepWorker->relid = rstate->relid;
+				MyLogicalRepWorker->relstate = rstate->state;
+				MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+				MyLogicalRepWorker->move_to_next_rel = true;
+				LWLockRelease(LogicalRepWorkerLock);
+				break;
+			}
+			LWLockRelease(LogicalRepWorkerLock);
+		}
+
+		/* Cleanup before next run or ending the worker. */
+		if (!MyLogicalRepWorker->move_to_next_rel)
+		{
+			/*
+			 * It is important to give an error if we are unable to drop the
+			 * slot, otherwise, it won't be dropped till the corresponding
+			 * subscription is dropped. So passing missing_ok = false.
+			 */
+			if (MyLogicalRepWorker->created_slot)
+			{
+				/* End streaming if it's not already ended. */
+				if (!is_streaming_ended)
+					walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+				ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
+			}
+
+			/* Drop replication origin before exiting. */
+			replorigin_drop_by_name(originname, true, false);
+
+			finish_sync_worker();
+		}
+		else
+		{
+			clean_sync_worker();
+		}
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -464,6 +555,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 			if (current_lsn >= rstate->lsn)
 			{
 				char		originname[NAMEDATALEN];
+				bool		is_origin_null = true;
 
 				rstate->state = SUBREL_STATE_READY;
 				rstate->lsn = current_lsn;
@@ -484,18 +576,27 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				 * error while dropping we won't restart it to drop the
 				 * origin. So passing missing_ok = true.
 				 */
-				ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-												   rstate->relid,
-												   originname,
-												   sizeof(originname));
-				replorigin_drop_by_name(originname, true, false);
+				GetSubscriptionRelOrigin(MyLogicalRepWorker->subid,
+										 rstate->relid, originname,
+										 &is_origin_null);
+
+				if (!is_origin_null)
+				{
+					replorigin_drop_by_name(originname, true, false);
+				}
 
 				/*
 				 * Update the state to READY only after the origin cleanup.
 				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-										   rstate->relid, rstate->state,
-										   rstate->lsn);
+				UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+									  rstate->relid,
+									  rstate->state,
+									  rstate->lsn,
+									  NULL,
+									  NULL);
+
+				CommitTransactionCommand();
+				started_tx = false;
 			}
 		}
 		else
@@ -584,12 +685,22 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						TimestampDifferenceExceeds(hentry->last_start_time, now,
 												   wal_retrieve_retry_interval))
 					{
+						if (IsTransactionState())
+							CommitTransactionCommand();
+						StartTransactionCommand();
+						started_tx = true;
+
+						MySubscription->lastusedid++;
+						UpdateSubscriptionLastSlotId(MyLogicalRepWorker->subid,
+													 MySubscription->lastusedid);
+
 						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
 												 MySubscription->oid,
 												 MySubscription->name,
 												 MyLogicalRepWorker->userid,
 												 rstate->relid,
-												 DSM_HANDLE_INVALID);
+												 DSM_HANDLE_INVALID,
+												 MySubscription->lastusedid);
 						hentry->last_start_time = now;
 					}
 				}
@@ -1198,8 +1309,8 @@ copy_table(Relation rel)
  * The name must not exceed NAMEDATALEN - 1 because of remote node constraints
  * on slot name length. We append system_identifier to avoid slot_name
  * collision with subscriptions in other clusters. With the current scheme
- * pg_%u_sync_%u_UINT64_FORMAT (3 + 10 + 6 + 10 + 20 + '\0'), the maximum
- * length of slot_name will be 50.
+ * pg_%u_sync_%lu_UINT64_FORMAT (3 + 10 + 6 + 20 + 20 + '\0'), the maximum
+ * length of slot_name will be 45.
  *
  * The returned slot name is stored in the supplied buffer (syncslotname) with
  * the given size.
@@ -1210,11 +1321,11 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
+ReplicationSlotNameForTablesync(Oid suboid, int64 slotid,
 								char *syncslotname, Size szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%lld_" UINT64_FORMAT, suboid,
+			(long long) slotid, GetSystemIdentifier());
 }
 
 /*
@@ -1237,6 +1348,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	WalRcvExecResult *res;
 	char		originname[NAMEDATALEN];
 	RepOriginId originid;
+	char	   *prev_slotname;
 
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
@@ -1265,7 +1377,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
+									MyLogicalRepWorker->rep_slot_id,
 									slotname,
 									NAMEDATALEN);
 
@@ -1285,12 +1397,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC ||
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY);
 
+	/*
+	 * See if tablesync of the current relation has been started with another
+	 * replication slot.
+	 *
+	 * Read previous slot name from the catalog, if exists.
+	 */
+	prev_slotname = (char *) palloc0(NAMEDATALEN);
+	StartTransactionCommand();
+	GetSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+									  MyLogicalRepWorker->relid,
+									  prev_slotname);
+
 	/* Assign the origin tracking record name. */
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   sizeof(originname));
 
+	CommitTransactionCommand();
+
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC)
 	{
 		/*
@@ -1304,10 +1430,48 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		 * breakdown then it wouldn't have succeeded so trying it next time
 		 * seems like a better bet.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, slotname, true);
+		if (strlen(prev_slotname) > 0)
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);
+
+			StartTransactionCommand();
+			/* Replication drop might still exist. Try to drop */
+			replorigin_drop_by_name(originname, true, false);
+
+			/*
+			 * Remove replication slot and origin name from the relation's
+			 * catalog record
+			 */
+			UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+								  MyLogicalRepWorker->relid,
+								  MyLogicalRepWorker->relstate,
+								  MyLogicalRepWorker->relstate_lsn,
+								  NULL,
+								  NULL);
+			CommitTransactionCommand();
+		}
 	}
 	else if (MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY)
 	{
+		/*
+		 * At this point, the table that is currently being synchronized
+		 * should have its replication slot name filled in the catalog. The
+		 * tablesync process was started with another sync worker and
+		 * replication slot. We need to continue using the same replication
+		 * slot in this worker too.
+		 */
+		if (strlen(prev_slotname) == 0)
+		{
+			elog(ERROR, "Replication slot could not be found for relation %u",
+				 MyLogicalRepWorker->relid);
+		}
+
+		/*
+		 * Proceed with the correct replication slot. Use previously created
+		 * replication slot to sync this table.
+		 */
+		slotname = prev_slotname;
+
 		/*
 		 * The COPY phase was previously done, but tablesync then crashed
 		 * before it was able to finish normally.
@@ -1328,6 +1492,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		goto copy_table_done;
 	}
 
+	/* Preparing for table copy operation */
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 	MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
 	MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
@@ -1335,10 +1500,12 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   MyLogicalRepWorker->relstate,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  MyLogicalRepWorker->relstate,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 	CommitTransactionCommand();
 	pgstat_report_stat(true);
 
@@ -1377,6 +1544,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						GetUserNameFromId(GetUserId(), true),
 						RelationGetRelationName(rel))));
 
+
 	/*
 	 * Start a transaction in the remote node in REPEATABLE READ mode.  This
 	 * ensures that both the replication slot we create (see below) and the
@@ -1392,55 +1560,99 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						res->err)));
 	walrcv_clear_result(res);
 
+	originid = replorigin_by_name(originname, true);
+
 	/*
 	 * Create a new permanent logical decoding slot. This slot will be used
 	 * for the catchup phase after COPY is done, so tell it to use the
 	 * snapshot to make the final data consistent.
 	 *
+	 * Replication slot will only be created if either this is the first run
+	 * of the worker or we're not using a previous replication slot.
+	 *
 	 * Prevent cancel/die interrupts while creating slot here because it is
 	 * possible that before the server finishes this command, a concurrent
 	 * drop subscription happens which would complete without removing this
 	 * slot leading to a dangling slot on the server.
+	 *
 	 */
-	HOLD_INTERRUPTS();
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-	RESUME_INTERRUPTS();
-
-	/*
-	 * Setup replication origin tracking. The purpose of doing this before the
-	 * copy is to avoid doing the copy again due to any error in setting up
-	 * origin tracking.
-	 */
-	originid = replorigin_by_name(originname, true);
-	if (!OidIsValid(originid))
+	if (!MyLogicalRepWorker->created_slot)
 	{
+		HOLD_INTERRUPTS();
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						   slotname, false /* permanent */ , false /* two_phase */ ,
+						   CRS_USE_SNAPSHOT, origin_startpos);
+		RESUME_INTERRUPTS();
+
 		/*
-		 * Origin tracking does not exist, so create it now.
-		 *
-		 * Then advance to the LSN got from walrcv_create_slot. This is WAL
-		 * logged for the purpose of recovery. Locks are to prevent the
-		 * replication origin from vanishing while advancing.
+		 * Remember that we created the slot so that we will not try to create
+		 * it again.
 		 */
-		originid = replorigin_create(originname);
-
-		LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
-		replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
-						   true /* go backward */ , true /* WAL log */ );
-		UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->created_slot = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
+		/*
+		 * Setup replication origin tracking. The purpose of doing this before
+		 * the copy is to avoid doing the copy again due to any error in
+		 * setting up origin tracking.
+		 */
+		if (!OidIsValid(originid))
+		{
+			/*
+			 * Origin tracking does not exist, so create it now.
+			 */
+			originid = replorigin_create(originname);
+		}
+		else
+		{
+			/*
+			 * At this point, there shouldn't be any existing replication
+			 * origin with the same name.
+			 */
+			ereport(ERROR,
+					(errcode(ERRCODE_DUPLICATE_OBJECT),
+					 errmsg("replication origin \"%s\" already exists",
+							originname)));
+		}
 	}
 	else
 	{
-		ereport(ERROR,
-				(errcode(ERRCODE_DUPLICATE_OBJECT),
-				 errmsg("replication origin \"%s\" already exists",
-						originname)));
+		/*
+		 * Do not create a new replication slot, reuse the existing one
+		 * instead. Use a new snapshot for the replication slot to ensure that
+		 * tablesync and apply proceses are consistent with each other.
+		 */
+		WalRcvStreamOptions options;
+		int			server_version;
+
+		server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+		options.proto.logical.proto_version =
+			server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+			server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+			LOGICALREP_PROTO_VERSION_NUM;
+		options.proto.logical.publication_names = MySubscription->publications;
+
+		HOLD_INTERRUPTS();
+		walrcv_slot_snapshot(LogRepWorkerWalRcvConn, slotname, &options, origin_startpos);
+		RESUME_INTERRUPTS();
 	}
 
+	/*
+	 * Advance to the LSN got from walrcv_create_slot. This is WAL
+	 * logged for the purpose of recovery. Locks are to prevent the
+	 * replication origin from vanishing while advancing.
+	 *
+	 * Then setup replication origin tracking.
+	 */
+	LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+	replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
+					   true /* go backward */ , true /* WAL log */ );
+	UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+
 	/* Now do the initial data copy */
 	PushActiveSnapshot(GetTransactionSnapshot());
 	copy_table(rel);
@@ -1463,10 +1675,12 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	 * Update the persisted state to indicate the COPY phase is done; make it
 	 * visible to others.
 	 */
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   SUBREL_STATE_FINISHEDCOPY,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  SUBREL_STATE_FINISHEDCOPY,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 
 	CommitTransactionCommand();
 
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index cfb2ab6248..c12924e4da 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -388,6 +388,7 @@ static void stream_open_file(Oid subid, TransactionId xid,
 static void stream_write_change(char action, StringInfo s);
 static void stream_open_and_write_change(TransactionId xid, char action, StringInfo s);
 static void stream_close_file(void);
+static void stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
@@ -456,13 +457,22 @@ ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 {
 	if (OidIsValid(relid))
 	{
-		/* Replication origin name for tablesync workers. */
-		snprintf(originname, szoriginname, "pg_%u_%u", suboid, relid);
+		bool		is_null = true;
+
+		/*
+		 * Replication origin name for tablesync workers. First, look into the
+		 * catalog. If originname does not exist, then use the default name.
+		 */
+		GetSubscriptionRelOrigin(suboid, relid,
+								 originname, &is_null);
+		if (is_null)
+			snprintf(originname, szoriginname, "pg_%u_%lld", suboid, (long long) MyLogicalRepWorker->rep_slot_id);
 	}
 	else
 	{
 		/* Replication origin name for non-tablesync workers. */
 		snprintf(originname, szoriginname, "pg_%u", suboid);
+		elog(LOG, "apply worker originname %s", originname);
 	}
 }
 
@@ -3576,6 +3586,23 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can move to next table while running
+				 * process_syncing_tables_for_sync. Before we were able to
+				 * reuse tablesync workers, that
+				 * process_syncing_tables_for_sync call would exit the worker
+				 * instead of moving to next table. Now that tablesync workers
+				 * can be reused, we need to take care of memory contexts here
+				 * before moving to sync a table.
+				 */
+				if (MyLogicalRepWorker->move_to_next_rel)
+				{
+					MemoryContextResetAndDeleteChildren(ApplyMessageContext);
+					MemoryContextSwitchTo(TopMemoryContext);
+					return;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -3595,6 +3622,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+			if (MyLogicalRepWorker->move_to_next_rel)
+			{
+				endofstream = true;
+			}
 		}
 
 		/* Cleanup the memory. */
@@ -3697,8 +3728,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * If it's moving to next relation, this is a sync worker. Sync workers
+	 * end the streaming during process_syncing_tables_for_sync. Calling
+	 * endstreaming twice causes "no COPY in progress" errors.
+	 */
+	if (!MyLogicalRepWorker->move_to_next_rel)
+	{
+		/* All done */
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	}
 }
 
 /*
@@ -4282,6 +4321,56 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+ /* stream_build_options
+  * 	Build logical replication streaming options.
+  *
+  * This function sets streaming options including replication slot name
+  * and origin start position. Workers need these options for logical replication.
+  */
+static void
+stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4356,6 +4445,9 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 
 	/* allocate slot name in long-lived context */
 	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+
+	/* Keep the replication slot name used for this sync. */
+	MyLogicalRepWorker->slot_name = *myslotname;
 	pfree(syncslotname);
 }
 
@@ -4393,6 +4485,135 @@ start_apply(XLogRecPtr origin_startpos)
 	PG_END_TRY();
 }
 
+/*
+ * Runs the tablesync worker.
+ * It starts table sync. After successful sync,
+ * builds streaming options and starts streaming.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Set this to false for safety, in case we're already reusing the worker */
+	MyLogicalRepWorker->move_to_next_rel = false;
+
+	start_table_sync(origin_startpos, &slotname);
+
+	/*
+	 * Allocate the origin name in long-lived context for error context
+	 * message.
+	 */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   originname_size);
+	CommitTransactionCommand();
+
+	set_apply_error_context_origin(originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/* Start normal logical streaming replication. */
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+}
+
+/*
+ * Runs the apply worker.
+ * It sets up replication origin, the streaming options
+ * and then starts streaming.
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
+{
+	/* This is the leader apply worker */
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+
+	slotname = MySubscription->slotname;
+
+	/*
+		* This shouldn't happen if the subscription is enabled, but guard
+		* against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+		* crash if slot is NULL.)
+		*/
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											MySubscription->name, &err);
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+		* We don't really use the output identify_system for anything but it
+		* does some initializations on the upstream so let's still call it.
+		*/
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+	{
+		/* Start streaming with two_phase enabled */
+		options->proto.logical.twophase = true;
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
+	}
+	else
+	{
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+	}
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+}
+
 /*
  * Common initialization for leader apply worker and parallel apply worker.
  *
@@ -4485,7 +4706,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	/* Attach to slot */
 	logicalrep_worker_attach(worker_slot);
@@ -4513,156 +4733,48 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
-	 * the subscription relation state.
+	 * the subscription relation state. Do this outside the loop to avoid
+	 * exceeding MAX_SYSCACHE_CALLBACKS
 	 */
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
-
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
-
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-
 	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 * The loop where worker does its job. It loops until the worker is not
+	 * reused.
 	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
-
-	if (!am_tablesync_worker())
+	while (MyLogicalRepWorker->is_first_run ||
+		   MyLogicalRepWorker->move_to_next_rel)
 	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
+		if (am_tablesync_worker())
 		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
+			/*
+			 * This is a tablesync worker. Start syncing tables before
+			 * starting the apply loop.
+			 */
+			run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 		}
 		else
 		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+			/* This is main apply worker */
+			run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 		}
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-						MySubscription->name,
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-						"?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
-
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+		/* Run the main loop. */
+		start_apply(origin_startpos);
 
+		if (MyLogicalRepWorker->move_to_next_rel)
+		{
+			StartTransactionCommand();
+			ereport(LOG,
+					(errmsg("logical replication table synchronization worker for subscription \"%s\" has moved to sync table \"%s\".",
+							MySubscription->name, get_rel_name(MyLogicalRepWorker->relid))));
+			CommitTransactionCommand();
+		}
+	}
 	proc_exit(0);
 }
 
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index b0f2a1705d..a0ee12e259 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -103,6 +103,9 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
 
 	/* Only publish data originating from the specified origin */
 	text		suborigin BKI_DEFAULT(LOGICALREP_ORIGIN_ANY);
+
+	/* The last used ID to create a replication slot for tablesync */
+	int64		sublastusedid BKI_DEFAULT(0);
 #endif
 } FormData_pg_subscription;
 
@@ -137,6 +140,8 @@ typedef struct Subscription
 	List	   *publications;	/* List of publication names to subscribe to */
 	char	   *origin;			/* Only publish data originating from the
 								 * specified origin */
+	int64		lastusedid;		/* Last used unique ID to create replication
+								 * slots in tablesync */
 } Subscription;
 
 /* Disallow streaming in-progress transactions. */
@@ -157,6 +162,7 @@ typedef struct Subscription
 extern Subscription *GetSubscription(Oid subid, bool missing_ok);
 extern void FreeSubscription(Subscription *sub);
 extern void DisableSubscription(Oid subid);
+extern void UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid);
 
 extern int	CountDBSubscriptions(Oid dbid);
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 60a2bcca23..a35d04cccd 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -44,6 +44,12 @@ CATALOG(pg_subscription_rel,6102,SubscriptionRelRelationId)
 											 * used for synchronization
 											 * coordination, or NULL if not
 											 * valid */
+	text		srrelslotname BKI_FORCE_NULL;	/* name of the replication
+												 * slot for relation in
+												 * subscription */
+	text		srreloriginname BKI_FORCE_NULL; /* origin name for relation in
+												 * subscription */
+
 #endif
 } FormData_pg_subscription_rel;
 
@@ -81,10 +87,17 @@ typedef struct SubscriptionRelState
 } SubscriptionRelState;
 
 extern void AddSubscriptionRelState(Oid subid, Oid relid, char state,
-									XLogRecPtr sublsn);
+									XLogRecPtr sublsn, char *relslotname, char *reloriginname);
 extern void UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 									   XLogRecPtr sublsn);
+extern void UpdateSubscriptionRel(Oid subid, Oid relid, char state,
+								  XLogRecPtr sublsn, char *relslotname, char *reloriginname);
+extern void UpdateSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *relslotname);
+
 extern char GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn);
+extern void GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname);
+extern void GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull);
+
 extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 
 extern bool HasSubscriptionRelations(Oid subid);
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 8872c80cdf..3547daaaec 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -219,8 +219,9 @@ extern bool InvalidateObsoleteReplicationSlots(XLogSegNo oldestSegno);
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
 extern bool ReplicationSlotName(int index, Name name);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, Size szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, int64 slotid, char *syncslotname, Size szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List *GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index dc87a4edd1..5f4b7b1f7c 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -35,6 +35,26 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/*
+	 * Indicates if worker is running for the first time or in reuse
+	 */
+	bool		is_first_run;
+
+	/*
+	 * Indicates if the sync worker created a replication slot or it reuses an
+	 * existing one created by another worker.
+	 */
+	bool		created_slot;
+
+	/*
+	 * Unique identifier for replication slot to be created by tablesnync
+	 * workers, if needed.
+	 */
+	int64		rep_slot_id;
+
+	/* Replication slot name used by the worker. */
+	char	   *slot_name;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
@@ -56,6 +76,12 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/*
+	 * Used to indicate whether sync worker will be reused for another
+	 * relation
+	 */
+	bool		move_to_next_rel;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction or
@@ -231,7 +257,8 @@ extern LogicalRepWorker *logicalrep_worker_find(Oid subid, Oid relid,
 extern List *logicalrep_workers_find(Oid subid, bool only_running);
 extern bool logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname,
 									 Oid userid, Oid relid,
-									 dsm_handle subworker_dsm);
+									 dsm_handle subworker_dsm,
+									 int64 slotid);
 extern void logicalrep_worker_stop(Oid subid, Oid relid);
 extern void logicalrep_pa_worker_stop(int slot_no, uint16 generation);
 extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/test/regress/expected/misc_sanity.out b/src/test/regress/expected/misc_sanity.out
index a57fd142a9..3d34a21421 100644
--- a/src/test/regress/expected/misc_sanity.out
+++ b/src/test/regress/expected/misc_sanity.out
@@ -47,20 +47,22 @@ WHERE c.oid < 16384 AND
       relkind = 'r' AND
       attstorage != 'p'
 ORDER BY 1, 2;
-         relname         |    attname    |   atttypid   
--------------------------+---------------+--------------
- pg_attribute            | attacl        | aclitem[]
- pg_attribute            | attfdwoptions | text[]
- pg_attribute            | attmissingval | anyarray
- pg_attribute            | attoptions    | text[]
- pg_class                | relacl        | aclitem[]
- pg_class                | reloptions    | text[]
- pg_class                | relpartbound  | pg_node_tree
- pg_index                | indexprs      | pg_node_tree
- pg_index                | indpred       | pg_node_tree
- pg_largeobject          | data          | bytea
- pg_largeobject_metadata | lomacl        | aclitem[]
-(11 rows)
+         relname         |     attname     |   atttypid   
+-------------------------+-----------------+--------------
+ pg_attribute            | attacl          | aclitem[]
+ pg_attribute            | attfdwoptions   | text[]
+ pg_attribute            | attmissingval   | anyarray
+ pg_attribute            | attoptions      | text[]
+ pg_class                | relacl          | aclitem[]
+ pg_class                | reloptions      | text[]
+ pg_class                | relpartbound    | pg_node_tree
+ pg_index                | indexprs        | pg_node_tree
+ pg_index                | indpred         | pg_node_tree
+ pg_largeobject          | data            | bytea
+ pg_largeobject_metadata | lomacl          | aclitem[]
+ pg_subscription_rel     | srreloriginname | text
+ pg_subscription_rel     | srrelslotname   | text
+(13 rows)
 
 -- system catalogs without primary keys
 --
-- 
2.25.1

#29shveta malik
shveta.malik@gmail.com
In reply to: Melih Mutlu (#28)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Mon, Jan 23, 2023 at 6:30 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi,

Thanks for your review.
Attached updated versions of the patches.

Hello,
I am still in the process of reviewing the patch, before that I tried
to run below test:

--publisher
create table tab1(id int , name varchar);
create table tab2(id int primary key , name varchar);
create table tab3(id int primary key , name varchar);
Insert into tab1 values(10, 'a');
Insert into tab1 values(20, 'b');
Insert into tab1 values(30, 'c');

Insert into tab2 values(10, 'a');
Insert into tab2 values(20, 'b');
Insert into tab2 values(30, 'c');

Insert into tab3 values(10, 'a');
Insert into tab3 values(20, 'b');
Insert into tab3 values(30, 'c');

create publication mypub for table tab1, tab2, tab3;

--subscriber
create table tab1(id int , name varchar);
create table tab2(id int primary key , name varchar);
create table tab3(id int primary key , name varchar);
create subscription mysub connection 'dbname=postgres host=localhost
user=shveta port=5432' publication mypub;

--I see initial data copied, but new catalog columns srrelslotname
and srreloriginname are not updated:
postgres=# select sublastusedid from pg_subscription;
sublastusedid
---------------
2

postgres=# select * from pg_subscription_rel;
srsubid | srrelid | srsubstate | srsublsn | srrelslotname | srreloriginname
---------+---------+------------+-----------+---------------+-----------------
16409 | 16384 | r | 0/15219E0 | |
16409 | 16389 | r | 0/15219E0 | |
16409 | 16396 | r | 0/15219E0 | |

When are these supposed to be updated? I thought the slotname created
will be updated here. Am I missing something here?

Also the v8 patch does not apply on HEAD, giving merge conflicts.

thanks
Shveta

#30Melih Mutlu
m.melihmutlu@gmail.com
In reply to: shveta malik (#29)
1 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Shveta,

Thanks for reviewing.

shveta malik <shveta.malik@gmail.com>, 25 Oca 2023 Çar, 16:02 tarihinde
şunu yazdı:

On Mon, Jan 23, 2023 at 6:30 PM Melih Mutlu <m.melihmutlu@gmail.com>
wrote:
--I see initial data copied, but new catalog columns srrelslotname
and srreloriginname are not updated:
postgres=# select sublastusedid from pg_subscription;
sublastusedid
---------------
2

postgres=# select * from pg_subscription_rel;
srsubid | srrelid | srsubstate | srsublsn | srrelslotname |
srreloriginname

---------+---------+------------+-----------+---------------+-----------------
16409 | 16384 | r | 0/15219E0 | |
16409 | 16389 | r | 0/15219E0 | |
16409 | 16396 | r | 0/15219E0 | |

When are these supposed to be updated? I thought the slotname created
will be updated here. Am I missing something here?

If a relation is currently being synced by a tablesync worker and uses a
replication slot/origin for that operation, then srrelslotname and
srreloriginname fields will have values.
When a relation is done with its replication slot/origin, their info gets
removed from related catalog row, so that slot/origin can be reused for
another table or dropped if not needed anymore.
In your case, all relations are in READY state so it's expected that
srrelslotname and srreloriginname are empty. READY relations do not need a
replication slot/origin anymore.

Tables are probably synced so quickly that you're missing the moments when
a tablesync worker copies a relation and stores its rep. slot/origin in the
catalog.
If initial sync is long enough, then you should be able to see the columns
get updated. I follow [1]publisher: SELECT 'CREATE TABLE table_'||i||'(i int);' FROM generate_series(1, 100) g(i) \gexec SELECT 'INSERT INTO table_'||i||' SELECT x FROM generate_series(1, 10000) x' FROM generate_series(1, 100) g(i) \gexec CREATE PUBLICATION mypub FOR ALL TABLES; to make it longer and test if the patch really
updates the catalog.

Also the v8 patch does not apply on HEAD, giving merge conflicts.

Rebased and resolved conflicts. Please check the new version

---
[1]: publisher: SELECT 'CREATE TABLE table_'||i||'(i int);' FROM generate_series(1, 100) g(i) \gexec SELECT 'INSERT INTO table_'||i||' SELECT x FROM generate_series(1, 10000) x' FROM generate_series(1, 100) g(i) \gexec CREATE PUBLICATION mypub FOR ALL TABLES;
publisher:
SELECT 'CREATE TABLE table_'||i||'(i int);' FROM generate_series(1, 100)
g(i) \gexec
SELECT 'INSERT INTO table_'||i||' SELECT x FROM generate_series(1, 10000)
x' FROM generate_series(1, 100) g(i) \gexec
CREATE PUBLICATION mypub FOR ALL TABLES;

subscriber:
SELECT 'CREATE TABLE table_'||i||'(i int);' FROM generate_series(1, 100)
g(i) \gexec
CREATE SUBSCRIPTION mysub CONNECTION 'dbname=postgres port=5432 '
PUBLICATION mypub;
select * from pg_subscription_rel where srrelslotname <> ''; \watch 0.5

Thanks,
--
Melih Mutlu
Microsoft

Attachments:

v9-0002-Reuse-Logical-Replication-Background-worker.patchapplication/octet-stream; name=v9-0002-Reuse-Logical-Replication-Background-worker.patchDownload
From 757e519b98805ace371e5c3f972979d2b85ffd0c Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH 2/2] Reuse Logical Replication Background worker

This commit allows tablesync workers to move to another table that needs synchronization,
when they're done with the current table in tablesync phase of Logical Replication.

Before this commit, tablesync workers were capable of syncing only one
relation. A new worker, replication slot and origin were needed for each
relation in the subscription.

Now, tablesync workers are not only limited with one relation and can move to another relation and reuse existing
replication slots and origins

This reduces the overhead of launching/killing a new background worker for each relation.
By reusing tablesync workers, replication slots and origins created for tablesync can be reused as well.
Removing the burden of creating/dropping replication slot/origin improves tablesync speed significantly especially for empty or small tables.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/catalogs.sgml                    |  30 ++
 src/backend/catalog/pg_subscription.c         | 284 +++++++++++-
 src/backend/commands/subscriptioncmds.c       | 226 ++++++----
 .../replication/logical/applyparallelworker.c |   3 +-
 src/backend/replication/logical/launcher.c    |   9 +-
 src/backend/replication/logical/tablesync.c   | 421 +++++++++++++-----
 src/backend/replication/logical/worker.c      | 388 ++++++++++------
 src/include/catalog/pg_subscription.h         |   6 +
 src/include/catalog/pg_subscription_rel.h     |  15 +-
 src/include/replication/slot.h                |   3 +-
 src/include/replication/worker_internal.h     |  29 +-
 src/test/regress/expected/misc_sanity.out     |  30 +-
 12 files changed, 1094 insertions(+), 350 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index c1e4048054..eb8d4e1005 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8002,6 +8002,18 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        origin.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>sublastusedid</structfield> <type>int8</type>
+      </para>
+      <para>
+       The last used ID for tablesync workers. This ID is used to
+       create replication slots. The last used ID needs to be stored
+       to make logical replication can safely proceed after any interruption.
+       If sublastusedid is 0, then no table has been synced yet.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
@@ -8086,6 +8098,24 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        otherwise null
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srrelslotname</structfield> <type>text</type>
+      </para>
+      <para>
+       Replication slot name that is used for synchronization of relation
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srreloriginname</structfield> <type>text</type>
+      </para>
+      <para>
+       Origin name that is used for tracking synchronization of relation
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index a56ae311c3..f8dcad16ee 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -114,6 +114,14 @@ GetSubscription(Oid subid, bool missing_ok)
 	Assert(!isnull);
 	sub->origin = TextDatumGetCString(datum);
 
+	/* Get last used id */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
+							tup,
+							Anum_pg_subscription_sublastusedid,
+							&isnull);
+	Assert(!isnull);
+	sub->lastusedid = DatumGetInt64(datum);
+
 	ReleaseSysCache(tup);
 
 	return sub;
@@ -205,6 +213,44 @@ DisableSubscription(Oid subid)
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update the last used replication slot ID for the given subscription.
+ */
+void
+UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid)
+{
+	Relation	rel;
+	bool		nulls[Natts_pg_subscription];
+	bool		replaces[Natts_pg_subscription];
+	Datum		values[Natts_pg_subscription];
+	HeapTuple	tup;
+
+	/* Look up the subscription in the catalog */
+	rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+	tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	/* Form a new tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	replaces[Anum_pg_subscription_sublastusedid - 1] = true;
+	values[Anum_pg_subscription_sublastusedid- 1] = Int64GetDatum(lastusedid);
+
+	/* Update the catalog */
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+	heap_freetuple(tup);
+
+	table_close(rel, NoLock);
+}
+
 /*
  * Convert text array to list of strings.
  *
@@ -234,7 +280,7 @@ textarray_to_stringlist(ArrayType *textarray)
  */
 void
 AddSubscriptionRelState(Oid subid, Oid relid, char state,
-						XLogRecPtr sublsn)
+						XLogRecPtr sublsn, char *relslotname, char *reloriginname)
 {
 	Relation	rel;
 	HeapTuple	tup;
@@ -263,6 +309,14 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
 	else
 		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] = CStringGetTextDatum(relslotname);
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] = CStringGetTextDatum(reloriginname);
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -275,6 +329,58 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Internal function to modify columns for relation state update
+ */
+static void
+UpdateSubscriptionRelState_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char state,
+									XLogRecPtr sublsn)
+{
+	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
+	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+
+	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (sublsn != InvalidXLogRecPtr)
+		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
+	else
+		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication slot update
+ */
+static void
+UpdateSubscriptionRelReplicationSlot_internal(Datum *values,
+											bool *nulls,
+											bool *replaces,
+											char *relslotname)
+{
+	replaces[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] = CStringGetTextDatum(relslotname);
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication origin update
+ */
+static void
+UpdateSubscriptionRelOrigin_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char *reloriginname)
+{
+	replaces[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] = CStringGetTextDatum(reloriginname);
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+}
+
 /*
  * Update the state of a subscription table.
  */
@@ -305,14 +411,48 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	memset(nulls, false, sizeof(nulls));
 	memset(replaces, false, sizeof(replaces));
 
-	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
-	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
 
-	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
-	if (sublsn != InvalidXLogRecPtr)
-		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
-	else
-		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update the replication slot name of a subscription table.
+ */
+void
+UpdateSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *relslotname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
 
 	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
 							replaces);
@@ -324,6 +464,134 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update replication slot name, origin name and state of
+ * a subscription table in one transaction.
+ */
+void
+UpdateSubscriptionRel(Oid subid,
+					  Oid relid,
+					  char state,
+					  XLogRecPtr sublsn,
+					  char *relslotname,
+					  char *reloriginname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
+	UpdateSubscriptionRelOrigin_internal(values, nulls, replaces, reloriginname);
+
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Get origin name of subscription table.
+ *
+ * reloriginname's value has the replication origin name if the origin exists.
+ */
+void
+GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*originname;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srreloriginname, isnull);
+	if (!*isnull)
+	{
+		originname = TextDatumGetCString(d);
+		memcpy(reloriginname, originname, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
+/*
+ * Get replication slot name of subscription table.
+ *
+ * slotname's value has the replication slot name if the subscription has any.
+ */
+void
+GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*relrepslot;
+	bool		isnull;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srrelslotname, &isnull);
+	if (!isnull)
+	{
+		relrepslot = TextDatumGetCString(d);
+		memcpy(slotname, relrepslot, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
 /*
  * Get state of subscription table.
  *
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 464db6d247..1c4033fc79 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -649,6 +649,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 		publicationListToArray(publications);
 	values[Anum_pg_subscription_suborigin - 1] =
 		CStringGetTextDatum(opts.origin);
+	values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(0);
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -709,7 +710,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 										 rv->schemaname, rv->relname);
 
 				AddSubscriptionRelState(subid, relid, table_state,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 			}
 
 			/*
@@ -799,6 +800,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	} SubRemoveRels;
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -876,7 +879,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				AddSubscriptionRelState(sub->oid, relid,
 										copy_data ? SUBREL_STATE_INIT : SUBREL_STATE_READY,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 				ereport(DEBUG1,
 						(errmsg_internal("table \"%s.%s\" added to subscription \"%s\"",
 										 rv->schemaname, rv->relname, sub->name)));
@@ -900,6 +903,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				char		state;
 				XLogRecPtr	statelsn;
+				char		slotname[NAMEDATALEN] = {0};
 
 				/*
 				 * Lock pg_subscription_rel with AccessExclusiveLock to
@@ -926,7 +930,29 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/*
+				 * Find the logical replication sync worker if exists store
+				 * the slot number for dropping associated replication slots
+				 * later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot_name);
+				}
+				else
+				{
+					/*
+					 * Sync of this relation might be failed in an earlier
+					 * attempt, but the replication slot might still exist.
+					 */
+					GetSubscriptionRelReplicationSlot(sub->oid, relid, slotname);
+					if (strlen(slotname) > 0)
+						sub_remove_slots = lappend(sub_remove_slots, slotname);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
@@ -960,31 +986,24 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
-		 * the database operations we won't be able to rollback dropped slots.
+		 * Drop the replication slots associated with tablesync workers for
+		 * removed tables. This has to be at the end because otherwise if
+		 * there is an error while doing the database operations we won't be
+		 * able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			memcpy(syncslotname, lfirst(lc), sizeof(NAMEDATALEN));
+
+			/*
+			 * There is no certainty, maybe the slot does not exist yet. Also,
+			 * if we fail after removing some of the slots, next time, it will
+			 * again try to drop already dropped slots and fail. For these
+			 * reasons, we allow missing_ok = true for the drop.
+			 */
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1384,6 +1403,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	char	   *subname;
 	char	   *conninfo;
 	char	   *slotname;
+	int64		lastusedid;
 	List	   *subworkers;
 	ListCell   *lc;
 	char		originname[NAMEDATALEN];
@@ -1455,6 +1475,14 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	else
 		slotname = NULL;
 
+	/* Get the last used identifier by the subscription */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID, tup,
+							Anum_pg_subscription_sublastusedid, &isnull);
+	if (!isnull)
+		lastusedid = DatumGetInt64(datum);
+	else
+		lastusedid = 0;
+
 	/*
 	 * Since dropping a replication slot is not transactional, the replication
 	 * slot stays dropped even if the transaction rolls back.  So we cannot
@@ -1504,6 +1532,8 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	}
 	list_free(subworkers);
 
+	rstates = GetSubscriptionRelations(subid, true);
+
 	/*
 	 * Remove the no-longer-useful entry in the launcher's table of apply
 	 * worker start times.
@@ -1515,36 +1545,26 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	ApplyLauncherForgetWorkerStartTime(subid);
 
 	/*
-	 * Cleanup of tablesync replication origins.
-	 *
-	 * Any READY-state relations would already have dealt with clean-ups.
+	 * Cleanup of tablesync replication origins associated with the
+	 * subscription, if exists. Try to drop origins by creating all origin
+	 * names created for this subscription.
 	 *
 	 * Note that the state can't change because we have already stopped both
 	 * the apply and tablesync workers and they can't restart because of
 	 * exclusive lock on the subscription.
+	 *
+	 * XXX: This can be handled better instead of looping through all possible
 	 */
-	rstates = GetSubscriptionRelations(subid, true);
-	foreach(lc, rstates)
+	for (int64 i = 1; i <= lastusedid; i++)
 	{
-		SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-		Oid			relid = rstate->relid;
-
-		/* Only cleanup resources of tablesync workers */
-		if (!OidIsValid(relid))
-			continue;
+		char		originname_to_drop[NAMEDATALEN] = {0};
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * It is possible that the origin is not yet created for tablesync
-		 * worker so passing missing_ok = true. This can happen for the states
-		 * before SUBREL_STATE_FINISHEDCOPY.
-		 */
-		ReplicationOriginNameForLogicalRep(subid, relid, originname,
-										   sizeof(originname));
-		replorigin_drop_by_name(originname, true, false);
+		snprintf(originname_to_drop, sizeof(originname_to_drop), "pg_%u_%lld", subid, (long long) i);
+		/* missin_ok = true, since the origin might be already dropped. */
+		replorigin_drop_by_name(originname_to_drop, true, false);
 	}
 
+
 	/* Clean up dependencies */
 	deleteSharedDependencyRecordsFor(SubscriptionRelationId, subid, 0);
 
@@ -1596,39 +1616,17 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
+		List	   *slots = NULL;
 
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
 
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char	   *syncslotname = (char *) lfirst(lc);
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1651,6 +1649,71 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * Get the replication slot names associated with the subscription.
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok)
+{
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+					 " FROM pg_replication_slots"
+					 " WHERE slot_name LIKE 'pg_%i_sync_%%';",
+					 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+					errmsg("not tuple returned."));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();
+	\
+		return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -2005,6 +2068,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -2015,18 +2079,20 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
-			char		syncslotname[NAMEDATALEN] = {0};
-
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
-				 syncslotname);
+				 worker->slot_name);
 		}
 	}
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 3579e704fe..1cab625dfb 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -440,7 +440,8 @@ pa_launch_parallel_worker(void)
 										MySubscription->name,
 										MyLogicalRepWorker->userid,
 										InvalidOid,
-										dsm_segment_handle(winfo->dsm_seg));
+										dsm_segment_handle(winfo->dsm_seg),
+										0);
 
 	if (launched)
 	{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 970d170e73..77738e94a3 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -304,7 +304,7 @@ logicalrep_workers_find(Oid subid, bool only_running)
  */
 bool
 logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
-						 Oid relid, dsm_handle subworker_dsm)
+						 Oid relid, dsm_handle subworker_dsm, int64 slotid)
 {
 	BackgroundWorker bgw;
 	BackgroundWorkerHandle *bgw_handle;
@@ -429,7 +429,11 @@ retry:
 	/* Prepare the worker slot. */
 	worker->launch_time = now;
 	worker->in_use = true;
+	worker->is_first_run = true;
 	worker->generation++;
+	worker->created_slot = false;
+	worker->rep_slot_id = slotid;
+	worker->slot_name = (char *) palloc(NAMEDATALEN);
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -437,6 +441,7 @@ retry:
 	worker->relid = relid;
 	worker->relstate = SUBREL_STATE_UNKNOWN;
 	worker->relstate_lsn = InvalidXLogRecPtr;
+	worker->move_to_next_rel = false;
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
@@ -1155,7 +1160,7 @@ ApplyLauncherMain(Datum main_arg)
 				ApplyLauncherSetWorkerStartTime(sub->oid, now);
 				logicalrep_worker_launch(sub->dbid, sub->oid, sub->name,
 										 sub->owner, InvalidOid,
-										 DSM_HANDLE_INVALID);
+										 DSM_HANDLE_INVALID, 0);
 			}
 			else
 			{
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 07eea504ba..f39c037d79 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -126,12 +126,8 @@ static bool FetchTableStates(bool *started_tx);
 
 static StringInfo copybuf = NULL;
 
-/*
- * Exit routine for synchronization worker.
- */
 static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+clean_sync_worker(void)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -143,18 +139,28 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
-	/* And flush all writes. */
-	XLogFlush(GetXLogWriteRecPtr());
-
-	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
-	CommitTransactionCommand();
+	/*
+	 * Disconnect from publisher. Otherwise reused sync workers causes
+	 * exceeding max_wal_senders
+	 */
+	walrcv_disconnect(LogRepWorkerWalRcvConn);
+	LogRepWorkerWalRcvConn = NULL;
 
 	/* Find the leader apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
+}
+
+/*
+ * Exit routine for synchronization worker.
+ */
+static void
+pg_attribute_noreturn()
+finish_sync_worker(void)
+{
+	clean_sync_worker();
+
+	/* And flush all writes. */
+	XLogFlush(GetXLogWriteRecPtr());
 
 	/* Stop gracefully */
 	proc_exit(0);
@@ -284,6 +290,10 @@ invalidate_syncing_table_states(Datum arg, int cacheid, uint32 hashvalue)
 static void
 process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 {
+	List	   *rstates;
+	SubscriptionRelState *rstate;
+	ListCell   *lc;
+
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_CATCHUP &&
@@ -292,6 +302,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		TimeLineID	tli;
 		char		syncslotname[NAMEDATALEN] = {0};
 		char		originname[NAMEDATALEN] = {0};
+		bool		is_streaming_ended = false;
 
 		MyLogicalRepWorker->relstate = SUBREL_STATE_SYNCDONE;
 		MyLogicalRepWorker->relstate_lsn = current_lsn;
@@ -308,40 +319,29 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 								   MyLogicalRepWorker->relid,
 								   MyLogicalRepWorker->relstate,
 								   MyLogicalRepWorker->relstate_lsn);
+		CommitTransactionCommand();
 
 		/*
-		 * End streaming so that LogRepWorkerWalRcvConn can be used to drop
-		 * the slot.
-		 */
-		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
-
-		/*
-		 * Cleanup the tablesync slot.
+		 * Cleanup the tablesync slot. If the slot name used by this worker is
+		 * different from the default slot name for the worker, this means the
+		 * current table had started to being synchronized by another worker
+		 * and replication slot. And this worker is reusing a replication slot
+		 * from a previous attempt. We do not need that replication slot
+		 * anymore.
 		 *
 		 * This has to be done after updating the state because otherwise if
 		 * there is an error while doing the database operations we won't be
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
+										MyLogicalRepWorker->rep_slot_id,
 										syncslotname,
 										sizeof(syncslotname));
 
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
-		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
-
-		CommitTransactionCommand();
-		pgstat_report_stat(false);
-
-		/*
-		 * Start a new transaction to clean up the tablesync origin tracking.
-		 * This transaction will be ended within the finish_sync_worker().
-		 * Now, even, if we fail to remove this here, the apply worker will
-		 * ensure to clean it up afterward.
+		 * We are safe to drop the replication trackin origin after this
+		 * point. Now, even, if we fail to remove this here, the apply worker
+		 * will ensure to clean it up afterward.
 		 *
 		 * We need to do this after the table state is set to SYNCDONE.
 		 * Otherwise, if an error occurs while performing the database
@@ -350,34 +350,125 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 * have been cleared before restart. So, the restarted worker will use
 		 * invalid replication progress state resulting in replay of
 		 * transactions that have already been applied.
+		 *
+		 * Firstly reset the origin session to remove the ownership of the
+		 * slot. This is needed to allow the origin to be dropped or reused
+		 * later.
+		 */
+		replorigin_session_reset();
+		replorigin_session_origin = InvalidRepOriginId;
+		replorigin_session_origin_lsn = InvalidXLogRecPtr;
+		replorigin_session_origin_timestamp = 0;
+
+		StartTransactionCommand();
+		if (MyLogicalRepWorker->slot_name && strcmp(syncslotname, MyLogicalRepWorker->slot_name) != 0)
+		{
+			/*
+			 * End streaming so that LogRepWorkerWalRcvConn can be used to
+			 * drop the slot.
+			 */
+			walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+			is_streaming_ended = true;
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
+
+			ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+											   MyLogicalRepWorker->relid,
+											   originname,
+											   sizeof(originname));
+
+			/* Drop replication origin */
+			replorigin_drop_by_name(originname, true, false);
+		}
+
+		/*
+		 * We are safe to remove persisted replication slot and origin data,
+		 * since it's already in SYNCDONE state. They will not be needed
+		 * anymore.
 		 */
+		UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+							  MyLogicalRepWorker->relid,
+							  MyLogicalRepWorker->relstate,
+							  MyLogicalRepWorker->relstate_lsn,
+							  NULL,
+							  NULL);
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid))));
+
+		CommitTransactionCommand();
+		pgstat_report_stat(false);
+
 		StartTransactionCommand();
 
+		/*
+		 * This should return the default origin name for the worker. Even if
+		 * the worker used a different origin for this table, it should be
+		 * dropped and removed from the catalog so far.
+		 */
 		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
 										   MyLogicalRepWorker->relid,
 										   originname,
 										   sizeof(originname));
 
 		/*
-		 * Resetting the origin session removes the ownership of the slot.
-		 * This is needed to allow the origin to be dropped.
+		 * Check if any table whose relation state is still INIT. If a table
+		 * in INIT state is found, the worker will not be finished, it will be
+		 * reused instead.
 		 */
-		replorigin_session_reset();
-		replorigin_session_origin = InvalidRepOriginId;
-		replorigin_session_origin_lsn = InvalidXLogRecPtr;
-		replorigin_session_origin_timestamp = 0;
+		rstates = GetSubscriptionRelations(MySubscription->oid, true);
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * There is a chance that the user is concurrently performing refresh
-		 * for the subscription where we remove the table state and its origin
-		 * or the apply worker would have removed this origin. So passing
-		 * missing_ok = true.
-		 */
-		replorigin_drop_by_name(originname, true, false);
+		foreach(lc, rstates)
+		{
+			rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
+			memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
+
+			/*
+			 * Pick the table for the next run if there is not another worker
+			 * already picked that table.
+			 */
+			LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+			if (rstate->state != SUBREL_STATE_SYNCDONE &&
+				!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+			{
+				/* Update worker state for the next table */
+				MyLogicalRepWorker->is_first_run = false;
+				MyLogicalRepWorker->relid = rstate->relid;
+				MyLogicalRepWorker->relstate = rstate->state;
+				MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+				MyLogicalRepWorker->move_to_next_rel = true;
+				LWLockRelease(LogicalRepWorkerLock);
+				break;
+			}
+			LWLockRelease(LogicalRepWorkerLock);
+		}
+
+		/* Cleanup before next run or ending the worker. */
+		if (!MyLogicalRepWorker->move_to_next_rel)
+		{
+			/*
+			 * It is important to give an error if we are unable to drop the
+			 * slot, otherwise, it won't be dropped till the corresponding
+			 * subscription is dropped. So passing missing_ok = false.
+			 */
+			if (MyLogicalRepWorker->created_slot)
+			{
+				/* End streaming if it's not already ended. */
+				if (!is_streaming_ended)
+					walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+				ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
+			}
 
-		finish_sync_worker();
+			/* Drop replication origin before exiting. */
+			replorigin_drop_by_name(originname, true, false);
+
+			finish_sync_worker();
+		}
+		else
+		{
+			clean_sync_worker();
+		}
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -464,6 +555,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 			if (current_lsn >= rstate->lsn)
 			{
 				char		originname[NAMEDATALEN];
+				bool		is_origin_null = true;
 
 				rstate->state = SUBREL_STATE_READY;
 				rstate->lsn = current_lsn;
@@ -484,18 +576,27 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				 * error while dropping we won't restart it to drop the
 				 * origin. So passing missing_ok = true.
 				 */
-				ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-												   rstate->relid,
-												   originname,
-												   sizeof(originname));
-				replorigin_drop_by_name(originname, true, false);
+				GetSubscriptionRelOrigin(MyLogicalRepWorker->subid,
+										 rstate->relid, originname,
+										 &is_origin_null);
+
+				if (!is_origin_null)
+				{
+					replorigin_drop_by_name(originname, true, false);
+				}
 
 				/*
 				 * Update the state to READY only after the origin cleanup.
 				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-										   rstate->relid, rstate->state,
-										   rstate->lsn);
+				UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+									  rstate->relid,
+									  rstate->state,
+									  rstate->lsn,
+									  NULL,
+									  NULL);
+
+				CommitTransactionCommand();
+				started_tx = false;
 			}
 		}
 		else
@@ -584,12 +685,22 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						TimestampDifferenceExceeds(hentry->last_start_time, now,
 												   wal_retrieve_retry_interval))
 					{
+						if (IsTransactionState())
+							CommitTransactionCommand();
+						StartTransactionCommand();
+						started_tx = true;
+
+						MySubscription->lastusedid++;
+						UpdateSubscriptionLastSlotId(MyLogicalRepWorker->subid,
+													 MySubscription->lastusedid);
+
 						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
 												 MySubscription->oid,
 												 MySubscription->name,
 												 MyLogicalRepWorker->userid,
 												 rstate->relid,
-												 DSM_HANDLE_INVALID);
+												 DSM_HANDLE_INVALID,
+												 MySubscription->lastusedid);
 						hentry->last_start_time = now;
 					}
 				}
@@ -1198,8 +1309,8 @@ copy_table(Relation rel)
  * The name must not exceed NAMEDATALEN - 1 because of remote node constraints
  * on slot name length. We append system_identifier to avoid slot_name
  * collision with subscriptions in other clusters. With the current scheme
- * pg_%u_sync_%u_UINT64_FORMAT (3 + 10 + 6 + 10 + 20 + '\0'), the maximum
- * length of slot_name will be 50.
+ * pg_%u_sync_%lu_UINT64_FORMAT (3 + 10 + 6 + 20 + 20 + '\0'), the maximum
+ * length of slot_name will be 45.
  *
  * The returned slot name is stored in the supplied buffer (syncslotname) with
  * the given size.
@@ -1210,11 +1321,11 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
+ReplicationSlotNameForTablesync(Oid suboid, int64 slotid,
 								char *syncslotname, Size szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%lld_" UINT64_FORMAT, suboid,
+			(long long) slotid, GetSystemIdentifier());
 }
 
 /*
@@ -1237,6 +1348,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	WalRcvExecResult *res;
 	char		originname[NAMEDATALEN];
 	RepOriginId originid;
+	char	   *prev_slotname;
 
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
@@ -1265,7 +1377,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
+									MyLogicalRepWorker->rep_slot_id,
 									slotname,
 									NAMEDATALEN);
 
@@ -1285,12 +1397,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC ||
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY);
 
+	/*
+	 * See if tablesync of the current relation has been started with another
+	 * replication slot.
+	 *
+	 * Read previous slot name from the catalog, if exists.
+	 */
+	prev_slotname = (char *) palloc0(NAMEDATALEN);
+	StartTransactionCommand();
+	GetSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+									  MyLogicalRepWorker->relid,
+									  prev_slotname);
+
 	/* Assign the origin tracking record name. */
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   sizeof(originname));
 
+	CommitTransactionCommand();
+
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC)
 	{
 		/*
@@ -1304,10 +1430,48 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		 * breakdown then it wouldn't have succeeded so trying it next time
 		 * seems like a better bet.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, slotname, true);
+		if (strlen(prev_slotname) > 0)
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);
+
+			StartTransactionCommand();
+			/* Replication drop might still exist. Try to drop */
+			replorigin_drop_by_name(originname, true, false);
+
+			/*
+			 * Remove replication slot and origin name from the relation's
+			 * catalog record
+			 */
+			UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+								  MyLogicalRepWorker->relid,
+								  MyLogicalRepWorker->relstate,
+								  MyLogicalRepWorker->relstate_lsn,
+								  NULL,
+								  NULL);
+			CommitTransactionCommand();
+		}
 	}
 	else if (MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY)
 	{
+		/*
+		 * At this point, the table that is currently being synchronized
+		 * should have its replication slot name filled in the catalog. The
+		 * tablesync process was started with another sync worker and
+		 * replication slot. We need to continue using the same replication
+		 * slot in this worker too.
+		 */
+		if (strlen(prev_slotname) == 0)
+		{
+			elog(ERROR, "Replication slot could not be found for relation %u",
+				 MyLogicalRepWorker->relid);
+		}
+
+		/*
+		 * Proceed with the correct replication slot. Use previously created
+		 * replication slot to sync this table.
+		 */
+		slotname = prev_slotname;
+
 		/*
 		 * The COPY phase was previously done, but tablesync then crashed
 		 * before it was able to finish normally.
@@ -1328,6 +1492,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		goto copy_table_done;
 	}
 
+	/* Preparing for table copy operation */
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 	MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
 	MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
@@ -1335,10 +1500,12 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   MyLogicalRepWorker->relstate,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  MyLogicalRepWorker->relstate,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 	CommitTransactionCommand();
 	pgstat_report_stat(true);
 
@@ -1377,6 +1544,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						GetUserNameFromId(GetUserId(), true),
 						RelationGetRelationName(rel))));
 
+
 	/*
 	 * Start a transaction in the remote node in REPEATABLE READ mode.  This
 	 * ensures that both the replication slot we create (see below) and the
@@ -1392,48 +1560,91 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						res->err)));
 	walrcv_clear_result(res);
 
+	originid = replorigin_by_name(originname, true);
+
 	/*
 	 * Create a new permanent logical decoding slot. This slot will be used
 	 * for the catchup phase after COPY is done, so tell it to use the
 	 * snapshot to make the final data consistent.
+	 *
+	 * Replication slot will only be created if either this is the first run
+	 * of the worker or we're not using a previous replication slot.
 	 */
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-
-	/*
-	 * Setup replication origin tracking. The purpose of doing this before the
-	 * copy is to avoid doing the copy again due to any error in setting up
-	 * origin tracking.
-	 */
-	originid = replorigin_by_name(originname, true);
-	if (!OidIsValid(originid))
+	if (!MyLogicalRepWorker->created_slot)
 	{
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						   slotname, false /* permanent */ , false /* two_phase */ ,
+						   CRS_USE_SNAPSHOT, origin_startpos);
+
 		/*
-		 * Origin tracking does not exist, so create it now.
-		 *
-		 * Then advance to the LSN got from walrcv_create_slot. This is WAL
-		 * logged for the purpose of recovery. Locks are to prevent the
-		 * replication origin from vanishing while advancing.
+		 * Remember that we created the slot so that we will not try to create
+		 * it again.
 		 */
-		originid = replorigin_create(originname);
-
-		LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
-		replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
-						   true /* go backward */ , true /* WAL log */ );
-		UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->created_slot = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
+		/*
+		 * Setup replication origin tracking. The purpose of doing this before
+		 * the copy is to avoid doing the copy again due to any error in
+		 * setting up origin tracking.
+		 */
+		if (!OidIsValid(originid))
+		{
+			/*
+			 * Origin tracking does not exist, so create it now.
+			 */
+			originid = replorigin_create(originname);
+		}
+		else
+		{
+			/*
+			 * At this point, there shouldn't be any existing replication
+			 * origin with the same name.
+			 */
+			ereport(ERROR,
+					(errcode(ERRCODE_DUPLICATE_OBJECT),
+					 errmsg("replication origin \"%s\" already exists",
+							originname)));
+		}
 	}
 	else
 	{
-		ereport(ERROR,
-				(errcode(ERRCODE_DUPLICATE_OBJECT),
-				 errmsg("replication origin \"%s\" already exists",
-						originname)));
+		/*
+		 * Do not create a new replication slot, reuse the existing one
+		 * instead. Use a new snapshot for the replication slot to ensure that
+		 * tablesync and apply proceses are consistent with each other.
+		 */
+		WalRcvStreamOptions options;
+		int			server_version;
+
+		server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+		options.proto.logical.proto_version =
+			server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+			server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+			LOGICALREP_PROTO_VERSION_NUM;
+		options.proto.logical.publication_names = MySubscription->publications;
+
+		HOLD_INTERRUPTS();
+		walrcv_slot_snapshot(LogRepWorkerWalRcvConn, slotname, &options, origin_startpos);
+		RESUME_INTERRUPTS();
 	}
 
+	/*
+	 * Advance to the LSN got from walrcv_create_slot. This is WAL
+	 * logged for the purpose of recovery. Locks are to prevent the
+	 * replication origin from vanishing while advancing.
+	 *
+	 * Then setup replication origin tracking.
+	 */
+	LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+	replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
+					   true /* go backward */ , true /* WAL log */ );
+	UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+
 	/* Now do the initial data copy */
 	PushActiveSnapshot(GetTransactionSnapshot());
 	copy_table(rel);
@@ -1456,10 +1667,12 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	 * Update the persisted state to indicate the COPY phase is done; make it
 	 * visible to others.
 	 */
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   SUBREL_STATE_FINISHEDCOPY,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  SUBREL_STATE_FINISHEDCOPY,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 
 	CommitTransactionCommand();
 
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index cfb2ab6248..c12924e4da 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -388,6 +388,7 @@ static void stream_open_file(Oid subid, TransactionId xid,
 static void stream_write_change(char action, StringInfo s);
 static void stream_open_and_write_change(TransactionId xid, char action, StringInfo s);
 static void stream_close_file(void);
+static void stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
@@ -456,13 +457,22 @@ ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 {
 	if (OidIsValid(relid))
 	{
-		/* Replication origin name for tablesync workers. */
-		snprintf(originname, szoriginname, "pg_%u_%u", suboid, relid);
+		bool		is_null = true;
+
+		/*
+		 * Replication origin name for tablesync workers. First, look into the
+		 * catalog. If originname does not exist, then use the default name.
+		 */
+		GetSubscriptionRelOrigin(suboid, relid,
+								 originname, &is_null);
+		if (is_null)
+			snprintf(originname, szoriginname, "pg_%u_%lld", suboid, (long long) MyLogicalRepWorker->rep_slot_id);
 	}
 	else
 	{
 		/* Replication origin name for non-tablesync workers. */
 		snprintf(originname, szoriginname, "pg_%u", suboid);
+		elog(LOG, "apply worker originname %s", originname);
 	}
 }
 
@@ -3576,6 +3586,23 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can move to next table while running
+				 * process_syncing_tables_for_sync. Before we were able to
+				 * reuse tablesync workers, that
+				 * process_syncing_tables_for_sync call would exit the worker
+				 * instead of moving to next table. Now that tablesync workers
+				 * can be reused, we need to take care of memory contexts here
+				 * before moving to sync a table.
+				 */
+				if (MyLogicalRepWorker->move_to_next_rel)
+				{
+					MemoryContextResetAndDeleteChildren(ApplyMessageContext);
+					MemoryContextSwitchTo(TopMemoryContext);
+					return;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -3595,6 +3622,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+			if (MyLogicalRepWorker->move_to_next_rel)
+			{
+				endofstream = true;
+			}
 		}
 
 		/* Cleanup the memory. */
@@ -3697,8 +3728,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * If it's moving to next relation, this is a sync worker. Sync workers
+	 * end the streaming during process_syncing_tables_for_sync. Calling
+	 * endstreaming twice causes "no COPY in progress" errors.
+	 */
+	if (!MyLogicalRepWorker->move_to_next_rel)
+	{
+		/* All done */
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	}
 }
 
 /*
@@ -4282,6 +4321,56 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+ /* stream_build_options
+  * 	Build logical replication streaming options.
+  *
+  * This function sets streaming options including replication slot name
+  * and origin start position. Workers need these options for logical replication.
+  */
+static void
+stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4356,6 +4445,9 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 
 	/* allocate slot name in long-lived context */
 	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+
+	/* Keep the replication slot name used for this sync. */
+	MyLogicalRepWorker->slot_name = *myslotname;
 	pfree(syncslotname);
 }
 
@@ -4393,6 +4485,135 @@ start_apply(XLogRecPtr origin_startpos)
 	PG_END_TRY();
 }
 
+/*
+ * Runs the tablesync worker.
+ * It starts table sync. After successful sync,
+ * builds streaming options and starts streaming.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Set this to false for safety, in case we're already reusing the worker */
+	MyLogicalRepWorker->move_to_next_rel = false;
+
+	start_table_sync(origin_startpos, &slotname);
+
+	/*
+	 * Allocate the origin name in long-lived context for error context
+	 * message.
+	 */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   originname_size);
+	CommitTransactionCommand();
+
+	set_apply_error_context_origin(originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/* Start normal logical streaming replication. */
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+}
+
+/*
+ * Runs the apply worker.
+ * It sets up replication origin, the streaming options
+ * and then starts streaming.
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
+{
+	/* This is the leader apply worker */
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+
+	slotname = MySubscription->slotname;
+
+	/*
+		* This shouldn't happen if the subscription is enabled, but guard
+		* against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+		* crash if slot is NULL.)
+		*/
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											MySubscription->name, &err);
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+		* We don't really use the output identify_system for anything but it
+		* does some initializations on the upstream so let's still call it.
+		*/
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+	{
+		/* Start streaming with two_phase enabled */
+		options->proto.logical.twophase = true;
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
+	}
+	else
+	{
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+	}
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+}
+
 /*
  * Common initialization for leader apply worker and parallel apply worker.
  *
@@ -4485,7 +4706,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	/* Attach to slot */
 	logicalrep_worker_attach(worker_slot);
@@ -4513,156 +4733,48 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
-	 * the subscription relation state.
+	 * the subscription relation state. Do this outside the loop to avoid
+	 * exceeding MAX_SYSCACHE_CALLBACKS
 	 */
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
-
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
-
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-
 	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 * The loop where worker does its job. It loops until the worker is not
+	 * reused.
 	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
-
-	if (!am_tablesync_worker())
+	while (MyLogicalRepWorker->is_first_run ||
+		   MyLogicalRepWorker->move_to_next_rel)
 	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
+		if (am_tablesync_worker())
 		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
+			/*
+			 * This is a tablesync worker. Start syncing tables before
+			 * starting the apply loop.
+			 */
+			run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 		}
 		else
 		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+			/* This is main apply worker */
+			run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 		}
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-						MySubscription->name,
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-						"?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
-
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+		/* Run the main loop. */
+		start_apply(origin_startpos);
 
+		if (MyLogicalRepWorker->move_to_next_rel)
+		{
+			StartTransactionCommand();
+			ereport(LOG,
+					(errmsg("logical replication table synchronization worker for subscription \"%s\" has moved to sync table \"%s\".",
+							MySubscription->name, get_rel_name(MyLogicalRepWorker->relid))));
+			CommitTransactionCommand();
+		}
+	}
 	proc_exit(0);
 }
 
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index b0f2a1705d..a0ee12e259 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -103,6 +103,9 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
 
 	/* Only publish data originating from the specified origin */
 	text		suborigin BKI_DEFAULT(LOGICALREP_ORIGIN_ANY);
+
+	/* The last used ID to create a replication slot for tablesync */
+	int64		sublastusedid BKI_DEFAULT(0);
 #endif
 } FormData_pg_subscription;
 
@@ -137,6 +140,8 @@ typedef struct Subscription
 	List	   *publications;	/* List of publication names to subscribe to */
 	char	   *origin;			/* Only publish data originating from the
 								 * specified origin */
+	int64		lastusedid;		/* Last used unique ID to create replication
+								 * slots in tablesync */
 } Subscription;
 
 /* Disallow streaming in-progress transactions. */
@@ -157,6 +162,7 @@ typedef struct Subscription
 extern Subscription *GetSubscription(Oid subid, bool missing_ok);
 extern void FreeSubscription(Subscription *sub);
 extern void DisableSubscription(Oid subid);
+extern void UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid);
 
 extern int	CountDBSubscriptions(Oid dbid);
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 60a2bcca23..a35d04cccd 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -44,6 +44,12 @@ CATALOG(pg_subscription_rel,6102,SubscriptionRelRelationId)
 											 * used for synchronization
 											 * coordination, or NULL if not
 											 * valid */
+	text		srrelslotname BKI_FORCE_NULL;	/* name of the replication
+												 * slot for relation in
+												 * subscription */
+	text		srreloriginname BKI_FORCE_NULL; /* origin name for relation in
+												 * subscription */
+
 #endif
 } FormData_pg_subscription_rel;
 
@@ -81,10 +87,17 @@ typedef struct SubscriptionRelState
 } SubscriptionRelState;
 
 extern void AddSubscriptionRelState(Oid subid, Oid relid, char state,
-									XLogRecPtr sublsn);
+									XLogRecPtr sublsn, char *relslotname, char *reloriginname);
 extern void UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 									   XLogRecPtr sublsn);
+extern void UpdateSubscriptionRel(Oid subid, Oid relid, char state,
+								  XLogRecPtr sublsn, char *relslotname, char *reloriginname);
+extern void UpdateSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *relslotname);
+
 extern char GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn);
+extern void GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname);
+extern void GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull);
+
 extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 
 extern bool HasSubscriptionRelations(Oid subid);
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 8872c80cdf..3547daaaec 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -219,8 +219,9 @@ extern bool InvalidateObsoleteReplicationSlots(XLogSegNo oldestSegno);
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
 extern bool ReplicationSlotName(int index, Name name);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, Size szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, int64 slotid, char *syncslotname, Size szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List *GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index dc87a4edd1..5f4b7b1f7c 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -35,6 +35,26 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/*
+	 * Indicates if worker is running for the first time or in reuse
+	 */
+	bool		is_first_run;
+
+	/*
+	 * Indicates if the sync worker created a replication slot or it reuses an
+	 * existing one created by another worker.
+	 */
+	bool		created_slot;
+
+	/*
+	 * Unique identifier for replication slot to be created by tablesnync
+	 * workers, if needed.
+	 */
+	int64		rep_slot_id;
+
+	/* Replication slot name used by the worker. */
+	char	   *slot_name;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
@@ -56,6 +76,12 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/*
+	 * Used to indicate whether sync worker will be reused for another
+	 * relation
+	 */
+	bool		move_to_next_rel;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction or
@@ -231,7 +257,8 @@ extern LogicalRepWorker *logicalrep_worker_find(Oid subid, Oid relid,
 extern List *logicalrep_workers_find(Oid subid, bool only_running);
 extern bool logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname,
 									 Oid userid, Oid relid,
-									 dsm_handle subworker_dsm);
+									 dsm_handle subworker_dsm,
+									 int64 slotid);
 extern void logicalrep_worker_stop(Oid subid, Oid relid);
 extern void logicalrep_pa_worker_stop(int slot_no, uint16 generation);
 extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/test/regress/expected/misc_sanity.out b/src/test/regress/expected/misc_sanity.out
index a57fd142a9..3d34a21421 100644
--- a/src/test/regress/expected/misc_sanity.out
+++ b/src/test/regress/expected/misc_sanity.out
@@ -47,20 +47,22 @@ WHERE c.oid < 16384 AND
       relkind = 'r' AND
       attstorage != 'p'
 ORDER BY 1, 2;
-         relname         |    attname    |   atttypid   
--------------------------+---------------+--------------
- pg_attribute            | attacl        | aclitem[]
- pg_attribute            | attfdwoptions | text[]
- pg_attribute            | attmissingval | anyarray
- pg_attribute            | attoptions    | text[]
- pg_class                | relacl        | aclitem[]
- pg_class                | reloptions    | text[]
- pg_class                | relpartbound  | pg_node_tree
- pg_index                | indexprs      | pg_node_tree
- pg_index                | indpred       | pg_node_tree
- pg_largeobject          | data          | bytea
- pg_largeobject_metadata | lomacl        | aclitem[]
-(11 rows)
+         relname         |     attname     |   atttypid   
+-------------------------+-----------------+--------------
+ pg_attribute            | attacl          | aclitem[]
+ pg_attribute            | attfdwoptions   | text[]
+ pg_attribute            | attmissingval   | anyarray
+ pg_attribute            | attoptions      | text[]
+ pg_class                | relacl          | aclitem[]
+ pg_class                | reloptions      | text[]
+ pg_class                | relpartbound    | pg_node_tree
+ pg_index                | indexprs        | pg_node_tree
+ pg_index                | indpred         | pg_node_tree
+ pg_largeobject          | data            | bytea
+ pg_largeobject_metadata | lomacl          | aclitem[]
+ pg_subscription_rel     | srreloriginname | text
+ pg_subscription_rel     | srrelslotname   | text
+(13 rows)
 
 -- system catalogs without primary keys
 --
-- 
2.25.1

#31shveta malik
shveta.malik@gmail.com
In reply to: Melih Mutlu (#30)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Thu, Jan 26, 2023 at 7:53 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

If a relation is currently being synced by a tablesync worker and uses a replication slot/origin for that operation, then srrelslotname and srreloriginname fields will have values.
When a relation is done with its replication slot/origin, their info gets removed from related catalog row, so that slot/origin can be reused for another table or dropped if not needed anymore.
In your case, all relations are in READY state so it's expected that srrelslotname and srreloriginname are empty. READY relations do not need a replication slot/origin anymore.

Tables are probably synced so quickly that you're missing the moments when a tablesync worker copies a relation and stores its rep. slot/origin in the catalog.
If initial sync is long enough, then you should be able to see the columns get updated. I follow [1] to make it longer and test if the patch really updates the catalog.

Thank You for the details. It is clear now.

Rebased and resolved conflicts. Please check the new version

Please find my suggestions on v9:

1.
--Can we please add a few more points to the Summary to make it more clear.
a) something telling that reusability of workers is for tables under
one subscription and not across multiple subscriptions.
b) Since we are reusing both workers and slots, can we add:
--when do we actually end up spawning a new worker
--when do we actually end up creating a new slot in a worker rather
than using existing one.
--if we reuse existing slots, what happens to the snapshot?

2.
+       The last used ID for tablesync workers. This ID is used to
+       create replication slots. The last used ID needs to be stored
+       to make logical replication can safely proceed after any interruption.
+       If sublastusedid is 0, then no table has been synced yet.

--typo:
to make logical replication can safely proceed ==> to make logical
replication safely proceed

--Also, does it sound better:
The last used ID for tablesync workers. It acts as an unique
identifier for replication slots
which are created by table-sync workers. The last used ID needs to be
persisted...

3.
is_first_run;
move_to_next_rel;
--Do you think one variable is enough here as we do not get any extra
info by using 2 variables? Can we have 1 which is more generic like
'ready_to_reuse'. Otherwise, please let me know if we must use 2.

4.
/* missin_ok = true, since the origin might be already dropped. */
typo: missing_ok

5. GetReplicationSlotNamesBySubId:
errmsg("not tuple returned."));

Can we have a better error msg:
ereport(ERROR,
errmsg("could not receive list of slots
associated with subscription %d, error: %s", subid, res->err));

6.
static void
clean_sync_worker(void)

--can we please add introductory comment for this function.

7.
/*
* Pick the table for the next run if there is not another worker
* already picked that table.
*/
Pick the table for the next run if it is not already picked up by
another worker.

8.
process_syncing_tables_for_sync()

/* Cleanup before next run or ending the worker. */
--can we please improve this comment:
if there is no more work left for this worker, stop the worker
gracefully, else do clean-up and make it ready for the next
relation/run.

9.
LogicalRepSyncTableStart:
* Read previous slot name from the catalog, if exists.
*/
prev_slotname = (char *) palloc0(NAMEDATALEN);
Do we need to free this at the end?

10.
if (strlen(prev_slotname) == 0)
{
elog(ERROR, "Replication slot could not be
found for relation %u",
MyLogicalRepWorker->relid);
}
shall we mention subid also in error msg.

I am reviewing further...
thanks
Shveta

#32vignesh C
vignesh21@gmail.com
In reply to: Melih Mutlu (#30)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Thu, 26 Jan 2023 at 19:53, Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi Shveta,

Thanks for reviewing.

shveta malik <shveta.malik@gmail.com>, 25 Oca 2023 Çar, 16:02 tarihinde şunu yazdı:

On Mon, Jan 23, 2023 at 6:30 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:
--I see initial data copied, but new catalog columns srrelslotname
and srreloriginname are not updated:
postgres=# select sublastusedid from pg_subscription;
sublastusedid
---------------
2

postgres=# select * from pg_subscription_rel;
srsubid | srrelid | srsubstate | srsublsn | srrelslotname | srreloriginname
---------+---------+------------+-----------+---------------+-----------------
16409 | 16384 | r | 0/15219E0 | |
16409 | 16389 | r | 0/15219E0 | |
16409 | 16396 | r | 0/15219E0 | |

When are these supposed to be updated? I thought the slotname created
will be updated here. Am I missing something here?

If a relation is currently being synced by a tablesync worker and uses a replication slot/origin for that operation, then srrelslotname and srreloriginname fields will have values.
When a relation is done with its replication slot/origin, their info gets removed from related catalog row, so that slot/origin can be reused for another table or dropped if not needed anymore.
In your case, all relations are in READY state so it's expected that srrelslotname and srreloriginname are empty. READY relations do not need a replication slot/origin anymore.

Tables are probably synced so quickly that you're missing the moments when a tablesync worker copies a relation and stores its rep. slot/origin in the catalog.
If initial sync is long enough, then you should be able to see the columns get updated. I follow [1] to make it longer and test if the patch really updates the catalog.

Also the v8 patch does not apply on HEAD, giving merge conflicts.

Rebased and resolved conflicts. Please check the new version

CFBot shows some compilation errors as in [1]https://cirrus-ci.com/task/4897131543134208?logs=build#L1236, please post an updated
version for the same:
[14:38:38.392] [827/1808] Compiling C object
src/backend/postgres_lib.a.p/replication_logical_tablesync.c.o
[14:38:38.392] ../src/backend/replication/logical/tablesync.c: In
function ‘LogicalRepSyncTableStart’:
[14:38:38.392] ../src/backend/replication/logical/tablesync.c:1629:3:
warning: implicit declaration of function ‘walrcv_slot_snapshot’
[-Wimplicit-function-declaration]
[14:38:38.392] 1629 | walrcv_slot_snapshot(LogRepWorkerWalRcvConn,
slotname, &options, origin_startpos);
[14:38:38.392] | ^~~~~~~~~~~~~~~~~~~~

[14:38:45.125] FAILED: src/backend/postgres
[14:38:45.125] cc @src/backend/postgres.rsp
[14:38:45.125] /usr/bin/ld:
src/backend/postgres_lib.a.p/replication_logical_tablesync.c.o: in
function `LogicalRepSyncTableStart':
[14:38:45.125] /tmp/cirrus-ci-build/build/../src/backend/replication/logical/tablesync.c:1629:
undefined reference to `walrcv_slot_snapshot'
[14:38:45.125] collect2: error: ld returned 1 exit status

[1]: https://cirrus-ci.com/task/4897131543134208?logs=build#L1236

Regards,
Vignesh

#33shveta malik
shveta.malik@gmail.com
In reply to: shveta malik (#31)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Fri, Jan 27, 2023 at 3:41 PM shveta malik <shveta.malik@gmail.com> wrote:

I am reviewing further...
thanks
Shveta

Few more comments:

v4-0001:

1)
REPLICATION_SLOT_SNAPSHOT
--Do we need 'CREATE' prefix with it i.e. CREATE_REPLICATION_SNAPSHOT
(or some other brief one with CREATE?). 'REPLICATION_SLOT_SNAPSHOT'
does not look like a command/action and thus is confusing.

2)
is used in the currenct transaction. This command is currently only supported
for logical replication.
slots.
--typo: currenct-->current
--slots can be moved to previous line

3)
/*
* Signal that we don't need the timeout mechanism. We're just creating
* the replication slot and don't yet accept feedback messages or send
* keepalives. As we possibly need to wait for further WAL the walsender
* would otherwise possibly be killed too soon.
*/
We're just creating the replication slot --> We're just creating the
replication snapshot

4)
I see XactReadOnly check in CreateReplicationSlot, do we need the same
in ReplicationSlotSnapshot() as well?

===============
v9-0002:

5)
/* We are safe to drop the replication trackin origin after this
--typo: tracking

6)
slot->data.catalog_xmin = xmin_horizon;
slot->effective_xmin = xmin_horizon;
SpinLockRelease(&slot->mutex);
xmin_horizon =
GetOldestSafeDecodingTransactionId(!need_full_snapshot);
ReplicationSlotsComputeRequiredXmin(true);

--do we need to set xmin_horizon in slot after
'GetOldestSafeDecodingTransactionId' call, otherwise it will be set to
InvalidId in slot. Is that intentional? I see that we do set this
correct xmin_horizon in builder->initial_xmin_horizon but the slot is
carrying Invalid one.

thanks
Shveta

#34wangw.fnst@fujitsu.com
wangw.fnst@fujitsu.com
In reply to: Melih Mutlu (#28)
RE: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Mon, Jan 23, 2023 21:00 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi,

Thanks for your review.
Attached updated versions of the patches.

Thanks for updating the patch set.

5. New member "created_slot" in structure LogicalRepWorker
+       /*
+        * Indicates if the sync worker created a replication slot or it reuses an
+        * existing one created by another worker.
+        */
+       bool            created_slot;

I think the second half of the sentence looks inaccurate.
Because I think this flag could be false even when we reuse an existing slot
created by another worker. Assuming the first run for the worker tries to sync
a table which is synced by another sync worker before, and the relstate is set
to SUBREL_STATE_FINISHEDCOPY by another sync worker, I think this flag will

not

be set to true. (see function LogicalRepSyncTableStart)

So, what if we simplify the description here and just say that this worker
already has it's default slot?

If I'm not missing something and you agree with this, please also kindly modify
the relevant comment atop the if-statement (!MyLogicalRepWorker-
created_slot)
in the function LogicalRepSyncTableStart.

This "created_slot" indicates whether the current worker has created a
replication slot for its own use. If so, created_slot will be true, otherwise false.
Let's say the tablesync worker has not created its own slot yet in its previous
runs or this is its first run. And the worker decides to reuse an existing
replication slot (which created by another tablesync worker). Then created_slot
is expected to be false. Because this particular tablesync worker has not created
its own slot yet in either of its runs.

In your example, the worker is in its first run and begin to sync a table whose
state is FINISHEDCOPY. If the table's state is FINISHEDCOPY then the table
should already have a replication slot created for its own sync process. The
worker will want to reuse that existing replication slot for this particular table
and it will not create a new replication slot. So created_slot will be false, because
the worker has not actually created any replication slot yet.

Basically, created_slot is set to true only if "walrcv_create_slot" is called by the
tablesync worker any time during its lifetime. Otherwise, it's possible that the
worker can use existing replication slots for each table it syncs. (e.g. if all the
tables that the worker has synced were in FINISHEDCOPY state, then the
worker will not need to create a new slot).

Does it make sense now? Maybe I need to improve comments to make it
clearer.

Yes, I think it makes sense. Thanks for the detailed explanation.
I think I misunderstood the second half of the comment. I previously thought it
meant that it was also true when reusing an existing slot.

I found one typo in v9-0002, but it seems already mentioned by Shi in [1].#5
before. Maybe you can have a look at that email for this and some other
comments.

Regards,
Wang Wei

#35wangw.fnst@fujitsu.com
wangw.fnst@fujitsu.com
In reply to: wangw.fnst@fujitsu.com (#34)
RE: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Tues, Jan 31, 2023 18:27 PM I wrote:

I found one typo in v9-0002, but it seems already mentioned by Shi in [1].#5
before. Maybe you can have a look at that email for this and some other
comments.

Sorry, I forgot to add the link to the email. Please refer to [1]/messages/by-id/OSZPR01MB631013C833C98E826B3CFCB9FDC69@OSZPR01MB6310.jpnprd01.prod.outlook.com.

[1]: /messages/by-id/OSZPR01MB631013C833C98E826B3CFCB9FDC69@OSZPR01MB6310.jpnprd01.prod.outlook.com

Regards,
Wang Wei

#36shveta malik
shveta.malik@gmail.com
In reply to: wangw.fnst@fujitsu.com (#34)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Tue, Jan 31, 2023 at 3:57 PM wangw.fnst@fujitsu.com
<wangw.fnst@fujitsu.com> wrote:

On Mon, Jan 23, 2023 21:00 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi,

Thanks for your review.
Attached updated versions of the patches.

Thanks for updating the patch set.

5. New member "created_slot" in structure LogicalRepWorker
+       /*
+        * Indicates if the sync worker created a replication slot or it reuses an
+        * existing one created by another worker.
+        */
+       bool            created_slot;

Yes, I think it makes sense. Thanks for the detailed explanation.
I think I misunderstood the second half of the comment. I previously thought it
meant that it was also true when reusing an existing slot.

I agree with Wang-san that the comment is confusing, I too
misunderstood it initially during my first run of the code. Maybe it
can be improved.
'Indicates if the sync worker created a replication slot for itself;
set to false if sync worker reuses an existing one created by another
worker'

thanks
Shveta

#37Melih Mutlu
m.melihmutlu@gmail.com
In reply to: shveta malik (#31)
2 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi,

Please see attached patches for the below changes.

shveta malik <shveta.malik@gmail.com>, 27 Oca 2023 Cum, 13:12 tarihinde
şunu yazdı:

On Thu, Jan 26, 2023 at 7:53 PM Melih Mutlu <m.melihmutlu@gmail.com>
wrote:
1.
--Can we please add a few more points to the Summary to make it more clear.
a) something telling that reusability of workers is for tables under
one subscription and not across multiple subscriptions.
b) Since we are reusing both workers and slots, can we add:
--when do we actually end up spawning a new worker
--when do we actually end up creating a new slot in a worker rather
than using existing one.
--if we reuse existing slots, what happens to the snapshot?

I modified the commit message if that's what you mean by the Summary.

2.
+       The last used ID for tablesync workers. This ID is used to
+       create replication slots. The last used ID needs to be stored
+       to make logical replication can safely proceed after any
interruption.
+       If sublastusedid is 0, then no table has been synced yet.

--typo:
to make logical replication can safely proceed ==> to make logical
replication safely proceed

Done

3.
is_first_run;
move_to_next_rel;
--Do you think one variable is enough here as we do not get any extra
info by using 2 variables? Can we have 1 which is more generic like
'ready_to_reuse'. Otherwise, please let me know if we must use 2.

Right. Removed is_first_run and renamed move_to_next_rel as ready_to_reuse.

4.
/* missin_ok = true, since the origin might be already dropped. */
typo: missing_ok

Done.

5. GetReplicationSlotNamesBySubId:
errmsg("not tuple returned."));

Can we have a better error msg:
ereport(ERROR,
errmsg("could not receive list of slots
associated with subscription %d, error: %s", subid, res->err));

Done.

6.
static void
clean_sync_worker(void)

--can we please add introductory comment for this function.

Done.

7.
/*
* Pick the table for the next run if there is not another worker
* already picked that table.
*/
Pick the table for the next run if it is not already picked up by
another worker.

Done.

8.
process_syncing_tables_for_sync()

/* Cleanup before next run or ending the worker. */
--can we please improve this comment:
if there is no more work left for this worker, stop the worker
gracefully, else do clean-up and make it ready for the next
relation/run.

Done

9.
LogicalRepSyncTableStart:
* Read previous slot name from the catalog, if exists.
*/
prev_slotname = (char *) palloc0(NAMEDATALEN);
Do we need to free this at the end?

Pfree'd prev_slotname after we're done with it.

10.
if (strlen(prev_slotname) == 0)
{
elog(ERROR, "Replication slot could not be
found for relation %u",
MyLogicalRepWorker->relid);
}
shall we mention subid also in error msg.

Done.

Thanks for reviewing,
--
Melih Mutlu
Microsoft

Attachments:

v5-0001-Add-replication-protocol-cmd-to-create-a-snapshot.patchapplication/octet-stream; name=v5-0001-Add-replication-protocol-cmd-to-create-a-snapshot.patchDownload
From 6c741266475dbe8f7088fa953b27ac6972987d18 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 13 Oct 2022 17:05:45 +0300
Subject: [PATCH 1/2] Add replication protocol cmd to create a snapshot

Introduced CREATE_REPLICATION_SNAPSHOT to be able to create and use a
snapshot without creating a new replication slot, but by using an
existing slot.

CREATE_REPLICATION_SNAPSHOT simply does what CREATE_REPLICATION_SLOT does
without creating a new replication slot.

CREATE_REPLICATION_SNAPSHOT command imports the snapshot into the current
transaction and returns consistent_point. The changes earlier than the
consistent_point will be applied by importing the snapshot. All changes
later than the consistent_point will be available to be consumed from
the replication slot.

This is useful for reusing replication slots in logical replication.
Otherwise, tablesync workers cannot start from a consistent point to copy a relation and then apply changes by consuming from replication slot.
---
 doc/src/sgml/protocol.sgml                    |  31 ++++++
 .../libpqwalreceiver/libpqwalreceiver.c       |  69 +++++++++++-
 src/backend/replication/logical/logical.c     |  40 ++++++-
 .../replication/logical/logicalfuncs.c        |   1 +
 src/backend/replication/repl_gram.y           |  18 +++-
 src/backend/replication/repl_scanner.l        |   2 +
 src/backend/replication/slotfuncs.c           |   1 +
 src/backend/replication/walsender.c           | 102 +++++++++++++++++-
 src/include/nodes/replnodes.h                 |  11 ++
 src/include/replication/logical.h             |   1 +
 src/include/replication/walreceiver.h         |  13 +++
 src/tools/pgindent/typedefs.list              |   2 +
 12 files changed, 287 insertions(+), 4 deletions(-)

diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 93fc7167d4..948a5aabd4 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2613,6 +2613,37 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
      </listitem>
     </varlistentry>
 
+    <varlistentry id="protocol-replication-replication-slot-snapshot">
+     <term><literal>CREATE_REPLICATION_SNAPSHOT</literal> <replaceable class="parameter">slot_name</replaceable> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
+      <indexterm><primary>CREATE_REPLICATION_SNAPSHOT</primary></indexterm>
+     </term>
+     <listitem>
+      <para>
+       Creates a snapshot including all the changes from the replication slot until
+       the point at which the replication slot becomes consistent. Then the snapshot
+       is used in the current transaction. This command is currently only supported
+       for logical replication slots.
+      </para>
+
+      <para>
+       In response to this command, the server will return a one-row result set,
+       containing the following field:
+       <variablelist>
+        <varlistentry>
+         <term><literal>consistent_point</literal> (<type>text</type>)</term>
+         <listitem>
+          <para>
+           The WAL location at which the slot became consistent.  This is the
+           earliest location from which streaming can start on this replication
+           slot.
+          </para>
+         </listitem>
+        </varlistentry>
+       </variablelist>
+      </para>
+     </listitem>
+    </varlistentry>
+
     <varlistentry id="protocol-replication-base-backup" xreflabel="BASE_BACKUP">
      <term><literal>BASE_BACKUP</literal> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
       <indexterm><primary>BASE_BACKUP</primary></indexterm>
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index 560ec974fa..9173163a07 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -81,6 +81,8 @@ static WalRcvExecResult *libpqrcv_exec(WalReceiverConn *conn,
 									   const int nRetTypes,
 									   const Oid *retTypes);
 static void libpqrcv_disconnect(WalReceiverConn *conn);
+static void libpqrcv_slot_snapshot(WalReceiverConn *conn, char *slotname,
+								   const WalRcvStreamOptions *options, XLogRecPtr *lsn);
 
 static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_connect = libpqrcv_connect,
@@ -97,7 +99,8 @@ static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_create_slot = libpqrcv_create_slot,
 	.walrcv_get_backend_pid = libpqrcv_get_backend_pid,
 	.walrcv_exec = libpqrcv_exec,
-	.walrcv_disconnect = libpqrcv_disconnect
+	.walrcv_disconnect = libpqrcv_disconnect,
+	.walrcv_slot_snapshot = libpqrcv_slot_snapshot
 };
 
 /* Prototypes for private functions */
@@ -933,6 +936,70 @@ libpqrcv_create_slot(WalReceiverConn *conn, const char *slotname,
 	return snapshot;
 }
 
+/*
+ * TODO
+ */
+static void
+libpqrcv_slot_snapshot(WalReceiverConn *conn,
+					   char *slotname,
+					   const WalRcvStreamOptions *options,
+					   XLogRecPtr *lsn)
+{
+	StringInfoData cmd;
+	PGresult   *res;
+	char	   *pubnames_str;
+	List	   *pubnames;
+	char	   *pubnames_literal;
+
+	initStringInfo(&cmd);
+
+	/* Build the command. */
+	appendStringInfo(&cmd, "CREATE_REPLICATION_SNAPSHOT \"%s\"", slotname);
+	appendStringInfoString(&cmd, " (");
+	appendStringInfo(&cmd, " proto_version '%u'",
+					 options->proto.logical.proto_version);
+
+	/* Add publication names. */
+	pubnames = options->proto.logical.publication_names;
+	pubnames_str = stringlist_to_identifierstr(conn->streamConn, pubnames);
+	if (!pubnames_str)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	pubnames_literal = PQescapeLiteral(conn->streamConn, pubnames_str,
+									   strlen(pubnames_str));
+	if (!pubnames_literal)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	appendStringInfo(&cmd, ", publication_names %s", pubnames_literal);
+	PQfreemem(pubnames_literal);
+	pfree(pubnames_str);
+
+	appendStringInfoString(&cmd, " )");
+
+	/* Execute the command. */
+	res = libpqrcv_PQexec(conn->streamConn, cmd.data);
+	pfree(cmd.data);
+
+	if (PQresultStatus(res) != PGRES_TUPLES_OK)
+	{
+		PQclear(res);
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg("Could not create a snapshot by replication slot \"%s\": %s",
+						slotname, pchomp(PQerrorMessage(conn->streamConn)))));
+	}
+
+	if (lsn)
+		*lsn = DatumGetLSN(DirectFunctionCall1Coll(pg_lsn_in, InvalidOid,
+												   CStringGetDatum(PQgetvalue(res, 0, 0))));
+
+	PQclear(res);
+}
+
 /*
  * Return PID of remote backend process.
  */
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 1a58dd7649..d1504f7ec6 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -461,6 +461,10 @@ CreateInitDecodingContext(const char *plugin,
  * fast_forward
  *		bypass the generation of logical changes.
  *
+ * need_full_snapshot
+ * 		if true, create a snapshot able to read all tables,
+ * 		otherwise do not create any snapshot.
+ *
  * xl_routine
  *		XLogReaderRoutine used by underlying xlogreader
  *
@@ -479,6 +483,7 @@ LogicalDecodingContext *
 CreateDecodingContext(XLogRecPtr start_lsn,
 					  List *output_plugin_options,
 					  bool fast_forward,
+					  bool need_full_snapshot,
 					  XLogReaderRoutine *xl_routine,
 					  LogicalOutputPluginWriterPrepareWrite prepare_write,
 					  LogicalOutputPluginWriterWrite do_write,
@@ -487,6 +492,7 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 	LogicalDecodingContext *ctx;
 	ReplicationSlot *slot;
 	MemoryContext old_context;
+	TransactionId xmin_horizon = InvalidTransactionId;
 
 	/* shorter lines... */
 	slot = MyReplicationSlot;
@@ -533,8 +539,40 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 		start_lsn = slot->data.confirmed_flush;
 	}
 
+
+	/*
+	 * We need to determine a safe xmin horizon to start decoding from if we
+	 * want to create a snapshot too. Otherwise we would end up with a
+	 * snapshot that cannot be imported since xmin value from the snapshot may
+	 * be less than the oldest safe xmin. To avoid this call
+	 * GetOldestSafeDecodingTransactionId() to return a safe xmin value, which
+	 * can be used while exporting/importing the snapshot.
+	 *
+	 * So we have to acquire the ProcArrayLock to prevent computation of new
+	 * xmin horizons by other backends, get the safe decoding xid, and inform
+	 * the slot machinery about the new limit. Once that's done the
+	 * ProcArrayLock can be released as the slot machinery now is protecting
+	 * against vacuum.
+	 */
+	if (need_full_snapshot)
+	{
+		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+		xmin_horizon = GetOldestSafeDecodingTransactionId(!need_full_snapshot);
+
+		SpinLockAcquire(&slot->mutex);
+		slot->effective_catalog_xmin = xmin_horizon;
+		slot->data.catalog_xmin = xmin_horizon;
+		slot->effective_xmin = xmin_horizon;
+		SpinLockRelease(&slot->mutex);
+
+		ReplicationSlotsComputeRequiredXmin(true);
+
+		LWLockRelease(ProcArrayLock);
+	}
+
 	ctx = StartupDecodingContext(output_plugin_options,
-								 start_lsn, InvalidTransactionId, false,
+								 start_lsn, xmin_horizon, need_full_snapshot,
 								 fast_forward, xl_routine, prepare_write,
 								 do_write, update_progress);
 
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index fa1b641a2b..1191c70eb0 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -208,6 +208,7 @@ pg_logical_slot_get_changes_guts(FunctionCallInfo fcinfo, bool confirm, bool bin
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									options,
 									false,
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/repl_gram.y b/src/backend/replication/repl_gram.y
index 0c874e33cf..957a5cc653 100644
--- a/src/backend/replication/repl_gram.y
+++ b/src/backend/replication/repl_gram.y
@@ -65,6 +65,7 @@ Node *replication_parse_result;
 %token K_CREATE_REPLICATION_SLOT
 %token K_DROP_REPLICATION_SLOT
 %token K_TIMELINE_HISTORY
+%token K_CREATE_REPLICATION_SNAPSHOT
 %token K_WAIT
 %token K_TIMELINE
 %token K_PHYSICAL
@@ -80,7 +81,7 @@ Node *replication_parse_result;
 %type <node>	command
 %type <node>	base_backup start_replication start_logical_replication
 				create_replication_slot drop_replication_slot identify_system
-				read_replication_slot timeline_history show
+				read_replication_slot timeline_history show create_replication_snapshot
 %type <list>	generic_option_list
 %type <defelt>	generic_option
 %type <uintval>	opt_timeline
@@ -114,6 +115,7 @@ command:
 			| read_replication_slot
 			| timeline_history
 			| show
+			| create_replication_snapshot
 			;
 
 /*
@@ -307,6 +309,19 @@ timeline_history:
 				}
 			;
 
+/*
+ * CREATE_REPLICATION_SNAPSHOT %s options
+ */
+create_replication_snapshot:
+			K_CREATE_REPLICATION_SNAPSHOT var_name plugin_options
+				{
+					CreateReplicationSnapshotCmd *n = makeNode(CreateReplicationSnapshotCmd);
+					n->slotname = $2;
+					n->options = $3;
+					$$ = (Node *) n;
+				}
+			;
+
 opt_physical:
 			K_PHYSICAL
 			| /* EMPTY */
@@ -400,6 +415,7 @@ ident_or_keyword:
 			| K_CREATE_REPLICATION_SLOT	{ $$ = "create_replication_slot"; }
 			| K_DROP_REPLICATION_SLOT		{ $$ = "drop_replication_slot"; }
 			| K_TIMELINE_HISTORY			{ $$ = "timeline_history"; }
+			| K_CREATE_REPLICATION_SNAPSHOT	{ $$ = "create_replication_snapshot"; }
 			| K_WAIT						{ $$ = "wait"; }
 			| K_TIMELINE					{ $$ = "timeline"; }
 			| K_PHYSICAL					{ $$ = "physical"; }
diff --git a/src/backend/replication/repl_scanner.l b/src/backend/replication/repl_scanner.l
index cb467ca46f..5ba2e9c54b 100644
--- a/src/backend/replication/repl_scanner.l
+++ b/src/backend/replication/repl_scanner.l
@@ -126,6 +126,7 @@ START_REPLICATION	{ return K_START_REPLICATION; }
 CREATE_REPLICATION_SLOT		{ return K_CREATE_REPLICATION_SLOT; }
 DROP_REPLICATION_SLOT		{ return K_DROP_REPLICATION_SLOT; }
 TIMELINE_HISTORY	{ return K_TIMELINE_HISTORY; }
+CREATE_REPLICATION_SNAPSHOT	{ return K_CREATE_REPLICATION_SNAPSHOT; }
 PHYSICAL			{ return K_PHYSICAL; }
 RESERVE_WAL			{ return K_RESERVE_WAL; }
 LOGICAL				{ return K_LOGICAL; }
@@ -303,6 +304,7 @@ replication_scanner_is_replication_command(void)
 		case K_DROP_REPLICATION_SLOT:
 		case K_READ_REPLICATION_SLOT:
 		case K_TIMELINE_HISTORY:
+		case K_CREATE_REPLICATION_SNAPSHOT:
 		case K_SHOW:
 			/* Yes; push back the first token so we can parse later. */
 			repl_pushed_back_token = first_token;
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 2f3c964824..b3ae11b2c8 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -478,6 +478,7 @@ pg_logical_replication_slot_advance(XLogRecPtr moveto)
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									NIL,
 									true,	/* fast_forward */
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 4ed3747e3f..09011d6add 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -238,6 +238,7 @@ static void CreateReplicationSlot(CreateReplicationSlotCmd *cmd);
 static void DropReplicationSlot(DropReplicationSlotCmd *cmd);
 static void StartReplication(StartReplicationCmd *cmd);
 static void StartLogicalReplication(StartReplicationCmd *cmd);
+static void CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd);
 static void ProcessStandbyMessage(void);
 static void ProcessStandbyReplyMessage(void);
 static void ProcessStandbyHSFeedbackMessage(void);
@@ -1280,7 +1281,7 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	 * are reported early.
 	 */
 	logical_decoding_ctx =
-		CreateDecodingContext(cmd->startpoint, cmd->options, false,
+		CreateDecodingContext(cmd->startpoint, cmd->options, false, false,
 							  XL_ROUTINE(.page_read = logical_read_xlog_page,
 										 .segment_open = WalSndSegmentOpen,
 										 .segment_close = wal_segment_close),
@@ -1332,6 +1333,96 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	EndCommand(&qc, DestRemote, false);
 }
 
+/*
+ * Create a snapshot from an existing replication slot.
+ */
+static void
+CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd)
+{
+	Snapshot	snap;
+	LogicalDecodingContext *ctx;
+	char		xloc[MAXFNAMELEN];
+	DestReceiver *dest;
+	TupOutputState *tstate;
+	TupleDesc	tupdesc;
+	Datum		values[1];
+	bool		nulls[1] = {0};
+
+	Assert(!MyReplicationSlot);
+
+	if (!IsTransactionBlock())
+		ereport(ERROR,
+				(errmsg("%s must be called inside a transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (XactIsoLevel != XACT_REPEATABLE_READ)
+		ereport(ERROR,
+				(errmsg("%s must be called in REPEATABLE READ isolation mode transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (!XactReadOnly)
+		ereport(ERROR,
+				(errmsg("%s must be called in a read only transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (FirstSnapshotSet)
+		ereport(ERROR,
+				(errmsg("%s must be called before any query",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (IsSubTransaction())
+		ereport(ERROR,
+				(errmsg("%s must not be called in a subtransaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	ReplicationSlotAcquire(cmd->slotname, false);
+
+	ctx = CreateDecodingContext(MyReplicationSlot->data.restart_lsn,
+								cmd->options,
+								false,
+								true,
+								XL_ROUTINE(.page_read = logical_read_xlog_page,
+										   .segment_open = WalSndSegmentOpen,
+										   .segment_close = wal_segment_close),
+								WalSndPrepareWrite, WalSndWriteData,
+								WalSndUpdateProgress);
+
+	/*
+	 * Signal that we don't need the timeout mechanism. We're just creating
+	 * the replication slot and don't yet accept feedback messages or send
+	 * keepalives. As we possibly need to wait for further WAL the walsender
+	 * would otherwise possibly be killed too soon.
+	 */
+	last_reply_timestamp = 0;
+
+	/* build initial snapshot, might take a while */
+	DecodingContextFindStartpoint(ctx);
+
+	snap = SnapBuildInitialSnapshot(ctx->snapshot_builder);
+	RestoreTransactionSnapshot(snap, MyProc);
+
+	/* Don't need the decoding context anymore */
+	FreeDecodingContext(ctx);
+
+	/* Create a tuple to send consisten WAL location */
+	snprintf(xloc, sizeof(xloc), "%X/%X",
+			 LSN_FORMAT_ARGS(MyReplicationSlot->data.confirmed_flush));
+
+	dest = CreateDestReceiver(DestRemoteSimple);
+	tupdesc = CreateTemplateTupleDesc(1);
+	TupleDescInitBuiltinEntry(tupdesc, (AttrNumber) 1, "consistent_point",
+							  TEXTOID, -1, 0);
+	tstate = begin_tup_output_tupdesc(dest, tupdesc, &TTSOpsVirtual);
+
+	/* consistent wal location */
+	values[0] = CStringGetTextDatum(xloc);
+
+	do_tup_output(tstate, values, nulls);
+	end_tup_output(tstate);
+
+	ReplicationSlotRelease();
+}
+
 /*
  * LogicalDecodingContext 'prepare_write' callback.
  *
@@ -1860,6 +1951,15 @@ exec_replication_command(const char *cmd_string)
 			}
 			break;
 
+		case T_CreateReplicationSnapshotCmd:
+			{
+				cmdtag = "CREATE_REPLICATION_SNAPSHOT";
+				set_ps_display(cmdtag);
+				CreateReplicationSnapshot((CreateReplicationSnapshotCmd *) cmd_node);
+				EndReplicationCommand(cmdtag);
+				break;
+			}
+
 		default:
 			elog(ERROR, "unrecognized replication command node tag: %u",
 				 cmd_node->type);
diff --git a/src/include/nodes/replnodes.h b/src/include/nodes/replnodes.h
index 4321ba8f86..154ab74f33 100644
--- a/src/include/nodes/replnodes.h
+++ b/src/include/nodes/replnodes.h
@@ -108,4 +108,15 @@ typedef struct TimeLineHistoryCmd
 	TimeLineID	timeline;
 } TimeLineHistoryCmd;
 
+/* ----------------------
+ *		CREATE_REPLICATION_SNAPSHOT command
+ * ----------------------
+ */
+typedef struct CreateReplicationSnapshotCmd
+{
+	NodeTag		type;
+	char	   *slotname;
+	List	   *options;
+} CreateReplicationSnapshotCmd;
+
 #endif							/* REPLNODES_H */
diff --git a/src/include/replication/logical.h b/src/include/replication/logical.h
index 5f49554ea0..6535786a0e 100644
--- a/src/include/replication/logical.h
+++ b/src/include/replication/logical.h
@@ -125,6 +125,7 @@ extern LogicalDecodingContext *CreateInitDecodingContext(const char *plugin,
 extern LogicalDecodingContext *CreateDecodingContext(XLogRecPtr start_lsn,
 													 List *output_plugin_options,
 													 bool fast_forward,
+													 bool need_full_snapshot,
 													 XLogReaderRoutine *xl_routine,
 													 LogicalOutputPluginWriterPrepareWrite prepare_write,
 													 LogicalOutputPluginWriterWrite do_write,
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index decffe352d..bd11f9f31e 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -384,6 +384,16 @@ typedef WalRcvExecResult *(*walrcv_exec_fn) (WalReceiverConn *conn,
  */
 typedef void (*walrcv_disconnect_fn) (WalReceiverConn *conn);
 
+/*
+ * walrcv_slot_snapshot_fn
+ *
+ * Create a snapshot by an existing replication slot
+ */
+typedef void (*walrcv_slot_snapshot_fn) (WalReceiverConn *conn,
+										 char *slotname,
+										 const WalRcvStreamOptions *options,
+										 XLogRecPtr *lsn);
+
 typedef struct WalReceiverFunctionsType
 {
 	walrcv_connect_fn walrcv_connect;
@@ -401,6 +411,7 @@ typedef struct WalReceiverFunctionsType
 	walrcv_get_backend_pid_fn walrcv_get_backend_pid;
 	walrcv_exec_fn walrcv_exec;
 	walrcv_disconnect_fn walrcv_disconnect;
+	walrcv_slot_snapshot_fn walrcv_slot_snapshot;
 } WalReceiverFunctionsType;
 
 extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
@@ -435,6 +446,8 @@ extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
 	WalReceiverFunctions->walrcv_exec(conn, exec, nRetTypes, retTypes)
 #define walrcv_disconnect(conn) \
 	WalReceiverFunctions->walrcv_disconnect(conn)
+#define walrcv_slot_snapshot(conn, slotname, options, lsn) \
+	WalReceiverFunctions->walrcv_slot_snapshot(conn, slotname, options, lsn)
 
 static inline void
 walrcv_clear_result(WalRcvExecResult *walres)
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 07fbb7ccf6..a3de441b7e 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2323,6 +2323,7 @@ ReplicationSlotCtlData
 ReplicationSlotOnDisk
 ReplicationSlotPersistency
 ReplicationSlotPersistentData
+CreateReplicationSnapshotCmd
 ReplicationState
 ReplicationStateCtl
 ReplicationStateOnDisk
@@ -3796,6 +3797,7 @@ walrcv_receive_fn
 walrcv_send_fn
 walrcv_server_version_fn
 walrcv_startstreaming_fn
+walrcv_slot_snapshot_fn
 wchar2mb_with_len_converter
 wchar_t
 win32_deadchild_waitinfo
-- 
2.25.1

v9-0002-Reuse-Logical-Replication-Background-worker.patchapplication/octet-stream; name=v9-0002-Reuse-Logical-Replication-Background-worker.patchDownload
From 757e519b98805ace371e5c3f972979d2b85ffd0c Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH 2/2] Reuse Logical Replication Background worker

This commit allows tablesync workers to move to another table that needs synchronization,
when they're done with the current table in tablesync phase of Logical Replication.

Before this commit, tablesync workers were capable of syncing only one
relation. A new worker, replication slot and origin were needed for each
relation in the subscription.

Now, tablesync workers are not only limited with one relation and can move to another relation and reuse existing
replication slots and origins

This reduces the overhead of launching/killing a new background worker for each relation.
By reusing tablesync workers, replication slots and origins created for tablesync can be reused as well.
Removing the burden of creating/dropping replication slot/origin improves tablesync speed significantly especially for empty or small tables.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/catalogs.sgml                    |  30 ++
 src/backend/catalog/pg_subscription.c         | 284 +++++++++++-
 src/backend/commands/subscriptioncmds.c       | 226 ++++++----
 .../replication/logical/applyparallelworker.c |   3 +-
 src/backend/replication/logical/launcher.c    |   9 +-
 src/backend/replication/logical/tablesync.c   | 421 +++++++++++++-----
 src/backend/replication/logical/worker.c      | 388 ++++++++++------
 src/include/catalog/pg_subscription.h         |   6 +
 src/include/catalog/pg_subscription_rel.h     |  15 +-
 src/include/replication/slot.h                |   3 +-
 src/include/replication/worker_internal.h     |  29 +-
 src/test/regress/expected/misc_sanity.out     |  30 +-
 12 files changed, 1094 insertions(+), 350 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index c1e4048054..eb8d4e1005 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8002,6 +8002,18 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        origin.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>sublastusedid</structfield> <type>int8</type>
+      </para>
+      <para>
+       The last used ID for tablesync workers. This ID is used to
+       create replication slots. The last used ID needs to be stored
+       to make logical replication can safely proceed after any interruption.
+       If sublastusedid is 0, then no table has been synced yet.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
@@ -8086,6 +8098,24 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        otherwise null
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srrelslotname</structfield> <type>text</type>
+      </para>
+      <para>
+       Replication slot name that is used for synchronization of relation
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srreloriginname</structfield> <type>text</type>
+      </para>
+      <para>
+       Origin name that is used for tracking synchronization of relation
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index a56ae311c3..f8dcad16ee 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -114,6 +114,14 @@ GetSubscription(Oid subid, bool missing_ok)
 	Assert(!isnull);
 	sub->origin = TextDatumGetCString(datum);
 
+	/* Get last used id */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
+							tup,
+							Anum_pg_subscription_sublastusedid,
+							&isnull);
+	Assert(!isnull);
+	sub->lastusedid = DatumGetInt64(datum);
+
 	ReleaseSysCache(tup);
 
 	return sub;
@@ -205,6 +213,44 @@ DisableSubscription(Oid subid)
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update the last used replication slot ID for the given subscription.
+ */
+void
+UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid)
+{
+	Relation	rel;
+	bool		nulls[Natts_pg_subscription];
+	bool		replaces[Natts_pg_subscription];
+	Datum		values[Natts_pg_subscription];
+	HeapTuple	tup;
+
+	/* Look up the subscription in the catalog */
+	rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+	tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	/* Form a new tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	replaces[Anum_pg_subscription_sublastusedid - 1] = true;
+	values[Anum_pg_subscription_sublastusedid- 1] = Int64GetDatum(lastusedid);
+
+	/* Update the catalog */
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+	heap_freetuple(tup);
+
+	table_close(rel, NoLock);
+}
+
 /*
  * Convert text array to list of strings.
  *
@@ -234,7 +280,7 @@ textarray_to_stringlist(ArrayType *textarray)
  */
 void
 AddSubscriptionRelState(Oid subid, Oid relid, char state,
-						XLogRecPtr sublsn)
+						XLogRecPtr sublsn, char *relslotname, char *reloriginname)
 {
 	Relation	rel;
 	HeapTuple	tup;
@@ -263,6 +309,14 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
 	else
 		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] = CStringGetTextDatum(relslotname);
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] = CStringGetTextDatum(reloriginname);
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -275,6 +329,58 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Internal function to modify columns for relation state update
+ */
+static void
+UpdateSubscriptionRelState_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char state,
+									XLogRecPtr sublsn)
+{
+	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
+	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+
+	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (sublsn != InvalidXLogRecPtr)
+		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
+	else
+		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication slot update
+ */
+static void
+UpdateSubscriptionRelReplicationSlot_internal(Datum *values,
+											bool *nulls,
+											bool *replaces,
+											char *relslotname)
+{
+	replaces[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] = CStringGetTextDatum(relslotname);
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication origin update
+ */
+static void
+UpdateSubscriptionRelOrigin_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char *reloriginname)
+{
+	replaces[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] = CStringGetTextDatum(reloriginname);
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+}
+
 /*
  * Update the state of a subscription table.
  */
@@ -305,14 +411,48 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	memset(nulls, false, sizeof(nulls));
 	memset(replaces, false, sizeof(replaces));
 
-	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
-	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
 
-	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
-	if (sublsn != InvalidXLogRecPtr)
-		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
-	else
-		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update the replication slot name of a subscription table.
+ */
+void
+UpdateSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *relslotname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
 
 	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
 							replaces);
@@ -324,6 +464,134 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update replication slot name, origin name and state of
+ * a subscription table in one transaction.
+ */
+void
+UpdateSubscriptionRel(Oid subid,
+					  Oid relid,
+					  char state,
+					  XLogRecPtr sublsn,
+					  char *relslotname,
+					  char *reloriginname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
+	UpdateSubscriptionRelOrigin_internal(values, nulls, replaces, reloriginname);
+
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Get origin name of subscription table.
+ *
+ * reloriginname's value has the replication origin name if the origin exists.
+ */
+void
+GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*originname;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srreloriginname, isnull);
+	if (!*isnull)
+	{
+		originname = TextDatumGetCString(d);
+		memcpy(reloriginname, originname, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
+/*
+ * Get replication slot name of subscription table.
+ *
+ * slotname's value has the replication slot name if the subscription has any.
+ */
+void
+GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*relrepslot;
+	bool		isnull;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srrelslotname, &isnull);
+	if (!isnull)
+	{
+		relrepslot = TextDatumGetCString(d);
+		memcpy(slotname, relrepslot, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
 /*
  * Get state of subscription table.
  *
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 464db6d247..1c4033fc79 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -649,6 +649,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 		publicationListToArray(publications);
 	values[Anum_pg_subscription_suborigin - 1] =
 		CStringGetTextDatum(opts.origin);
+	values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(0);
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -709,7 +710,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 										 rv->schemaname, rv->relname);
 
 				AddSubscriptionRelState(subid, relid, table_state,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 			}
 
 			/*
@@ -799,6 +800,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	} SubRemoveRels;
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -876,7 +879,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				AddSubscriptionRelState(sub->oid, relid,
 										copy_data ? SUBREL_STATE_INIT : SUBREL_STATE_READY,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 				ereport(DEBUG1,
 						(errmsg_internal("table \"%s.%s\" added to subscription \"%s\"",
 										 rv->schemaname, rv->relname, sub->name)));
@@ -900,6 +903,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				char		state;
 				XLogRecPtr	statelsn;
+				char		slotname[NAMEDATALEN] = {0};
 
 				/*
 				 * Lock pg_subscription_rel with AccessExclusiveLock to
@@ -926,7 +930,29 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/*
+				 * Find the logical replication sync worker if exists store
+				 * the slot number for dropping associated replication slots
+				 * later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot_name);
+				}
+				else
+				{
+					/*
+					 * Sync of this relation might be failed in an earlier
+					 * attempt, but the replication slot might still exist.
+					 */
+					GetSubscriptionRelReplicationSlot(sub->oid, relid, slotname);
+					if (strlen(slotname) > 0)
+						sub_remove_slots = lappend(sub_remove_slots, slotname);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
@@ -960,31 +986,24 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
-		 * the database operations we won't be able to rollback dropped slots.
+		 * Drop the replication slots associated with tablesync workers for
+		 * removed tables. This has to be at the end because otherwise if
+		 * there is an error while doing the database operations we won't be
+		 * able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			memcpy(syncslotname, lfirst(lc), sizeof(NAMEDATALEN));
+
+			/*
+			 * There is no certainty, maybe the slot does not exist yet. Also,
+			 * if we fail after removing some of the slots, next time, it will
+			 * again try to drop already dropped slots and fail. For these
+			 * reasons, we allow missing_ok = true for the drop.
+			 */
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1384,6 +1403,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	char	   *subname;
 	char	   *conninfo;
 	char	   *slotname;
+	int64		lastusedid;
 	List	   *subworkers;
 	ListCell   *lc;
 	char		originname[NAMEDATALEN];
@@ -1455,6 +1475,14 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	else
 		slotname = NULL;
 
+	/* Get the last used identifier by the subscription */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID, tup,
+							Anum_pg_subscription_sublastusedid, &isnull);
+	if (!isnull)
+		lastusedid = DatumGetInt64(datum);
+	else
+		lastusedid = 0;
+
 	/*
 	 * Since dropping a replication slot is not transactional, the replication
 	 * slot stays dropped even if the transaction rolls back.  So we cannot
@@ -1504,6 +1532,8 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	}
 	list_free(subworkers);
 
+	rstates = GetSubscriptionRelations(subid, true);
+
 	/*
 	 * Remove the no-longer-useful entry in the launcher's table of apply
 	 * worker start times.
@@ -1515,36 +1545,26 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	ApplyLauncherForgetWorkerStartTime(subid);
 
 	/*
-	 * Cleanup of tablesync replication origins.
-	 *
-	 * Any READY-state relations would already have dealt with clean-ups.
+	 * Cleanup of tablesync replication origins associated with the
+	 * subscription, if exists. Try to drop origins by creating all origin
+	 * names created for this subscription.
 	 *
 	 * Note that the state can't change because we have already stopped both
 	 * the apply and tablesync workers and they can't restart because of
 	 * exclusive lock on the subscription.
+	 *
+	 * XXX: This can be handled better instead of looping through all possible
 	 */
-	rstates = GetSubscriptionRelations(subid, true);
-	foreach(lc, rstates)
+	for (int64 i = 1; i <= lastusedid; i++)
 	{
-		SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-		Oid			relid = rstate->relid;
-
-		/* Only cleanup resources of tablesync workers */
-		if (!OidIsValid(relid))
-			continue;
+		char		originname_to_drop[NAMEDATALEN] = {0};
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * It is possible that the origin is not yet created for tablesync
-		 * worker so passing missing_ok = true. This can happen for the states
-		 * before SUBREL_STATE_FINISHEDCOPY.
-		 */
-		ReplicationOriginNameForLogicalRep(subid, relid, originname,
-										   sizeof(originname));
-		replorigin_drop_by_name(originname, true, false);
+		snprintf(originname_to_drop, sizeof(originname_to_drop), "pg_%u_%lld", subid, (long long) i);
+		/* missin_ok = true, since the origin might be already dropped. */
+		replorigin_drop_by_name(originname_to_drop, true, false);
 	}
 
+
 	/* Clean up dependencies */
 	deleteSharedDependencyRecordsFor(SubscriptionRelationId, subid, 0);
 
@@ -1596,39 +1616,17 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
+		List	   *slots = NULL;
 
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
 
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char	   *syncslotname = (char *) lfirst(lc);
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1651,6 +1649,71 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * Get the replication slot names associated with the subscription.
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok)
+{
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+					 " FROM pg_replication_slots"
+					 " WHERE slot_name LIKE 'pg_%i_sync_%%';",
+					 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+					errmsg("not tuple returned."));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();
+	\
+		return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -2005,6 +2068,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -2015,18 +2079,20 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
-			char		syncslotname[NAMEDATALEN] = {0};
-
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
-				 syncslotname);
+				 worker->slot_name);
 		}
 	}
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 3579e704fe..1cab625dfb 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -440,7 +440,8 @@ pa_launch_parallel_worker(void)
 										MySubscription->name,
 										MyLogicalRepWorker->userid,
 										InvalidOid,
-										dsm_segment_handle(winfo->dsm_seg));
+										dsm_segment_handle(winfo->dsm_seg),
+										0);
 
 	if (launched)
 	{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 970d170e73..77738e94a3 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -304,7 +304,7 @@ logicalrep_workers_find(Oid subid, bool only_running)
  */
 bool
 logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
-						 Oid relid, dsm_handle subworker_dsm)
+						 Oid relid, dsm_handle subworker_dsm, int64 slotid)
 {
 	BackgroundWorker bgw;
 	BackgroundWorkerHandle *bgw_handle;
@@ -429,7 +429,11 @@ retry:
 	/* Prepare the worker slot. */
 	worker->launch_time = now;
 	worker->in_use = true;
+	worker->is_first_run = true;
 	worker->generation++;
+	worker->created_slot = false;
+	worker->rep_slot_id = slotid;
+	worker->slot_name = (char *) palloc(NAMEDATALEN);
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -437,6 +441,7 @@ retry:
 	worker->relid = relid;
 	worker->relstate = SUBREL_STATE_UNKNOWN;
 	worker->relstate_lsn = InvalidXLogRecPtr;
+	worker->move_to_next_rel = false;
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
@@ -1155,7 +1160,7 @@ ApplyLauncherMain(Datum main_arg)
 				ApplyLauncherSetWorkerStartTime(sub->oid, now);
 				logicalrep_worker_launch(sub->dbid, sub->oid, sub->name,
 										 sub->owner, InvalidOid,
-										 DSM_HANDLE_INVALID);
+										 DSM_HANDLE_INVALID, 0);
 			}
 			else
 			{
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 07eea504ba..f39c037d79 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -126,12 +126,8 @@ static bool FetchTableStates(bool *started_tx);
 
 static StringInfo copybuf = NULL;
 
-/*
- * Exit routine for synchronization worker.
- */
 static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+clean_sync_worker(void)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -143,18 +139,28 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
-	/* And flush all writes. */
-	XLogFlush(GetXLogWriteRecPtr());
-
-	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
-	CommitTransactionCommand();
+	/*
+	 * Disconnect from publisher. Otherwise reused sync workers causes
+	 * exceeding max_wal_senders
+	 */
+	walrcv_disconnect(LogRepWorkerWalRcvConn);
+	LogRepWorkerWalRcvConn = NULL;
 
 	/* Find the leader apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
+}
+
+/*
+ * Exit routine for synchronization worker.
+ */
+static void
+pg_attribute_noreturn()
+finish_sync_worker(void)
+{
+	clean_sync_worker();
+
+	/* And flush all writes. */
+	XLogFlush(GetXLogWriteRecPtr());
 
 	/* Stop gracefully */
 	proc_exit(0);
@@ -284,6 +290,10 @@ invalidate_syncing_table_states(Datum arg, int cacheid, uint32 hashvalue)
 static void
 process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 {
+	List	   *rstates;
+	SubscriptionRelState *rstate;
+	ListCell   *lc;
+
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_CATCHUP &&
@@ -292,6 +302,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		TimeLineID	tli;
 		char		syncslotname[NAMEDATALEN] = {0};
 		char		originname[NAMEDATALEN] = {0};
+		bool		is_streaming_ended = false;
 
 		MyLogicalRepWorker->relstate = SUBREL_STATE_SYNCDONE;
 		MyLogicalRepWorker->relstate_lsn = current_lsn;
@@ -308,40 +319,29 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 								   MyLogicalRepWorker->relid,
 								   MyLogicalRepWorker->relstate,
 								   MyLogicalRepWorker->relstate_lsn);
+		CommitTransactionCommand();
 
 		/*
-		 * End streaming so that LogRepWorkerWalRcvConn can be used to drop
-		 * the slot.
-		 */
-		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
-
-		/*
-		 * Cleanup the tablesync slot.
+		 * Cleanup the tablesync slot. If the slot name used by this worker is
+		 * different from the default slot name for the worker, this means the
+		 * current table had started to being synchronized by another worker
+		 * and replication slot. And this worker is reusing a replication slot
+		 * from a previous attempt. We do not need that replication slot
+		 * anymore.
 		 *
 		 * This has to be done after updating the state because otherwise if
 		 * there is an error while doing the database operations we won't be
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
+										MyLogicalRepWorker->rep_slot_id,
 										syncslotname,
 										sizeof(syncslotname));
 
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
-		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
-
-		CommitTransactionCommand();
-		pgstat_report_stat(false);
-
-		/*
-		 * Start a new transaction to clean up the tablesync origin tracking.
-		 * This transaction will be ended within the finish_sync_worker().
-		 * Now, even, if we fail to remove this here, the apply worker will
-		 * ensure to clean it up afterward.
+		 * We are safe to drop the replication trackin origin after this
+		 * point. Now, even, if we fail to remove this here, the apply worker
+		 * will ensure to clean it up afterward.
 		 *
 		 * We need to do this after the table state is set to SYNCDONE.
 		 * Otherwise, if an error occurs while performing the database
@@ -350,34 +350,125 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 * have been cleared before restart. So, the restarted worker will use
 		 * invalid replication progress state resulting in replay of
 		 * transactions that have already been applied.
+		 *
+		 * Firstly reset the origin session to remove the ownership of the
+		 * slot. This is needed to allow the origin to be dropped or reused
+		 * later.
+		 */
+		replorigin_session_reset();
+		replorigin_session_origin = InvalidRepOriginId;
+		replorigin_session_origin_lsn = InvalidXLogRecPtr;
+		replorigin_session_origin_timestamp = 0;
+
+		StartTransactionCommand();
+		if (MyLogicalRepWorker->slot_name && strcmp(syncslotname, MyLogicalRepWorker->slot_name) != 0)
+		{
+			/*
+			 * End streaming so that LogRepWorkerWalRcvConn can be used to
+			 * drop the slot.
+			 */
+			walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+			is_streaming_ended = true;
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
+
+			ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+											   MyLogicalRepWorker->relid,
+											   originname,
+											   sizeof(originname));
+
+			/* Drop replication origin */
+			replorigin_drop_by_name(originname, true, false);
+		}
+
+		/*
+		 * We are safe to remove persisted replication slot and origin data,
+		 * since it's already in SYNCDONE state. They will not be needed
+		 * anymore.
 		 */
+		UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+							  MyLogicalRepWorker->relid,
+							  MyLogicalRepWorker->relstate,
+							  MyLogicalRepWorker->relstate_lsn,
+							  NULL,
+							  NULL);
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid))));
+
+		CommitTransactionCommand();
+		pgstat_report_stat(false);
+
 		StartTransactionCommand();
 
+		/*
+		 * This should return the default origin name for the worker. Even if
+		 * the worker used a different origin for this table, it should be
+		 * dropped and removed from the catalog so far.
+		 */
 		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
 										   MyLogicalRepWorker->relid,
 										   originname,
 										   sizeof(originname));
 
 		/*
-		 * Resetting the origin session removes the ownership of the slot.
-		 * This is needed to allow the origin to be dropped.
+		 * Check if any table whose relation state is still INIT. If a table
+		 * in INIT state is found, the worker will not be finished, it will be
+		 * reused instead.
 		 */
-		replorigin_session_reset();
-		replorigin_session_origin = InvalidRepOriginId;
-		replorigin_session_origin_lsn = InvalidXLogRecPtr;
-		replorigin_session_origin_timestamp = 0;
+		rstates = GetSubscriptionRelations(MySubscription->oid, true);
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * There is a chance that the user is concurrently performing refresh
-		 * for the subscription where we remove the table state and its origin
-		 * or the apply worker would have removed this origin. So passing
-		 * missing_ok = true.
-		 */
-		replorigin_drop_by_name(originname, true, false);
+		foreach(lc, rstates)
+		{
+			rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
+			memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
+
+			/*
+			 * Pick the table for the next run if there is not another worker
+			 * already picked that table.
+			 */
+			LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+			if (rstate->state != SUBREL_STATE_SYNCDONE &&
+				!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+			{
+				/* Update worker state for the next table */
+				MyLogicalRepWorker->is_first_run = false;
+				MyLogicalRepWorker->relid = rstate->relid;
+				MyLogicalRepWorker->relstate = rstate->state;
+				MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+				MyLogicalRepWorker->move_to_next_rel = true;
+				LWLockRelease(LogicalRepWorkerLock);
+				break;
+			}
+			LWLockRelease(LogicalRepWorkerLock);
+		}
+
+		/* Cleanup before next run or ending the worker. */
+		if (!MyLogicalRepWorker->move_to_next_rel)
+		{
+			/*
+			 * It is important to give an error if we are unable to drop the
+			 * slot, otherwise, it won't be dropped till the corresponding
+			 * subscription is dropped. So passing missing_ok = false.
+			 */
+			if (MyLogicalRepWorker->created_slot)
+			{
+				/* End streaming if it's not already ended. */
+				if (!is_streaming_ended)
+					walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+				ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
+			}
 
-		finish_sync_worker();
+			/* Drop replication origin before exiting. */
+			replorigin_drop_by_name(originname, true, false);
+
+			finish_sync_worker();
+		}
+		else
+		{
+			clean_sync_worker();
+		}
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -464,6 +555,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 			if (current_lsn >= rstate->lsn)
 			{
 				char		originname[NAMEDATALEN];
+				bool		is_origin_null = true;
 
 				rstate->state = SUBREL_STATE_READY;
 				rstate->lsn = current_lsn;
@@ -484,18 +576,27 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				 * error while dropping we won't restart it to drop the
 				 * origin. So passing missing_ok = true.
 				 */
-				ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-												   rstate->relid,
-												   originname,
-												   sizeof(originname));
-				replorigin_drop_by_name(originname, true, false);
+				GetSubscriptionRelOrigin(MyLogicalRepWorker->subid,
+										 rstate->relid, originname,
+										 &is_origin_null);
+
+				if (!is_origin_null)
+				{
+					replorigin_drop_by_name(originname, true, false);
+				}
 
 				/*
 				 * Update the state to READY only after the origin cleanup.
 				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-										   rstate->relid, rstate->state,
-										   rstate->lsn);
+				UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+									  rstate->relid,
+									  rstate->state,
+									  rstate->lsn,
+									  NULL,
+									  NULL);
+
+				CommitTransactionCommand();
+				started_tx = false;
 			}
 		}
 		else
@@ -584,12 +685,22 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						TimestampDifferenceExceeds(hentry->last_start_time, now,
 												   wal_retrieve_retry_interval))
 					{
+						if (IsTransactionState())
+							CommitTransactionCommand();
+						StartTransactionCommand();
+						started_tx = true;
+
+						MySubscription->lastusedid++;
+						UpdateSubscriptionLastSlotId(MyLogicalRepWorker->subid,
+													 MySubscription->lastusedid);
+
 						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
 												 MySubscription->oid,
 												 MySubscription->name,
 												 MyLogicalRepWorker->userid,
 												 rstate->relid,
-												 DSM_HANDLE_INVALID);
+												 DSM_HANDLE_INVALID,
+												 MySubscription->lastusedid);
 						hentry->last_start_time = now;
 					}
 				}
@@ -1198,8 +1309,8 @@ copy_table(Relation rel)
  * The name must not exceed NAMEDATALEN - 1 because of remote node constraints
  * on slot name length. We append system_identifier to avoid slot_name
  * collision with subscriptions in other clusters. With the current scheme
- * pg_%u_sync_%u_UINT64_FORMAT (3 + 10 + 6 + 10 + 20 + '\0'), the maximum
- * length of slot_name will be 50.
+ * pg_%u_sync_%lu_UINT64_FORMAT (3 + 10 + 6 + 20 + 20 + '\0'), the maximum
+ * length of slot_name will be 45.
  *
  * The returned slot name is stored in the supplied buffer (syncslotname) with
  * the given size.
@@ -1210,11 +1321,11 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
+ReplicationSlotNameForTablesync(Oid suboid, int64 slotid,
 								char *syncslotname, Size szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%lld_" UINT64_FORMAT, suboid,
+			(long long) slotid, GetSystemIdentifier());
 }
 
 /*
@@ -1237,6 +1348,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	WalRcvExecResult *res;
 	char		originname[NAMEDATALEN];
 	RepOriginId originid;
+	char	   *prev_slotname;
 
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
@@ -1265,7 +1377,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
+									MyLogicalRepWorker->rep_slot_id,
 									slotname,
 									NAMEDATALEN);
 
@@ -1285,12 +1397,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC ||
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY);
 
+	/*
+	 * See if tablesync of the current relation has been started with another
+	 * replication slot.
+	 *
+	 * Read previous slot name from the catalog, if exists.
+	 */
+	prev_slotname = (char *) palloc0(NAMEDATALEN);
+	StartTransactionCommand();
+	GetSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+									  MyLogicalRepWorker->relid,
+									  prev_slotname);
+
 	/* Assign the origin tracking record name. */
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   sizeof(originname));
 
+	CommitTransactionCommand();
+
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC)
 	{
 		/*
@@ -1304,10 +1430,48 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		 * breakdown then it wouldn't have succeeded so trying it next time
 		 * seems like a better bet.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, slotname, true);
+		if (strlen(prev_slotname) > 0)
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);
+
+			StartTransactionCommand();
+			/* Replication drop might still exist. Try to drop */
+			replorigin_drop_by_name(originname, true, false);
+
+			/*
+			 * Remove replication slot and origin name from the relation's
+			 * catalog record
+			 */
+			UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+								  MyLogicalRepWorker->relid,
+								  MyLogicalRepWorker->relstate,
+								  MyLogicalRepWorker->relstate_lsn,
+								  NULL,
+								  NULL);
+			CommitTransactionCommand();
+		}
 	}
 	else if (MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY)
 	{
+		/*
+		 * At this point, the table that is currently being synchronized
+		 * should have its replication slot name filled in the catalog. The
+		 * tablesync process was started with another sync worker and
+		 * replication slot. We need to continue using the same replication
+		 * slot in this worker too.
+		 */
+		if (strlen(prev_slotname) == 0)
+		{
+			elog(ERROR, "Replication slot could not be found for relation %u",
+				 MyLogicalRepWorker->relid);
+		}
+
+		/*
+		 * Proceed with the correct replication slot. Use previously created
+		 * replication slot to sync this table.
+		 */
+		slotname = prev_slotname;
+
 		/*
 		 * The COPY phase was previously done, but tablesync then crashed
 		 * before it was able to finish normally.
@@ -1328,6 +1492,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		goto copy_table_done;
 	}
 
+	/* Preparing for table copy operation */
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 	MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
 	MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
@@ -1335,10 +1500,12 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   MyLogicalRepWorker->relstate,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  MyLogicalRepWorker->relstate,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 	CommitTransactionCommand();
 	pgstat_report_stat(true);
 
@@ -1377,6 +1544,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						GetUserNameFromId(GetUserId(), true),
 						RelationGetRelationName(rel))));
 
+
 	/*
 	 * Start a transaction in the remote node in REPEATABLE READ mode.  This
 	 * ensures that both the replication slot we create (see below) and the
@@ -1392,48 +1560,91 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						res->err)));
 	walrcv_clear_result(res);
 
+	originid = replorigin_by_name(originname, true);
+
 	/*
 	 * Create a new permanent logical decoding slot. This slot will be used
 	 * for the catchup phase after COPY is done, so tell it to use the
 	 * snapshot to make the final data consistent.
+	 *
+	 * Replication slot will only be created if either this is the first run
+	 * of the worker or we're not using a previous replication slot.
 	 */
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-
-	/*
-	 * Setup replication origin tracking. The purpose of doing this before the
-	 * copy is to avoid doing the copy again due to any error in setting up
-	 * origin tracking.
-	 */
-	originid = replorigin_by_name(originname, true);
-	if (!OidIsValid(originid))
+	if (!MyLogicalRepWorker->created_slot)
 	{
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						   slotname, false /* permanent */ , false /* two_phase */ ,
+						   CRS_USE_SNAPSHOT, origin_startpos);
+
 		/*
-		 * Origin tracking does not exist, so create it now.
-		 *
-		 * Then advance to the LSN got from walrcv_create_slot. This is WAL
-		 * logged for the purpose of recovery. Locks are to prevent the
-		 * replication origin from vanishing while advancing.
+		 * Remember that we created the slot so that we will not try to create
+		 * it again.
 		 */
-		originid = replorigin_create(originname);
-
-		LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
-		replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
-						   true /* go backward */ , true /* WAL log */ );
-		UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->created_slot = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
+		/*
+		 * Setup replication origin tracking. The purpose of doing this before
+		 * the copy is to avoid doing the copy again due to any error in
+		 * setting up origin tracking.
+		 */
+		if (!OidIsValid(originid))
+		{
+			/*
+			 * Origin tracking does not exist, so create it now.
+			 */
+			originid = replorigin_create(originname);
+		}
+		else
+		{
+			/*
+			 * At this point, there shouldn't be any existing replication
+			 * origin with the same name.
+			 */
+			ereport(ERROR,
+					(errcode(ERRCODE_DUPLICATE_OBJECT),
+					 errmsg("replication origin \"%s\" already exists",
+							originname)));
+		}
 	}
 	else
 	{
-		ereport(ERROR,
-				(errcode(ERRCODE_DUPLICATE_OBJECT),
-				 errmsg("replication origin \"%s\" already exists",
-						originname)));
+		/*
+		 * Do not create a new replication slot, reuse the existing one
+		 * instead. Use a new snapshot for the replication slot to ensure that
+		 * tablesync and apply proceses are consistent with each other.
+		 */
+		WalRcvStreamOptions options;
+		int			server_version;
+
+		server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+		options.proto.logical.proto_version =
+			server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+			server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+			LOGICALREP_PROTO_VERSION_NUM;
+		options.proto.logical.publication_names = MySubscription->publications;
+
+		HOLD_INTERRUPTS();
+		walrcv_slot_snapshot(LogRepWorkerWalRcvConn, slotname, &options, origin_startpos);
+		RESUME_INTERRUPTS();
 	}
 
+	/*
+	 * Advance to the LSN got from walrcv_create_slot. This is WAL
+	 * logged for the purpose of recovery. Locks are to prevent the
+	 * replication origin from vanishing while advancing.
+	 *
+	 * Then setup replication origin tracking.
+	 */
+	LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+	replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
+					   true /* go backward */ , true /* WAL log */ );
+	UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+
 	/* Now do the initial data copy */
 	PushActiveSnapshot(GetTransactionSnapshot());
 	copy_table(rel);
@@ -1456,10 +1667,12 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	 * Update the persisted state to indicate the COPY phase is done; make it
 	 * visible to others.
 	 */
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   SUBREL_STATE_FINISHEDCOPY,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  SUBREL_STATE_FINISHEDCOPY,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 
 	CommitTransactionCommand();
 
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index cfb2ab6248..c12924e4da 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -388,6 +388,7 @@ static void stream_open_file(Oid subid, TransactionId xid,
 static void stream_write_change(char action, StringInfo s);
 static void stream_open_and_write_change(TransactionId xid, char action, StringInfo s);
 static void stream_close_file(void);
+static void stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
@@ -456,13 +457,22 @@ ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 {
 	if (OidIsValid(relid))
 	{
-		/* Replication origin name for tablesync workers. */
-		snprintf(originname, szoriginname, "pg_%u_%u", suboid, relid);
+		bool		is_null = true;
+
+		/*
+		 * Replication origin name for tablesync workers. First, look into the
+		 * catalog. If originname does not exist, then use the default name.
+		 */
+		GetSubscriptionRelOrigin(suboid, relid,
+								 originname, &is_null);
+		if (is_null)
+			snprintf(originname, szoriginname, "pg_%u_%lld", suboid, (long long) MyLogicalRepWorker->rep_slot_id);
 	}
 	else
 	{
 		/* Replication origin name for non-tablesync workers. */
 		snprintf(originname, szoriginname, "pg_%u", suboid);
+		elog(LOG, "apply worker originname %s", originname);
 	}
 }
 
@@ -3576,6 +3586,23 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can move to next table while running
+				 * process_syncing_tables_for_sync. Before we were able to
+				 * reuse tablesync workers, that
+				 * process_syncing_tables_for_sync call would exit the worker
+				 * instead of moving to next table. Now that tablesync workers
+				 * can be reused, we need to take care of memory contexts here
+				 * before moving to sync a table.
+				 */
+				if (MyLogicalRepWorker->move_to_next_rel)
+				{
+					MemoryContextResetAndDeleteChildren(ApplyMessageContext);
+					MemoryContextSwitchTo(TopMemoryContext);
+					return;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -3595,6 +3622,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+			if (MyLogicalRepWorker->move_to_next_rel)
+			{
+				endofstream = true;
+			}
 		}
 
 		/* Cleanup the memory. */
@@ -3697,8 +3728,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * If it's moving to next relation, this is a sync worker. Sync workers
+	 * end the streaming during process_syncing_tables_for_sync. Calling
+	 * endstreaming twice causes "no COPY in progress" errors.
+	 */
+	if (!MyLogicalRepWorker->move_to_next_rel)
+	{
+		/* All done */
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	}
 }
 
 /*
@@ -4282,6 +4321,56 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+ /* stream_build_options
+  * 	Build logical replication streaming options.
+  *
+  * This function sets streaming options including replication slot name
+  * and origin start position. Workers need these options for logical replication.
+  */
+static void
+stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4356,6 +4445,9 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 
 	/* allocate slot name in long-lived context */
 	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+
+	/* Keep the replication slot name used for this sync. */
+	MyLogicalRepWorker->slot_name = *myslotname;
 	pfree(syncslotname);
 }
 
@@ -4393,6 +4485,135 @@ start_apply(XLogRecPtr origin_startpos)
 	PG_END_TRY();
 }
 
+/*
+ * Runs the tablesync worker.
+ * It starts table sync. After successful sync,
+ * builds streaming options and starts streaming.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Set this to false for safety, in case we're already reusing the worker */
+	MyLogicalRepWorker->move_to_next_rel = false;
+
+	start_table_sync(origin_startpos, &slotname);
+
+	/*
+	 * Allocate the origin name in long-lived context for error context
+	 * message.
+	 */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   originname_size);
+	CommitTransactionCommand();
+
+	set_apply_error_context_origin(originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/* Start normal logical streaming replication. */
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+}
+
+/*
+ * Runs the apply worker.
+ * It sets up replication origin, the streaming options
+ * and then starts streaming.
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
+{
+	/* This is the leader apply worker */
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+
+	slotname = MySubscription->slotname;
+
+	/*
+		* This shouldn't happen if the subscription is enabled, but guard
+		* against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+		* crash if slot is NULL.)
+		*/
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											MySubscription->name, &err);
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+		* We don't really use the output identify_system for anything but it
+		* does some initializations on the upstream so let's still call it.
+		*/
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+	{
+		/* Start streaming with two_phase enabled */
+		options->proto.logical.twophase = true;
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
+	}
+	else
+	{
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+	}
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+}
+
 /*
  * Common initialization for leader apply worker and parallel apply worker.
  *
@@ -4485,7 +4706,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	/* Attach to slot */
 	logicalrep_worker_attach(worker_slot);
@@ -4513,156 +4733,48 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
-	 * the subscription relation state.
+	 * the subscription relation state. Do this outside the loop to avoid
+	 * exceeding MAX_SYSCACHE_CALLBACKS
 	 */
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
-
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
-
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-
 	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 * The loop where worker does its job. It loops until the worker is not
+	 * reused.
 	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
-
-	if (!am_tablesync_worker())
+	while (MyLogicalRepWorker->is_first_run ||
+		   MyLogicalRepWorker->move_to_next_rel)
 	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
+		if (am_tablesync_worker())
 		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
+			/*
+			 * This is a tablesync worker. Start syncing tables before
+			 * starting the apply loop.
+			 */
+			run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 		}
 		else
 		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+			/* This is main apply worker */
+			run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 		}
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-						MySubscription->name,
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-						"?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
-
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+		/* Run the main loop. */
+		start_apply(origin_startpos);
 
+		if (MyLogicalRepWorker->move_to_next_rel)
+		{
+			StartTransactionCommand();
+			ereport(LOG,
+					(errmsg("logical replication table synchronization worker for subscription \"%s\" has moved to sync table \"%s\".",
+							MySubscription->name, get_rel_name(MyLogicalRepWorker->relid))));
+			CommitTransactionCommand();
+		}
+	}
 	proc_exit(0);
 }
 
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index b0f2a1705d..a0ee12e259 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -103,6 +103,9 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
 
 	/* Only publish data originating from the specified origin */
 	text		suborigin BKI_DEFAULT(LOGICALREP_ORIGIN_ANY);
+
+	/* The last used ID to create a replication slot for tablesync */
+	int64		sublastusedid BKI_DEFAULT(0);
 #endif
 } FormData_pg_subscription;
 
@@ -137,6 +140,8 @@ typedef struct Subscription
 	List	   *publications;	/* List of publication names to subscribe to */
 	char	   *origin;			/* Only publish data originating from the
 								 * specified origin */
+	int64		lastusedid;		/* Last used unique ID to create replication
+								 * slots in tablesync */
 } Subscription;
 
 /* Disallow streaming in-progress transactions. */
@@ -157,6 +162,7 @@ typedef struct Subscription
 extern Subscription *GetSubscription(Oid subid, bool missing_ok);
 extern void FreeSubscription(Subscription *sub);
 extern void DisableSubscription(Oid subid);
+extern void UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid);
 
 extern int	CountDBSubscriptions(Oid dbid);
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 60a2bcca23..a35d04cccd 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -44,6 +44,12 @@ CATALOG(pg_subscription_rel,6102,SubscriptionRelRelationId)
 											 * used for synchronization
 											 * coordination, or NULL if not
 											 * valid */
+	text		srrelslotname BKI_FORCE_NULL;	/* name of the replication
+												 * slot for relation in
+												 * subscription */
+	text		srreloriginname BKI_FORCE_NULL; /* origin name for relation in
+												 * subscription */
+
 #endif
 } FormData_pg_subscription_rel;
 
@@ -81,10 +87,17 @@ typedef struct SubscriptionRelState
 } SubscriptionRelState;
 
 extern void AddSubscriptionRelState(Oid subid, Oid relid, char state,
-									XLogRecPtr sublsn);
+									XLogRecPtr sublsn, char *relslotname, char *reloriginname);
 extern void UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 									   XLogRecPtr sublsn);
+extern void UpdateSubscriptionRel(Oid subid, Oid relid, char state,
+								  XLogRecPtr sublsn, char *relslotname, char *reloriginname);
+extern void UpdateSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *relslotname);
+
 extern char GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn);
+extern void GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname);
+extern void GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull);
+
 extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 
 extern bool HasSubscriptionRelations(Oid subid);
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 8872c80cdf..3547daaaec 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -219,8 +219,9 @@ extern bool InvalidateObsoleteReplicationSlots(XLogSegNo oldestSegno);
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
 extern bool ReplicationSlotName(int index, Name name);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, Size szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, int64 slotid, char *syncslotname, Size szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List *GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index dc87a4edd1..5f4b7b1f7c 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -35,6 +35,26 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/*
+	 * Indicates if worker is running for the first time or in reuse
+	 */
+	bool		is_first_run;
+
+	/*
+	 * Indicates if the sync worker created a replication slot or it reuses an
+	 * existing one created by another worker.
+	 */
+	bool		created_slot;
+
+	/*
+	 * Unique identifier for replication slot to be created by tablesnync
+	 * workers, if needed.
+	 */
+	int64		rep_slot_id;
+
+	/* Replication slot name used by the worker. */
+	char	   *slot_name;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
@@ -56,6 +76,12 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/*
+	 * Used to indicate whether sync worker will be reused for another
+	 * relation
+	 */
+	bool		move_to_next_rel;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction or
@@ -231,7 +257,8 @@ extern LogicalRepWorker *logicalrep_worker_find(Oid subid, Oid relid,
 extern List *logicalrep_workers_find(Oid subid, bool only_running);
 extern bool logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname,
 									 Oid userid, Oid relid,
-									 dsm_handle subworker_dsm);
+									 dsm_handle subworker_dsm,
+									 int64 slotid);
 extern void logicalrep_worker_stop(Oid subid, Oid relid);
 extern void logicalrep_pa_worker_stop(int slot_no, uint16 generation);
 extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/test/regress/expected/misc_sanity.out b/src/test/regress/expected/misc_sanity.out
index a57fd142a9..3d34a21421 100644
--- a/src/test/regress/expected/misc_sanity.out
+++ b/src/test/regress/expected/misc_sanity.out
@@ -47,20 +47,22 @@ WHERE c.oid < 16384 AND
       relkind = 'r' AND
       attstorage != 'p'
 ORDER BY 1, 2;
-         relname         |    attname    |   atttypid   
--------------------------+---------------+--------------
- pg_attribute            | attacl        | aclitem[]
- pg_attribute            | attfdwoptions | text[]
- pg_attribute            | attmissingval | anyarray
- pg_attribute            | attoptions    | text[]
- pg_class                | relacl        | aclitem[]
- pg_class                | reloptions    | text[]
- pg_class                | relpartbound  | pg_node_tree
- pg_index                | indexprs      | pg_node_tree
- pg_index                | indpred       | pg_node_tree
- pg_largeobject          | data          | bytea
- pg_largeobject_metadata | lomacl        | aclitem[]
-(11 rows)
+         relname         |     attname     |   atttypid   
+-------------------------+-----------------+--------------
+ pg_attribute            | attacl          | aclitem[]
+ pg_attribute            | attfdwoptions   | text[]
+ pg_attribute            | attmissingval   | anyarray
+ pg_attribute            | attoptions      | text[]
+ pg_class                | relacl          | aclitem[]
+ pg_class                | reloptions      | text[]
+ pg_class                | relpartbound    | pg_node_tree
+ pg_index                | indexprs        | pg_node_tree
+ pg_index                | indpred         | pg_node_tree
+ pg_largeobject          | data            | bytea
+ pg_largeobject_metadata | lomacl          | aclitem[]
+ pg_subscription_rel     | srreloriginname | text
+ pg_subscription_rel     | srrelslotname   | text
+(13 rows)
 
 -- system catalogs without primary keys
 --
-- 
2.25.1

#38Melih Mutlu
m.melihmutlu@gmail.com
In reply to: shveta malik (#33)
2 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi,

I mistakenly attached v9 in my previous email.
Please see attached v6 and v10 for the previous and below changes.

shveta malik <shveta.malik@gmail.com>, 31 Oca 2023 Sal, 12:59 tarihinde
şunu yazdı:

On Fri, Jan 27, 2023 at 3:41 PM shveta malik <shveta.malik@gmail.com>
wrote:
1)
REPLICATION_SLOT_SNAPSHOT
--Do we need 'CREATE' prefix with it i.e. CREATE_REPLICATION_SNAPSHOT
(or some other brief one with CREATE?). 'REPLICATION_SLOT_SNAPSHOT'
does not look like a command/action and thus is confusing.

Renamed it as CREATE_REPLICATION_SNAPSHOT

2)
is used in the currenct transaction. This command is currently only
supported
for logical replication.
slots.
--typo: currenct-->current
--slots can be moved to previous line

Done.

3)
/*
* Signal that we don't need the timeout mechanism. We're just creating
* the replication slot and don't yet accept feedback messages or send
* keepalives. As we possibly need to wait for further WAL the walsender
* would otherwise possibly be killed too soon.
*/
We're just creating the replication slot --> We're just creating the
replication snapshot

Done.

4)
I see XactReadOnly check in CreateReplicationSlot, do we need the same
in ReplicationSlotSnapshot() as well?

Added this check too.

===============
v9-0002:

5)
/* We are safe to drop the replication trackin origin after this
--typo: tracking

Done.

6)
slot->data.catalog_xmin = xmin_horizon;
slot->effective_xmin = xmin_horizon;
SpinLockRelease(&slot->mutex);
xmin_horizon =
GetOldestSafeDecodingTransactionId(!need_full_snapshot);
ReplicationSlotsComputeRequiredXmin(true);

--do we need to set xmin_horizon in slot after
'GetOldestSafeDecodingTransactionId' call, otherwise it will be set to
InvalidId in slot. Is that intentional? I see that we do set this
correct xmin_horizon in builder->initial_xmin_horizon but the slot is
carrying Invalid one.

I think you're right. Moved GetOldestSafeDecodingTransactionId call before
xmin_horizon assignment.

Thanks,
--
Melih Mutlu
Microsoft

Attachments:

v6-0001-Add-replication-protocol-cmd-to-create-a-snapshot.patchapplication/octet-stream; name=v6-0001-Add-replication-protocol-cmd-to-create-a-snapshot.patchDownload
From 6b9a115dd35ee2e728abf8ee0a61e06c78a867b3 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 13 Oct 2022 17:05:45 +0300
Subject: [PATCH 1/2] Add replication protocol cmd to create a snapshot

Introduced CREATE_REPLICATION_SNAPSHOT to be able to create and use a
snapshot without creating a new replication slot, but by using an
existing slot.

CREATE_REPLICATION_SNAPSHOT simply does what CREATE_REPLICATION_SLOT does
without creating a new replication slot.

CREATE_REPLICATION_SNAPSHOT command imports the snapshot into the current
transaction and returns consistent_point. The changes earlier than the
consistent_point will be applied by importing the snapshot. All changes
later than the consistent_point will be available to be consumed from
the replication slot.

This is useful for reusing replication slots in logical replication.
Otherwise, tablesync workers cannot start from a consistent point to copy a relation and then apply changes by consuming from replication slot.
---
 doc/src/sgml/protocol.sgml                    |  31 ++++++
 .../libpqwalreceiver/libpqwalreceiver.c       |  69 +++++++++++-
 src/backend/replication/logical/logical.c     |  40 ++++++-
 .../replication/logical/logicalfuncs.c        |   1 +
 src/backend/replication/repl_gram.y           |  18 +++-
 src/backend/replication/repl_scanner.l        |   2 +
 src/backend/replication/slotfuncs.c           |   1 +
 src/backend/replication/walsender.c           | 102 +++++++++++++++++-
 src/include/nodes/replnodes.h                 |  11 ++
 src/include/replication/logical.h             |   1 +
 src/include/replication/walreceiver.h         |  13 +++
 src/tools/pgindent/typedefs.list              |   2 +
 12 files changed, 287 insertions(+), 4 deletions(-)

diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 93fc7167d4..948a5aabd4 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2613,6 +2613,37 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
      </listitem>
     </varlistentry>
 
+    <varlistentry id="protocol-replication-replication-slot-snapshot">
+     <term><literal>CREATE_REPLICATION_SNAPSHOT</literal> <replaceable class="parameter">slot_name</replaceable> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
+      <indexterm><primary>CREATE_REPLICATION_SNAPSHOT</primary></indexterm>
+     </term>
+     <listitem>
+      <para>
+       Creates a snapshot including all the changes from the replication slot until
+       the point at which the replication slot becomes consistent. Then the snapshot
+       is used in the current transaction. This command is currently only supported
+       for logical replication slots.
+      </para>
+
+      <para>
+       In response to this command, the server will return a one-row result set,
+       containing the following field:
+       <variablelist>
+        <varlistentry>
+         <term><literal>consistent_point</literal> (<type>text</type>)</term>
+         <listitem>
+          <para>
+           The WAL location at which the slot became consistent.  This is the
+           earliest location from which streaming can start on this replication
+           slot.
+          </para>
+         </listitem>
+        </varlistentry>
+       </variablelist>
+      </para>
+     </listitem>
+    </varlistentry>
+
     <varlistentry id="protocol-replication-base-backup" xreflabel="BASE_BACKUP">
      <term><literal>BASE_BACKUP</literal> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
       <indexterm><primary>BASE_BACKUP</primary></indexterm>
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index 560ec974fa..9173163a07 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -81,6 +81,8 @@ static WalRcvExecResult *libpqrcv_exec(WalReceiverConn *conn,
 									   const int nRetTypes,
 									   const Oid *retTypes);
 static void libpqrcv_disconnect(WalReceiverConn *conn);
+static void libpqrcv_slot_snapshot(WalReceiverConn *conn, char *slotname,
+								   const WalRcvStreamOptions *options, XLogRecPtr *lsn);
 
 static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_connect = libpqrcv_connect,
@@ -97,7 +99,8 @@ static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_create_slot = libpqrcv_create_slot,
 	.walrcv_get_backend_pid = libpqrcv_get_backend_pid,
 	.walrcv_exec = libpqrcv_exec,
-	.walrcv_disconnect = libpqrcv_disconnect
+	.walrcv_disconnect = libpqrcv_disconnect,
+	.walrcv_slot_snapshot = libpqrcv_slot_snapshot
 };
 
 /* Prototypes for private functions */
@@ -933,6 +936,70 @@ libpqrcv_create_slot(WalReceiverConn *conn, const char *slotname,
 	return snapshot;
 }
 
+/*
+ * TODO
+ */
+static void
+libpqrcv_slot_snapshot(WalReceiverConn *conn,
+					   char *slotname,
+					   const WalRcvStreamOptions *options,
+					   XLogRecPtr *lsn)
+{
+	StringInfoData cmd;
+	PGresult   *res;
+	char	   *pubnames_str;
+	List	   *pubnames;
+	char	   *pubnames_literal;
+
+	initStringInfo(&cmd);
+
+	/* Build the command. */
+	appendStringInfo(&cmd, "CREATE_REPLICATION_SNAPSHOT \"%s\"", slotname);
+	appendStringInfoString(&cmd, " (");
+	appendStringInfo(&cmd, " proto_version '%u'",
+					 options->proto.logical.proto_version);
+
+	/* Add publication names. */
+	pubnames = options->proto.logical.publication_names;
+	pubnames_str = stringlist_to_identifierstr(conn->streamConn, pubnames);
+	if (!pubnames_str)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	pubnames_literal = PQescapeLiteral(conn->streamConn, pubnames_str,
+									   strlen(pubnames_str));
+	if (!pubnames_literal)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	appendStringInfo(&cmd, ", publication_names %s", pubnames_literal);
+	PQfreemem(pubnames_literal);
+	pfree(pubnames_str);
+
+	appendStringInfoString(&cmd, " )");
+
+	/* Execute the command. */
+	res = libpqrcv_PQexec(conn->streamConn, cmd.data);
+	pfree(cmd.data);
+
+	if (PQresultStatus(res) != PGRES_TUPLES_OK)
+	{
+		PQclear(res);
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg("Could not create a snapshot by replication slot \"%s\": %s",
+						slotname, pchomp(PQerrorMessage(conn->streamConn)))));
+	}
+
+	if (lsn)
+		*lsn = DatumGetLSN(DirectFunctionCall1Coll(pg_lsn_in, InvalidOid,
+												   CStringGetDatum(PQgetvalue(res, 0, 0))));
+
+	PQclear(res);
+}
+
 /*
  * Return PID of remote backend process.
  */
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 1a58dd7649..d1504f7ec6 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -461,6 +461,10 @@ CreateInitDecodingContext(const char *plugin,
  * fast_forward
  *		bypass the generation of logical changes.
  *
+ * need_full_snapshot
+ * 		if true, create a snapshot able to read all tables,
+ * 		otherwise do not create any snapshot.
+ *
  * xl_routine
  *		XLogReaderRoutine used by underlying xlogreader
  *
@@ -479,6 +483,7 @@ LogicalDecodingContext *
 CreateDecodingContext(XLogRecPtr start_lsn,
 					  List *output_plugin_options,
 					  bool fast_forward,
+					  bool need_full_snapshot,
 					  XLogReaderRoutine *xl_routine,
 					  LogicalOutputPluginWriterPrepareWrite prepare_write,
 					  LogicalOutputPluginWriterWrite do_write,
@@ -487,6 +492,7 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 	LogicalDecodingContext *ctx;
 	ReplicationSlot *slot;
 	MemoryContext old_context;
+	TransactionId xmin_horizon = InvalidTransactionId;
 
 	/* shorter lines... */
 	slot = MyReplicationSlot;
@@ -533,8 +539,40 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 		start_lsn = slot->data.confirmed_flush;
 	}
 
+
+	/*
+	 * We need to determine a safe xmin horizon to start decoding from if we
+	 * want to create a snapshot too. Otherwise we would end up with a
+	 * snapshot that cannot be imported since xmin value from the snapshot may
+	 * be less than the oldest safe xmin. To avoid this call
+	 * GetOldestSafeDecodingTransactionId() to return a safe xmin value, which
+	 * can be used while exporting/importing the snapshot.
+	 *
+	 * So we have to acquire the ProcArrayLock to prevent computation of new
+	 * xmin horizons by other backends, get the safe decoding xid, and inform
+	 * the slot machinery about the new limit. Once that's done the
+	 * ProcArrayLock can be released as the slot machinery now is protecting
+	 * against vacuum.
+	 */
+	if (need_full_snapshot)
+	{
+		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+		xmin_horizon = GetOldestSafeDecodingTransactionId(!need_full_snapshot);
+
+		SpinLockAcquire(&slot->mutex);
+		slot->effective_catalog_xmin = xmin_horizon;
+		slot->data.catalog_xmin = xmin_horizon;
+		slot->effective_xmin = xmin_horizon;
+		SpinLockRelease(&slot->mutex);
+
+		ReplicationSlotsComputeRequiredXmin(true);
+
+		LWLockRelease(ProcArrayLock);
+	}
+
 	ctx = StartupDecodingContext(output_plugin_options,
-								 start_lsn, InvalidTransactionId, false,
+								 start_lsn, xmin_horizon, need_full_snapshot,
 								 fast_forward, xl_routine, prepare_write,
 								 do_write, update_progress);
 
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index fa1b641a2b..1191c70eb0 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -208,6 +208,7 @@ pg_logical_slot_get_changes_guts(FunctionCallInfo fcinfo, bool confirm, bool bin
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									options,
 									false,
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/repl_gram.y b/src/backend/replication/repl_gram.y
index 0c874e33cf..957a5cc653 100644
--- a/src/backend/replication/repl_gram.y
+++ b/src/backend/replication/repl_gram.y
@@ -65,6 +65,7 @@ Node *replication_parse_result;
 %token K_CREATE_REPLICATION_SLOT
 %token K_DROP_REPLICATION_SLOT
 %token K_TIMELINE_HISTORY
+%token K_CREATE_REPLICATION_SNAPSHOT
 %token K_WAIT
 %token K_TIMELINE
 %token K_PHYSICAL
@@ -80,7 +81,7 @@ Node *replication_parse_result;
 %type <node>	command
 %type <node>	base_backup start_replication start_logical_replication
 				create_replication_slot drop_replication_slot identify_system
-				read_replication_slot timeline_history show
+				read_replication_slot timeline_history show create_replication_snapshot
 %type <list>	generic_option_list
 %type <defelt>	generic_option
 %type <uintval>	opt_timeline
@@ -114,6 +115,7 @@ command:
 			| read_replication_slot
 			| timeline_history
 			| show
+			| create_replication_snapshot
 			;
 
 /*
@@ -307,6 +309,19 @@ timeline_history:
 				}
 			;
 
+/*
+ * CREATE_REPLICATION_SNAPSHOT %s options
+ */
+create_replication_snapshot:
+			K_CREATE_REPLICATION_SNAPSHOT var_name plugin_options
+				{
+					CreateReplicationSnapshotCmd *n = makeNode(CreateReplicationSnapshotCmd);
+					n->slotname = $2;
+					n->options = $3;
+					$$ = (Node *) n;
+				}
+			;
+
 opt_physical:
 			K_PHYSICAL
 			| /* EMPTY */
@@ -400,6 +415,7 @@ ident_or_keyword:
 			| K_CREATE_REPLICATION_SLOT	{ $$ = "create_replication_slot"; }
 			| K_DROP_REPLICATION_SLOT		{ $$ = "drop_replication_slot"; }
 			| K_TIMELINE_HISTORY			{ $$ = "timeline_history"; }
+			| K_CREATE_REPLICATION_SNAPSHOT	{ $$ = "create_replication_snapshot"; }
 			| K_WAIT						{ $$ = "wait"; }
 			| K_TIMELINE					{ $$ = "timeline"; }
 			| K_PHYSICAL					{ $$ = "physical"; }
diff --git a/src/backend/replication/repl_scanner.l b/src/backend/replication/repl_scanner.l
index cb467ca46f..5ba2e9c54b 100644
--- a/src/backend/replication/repl_scanner.l
+++ b/src/backend/replication/repl_scanner.l
@@ -126,6 +126,7 @@ START_REPLICATION	{ return K_START_REPLICATION; }
 CREATE_REPLICATION_SLOT		{ return K_CREATE_REPLICATION_SLOT; }
 DROP_REPLICATION_SLOT		{ return K_DROP_REPLICATION_SLOT; }
 TIMELINE_HISTORY	{ return K_TIMELINE_HISTORY; }
+CREATE_REPLICATION_SNAPSHOT	{ return K_CREATE_REPLICATION_SNAPSHOT; }
 PHYSICAL			{ return K_PHYSICAL; }
 RESERVE_WAL			{ return K_RESERVE_WAL; }
 LOGICAL				{ return K_LOGICAL; }
@@ -303,6 +304,7 @@ replication_scanner_is_replication_command(void)
 		case K_DROP_REPLICATION_SLOT:
 		case K_READ_REPLICATION_SLOT:
 		case K_TIMELINE_HISTORY:
+		case K_CREATE_REPLICATION_SNAPSHOT:
 		case K_SHOW:
 			/* Yes; push back the first token so we can parse later. */
 			repl_pushed_back_token = first_token;
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 2f3c964824..b3ae11b2c8 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -478,6 +478,7 @@ pg_logical_replication_slot_advance(XLogRecPtr moveto)
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									NIL,
 									true,	/* fast_forward */
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 4ed3747e3f..33e8c63719 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -238,6 +238,7 @@ static void CreateReplicationSlot(CreateReplicationSlotCmd *cmd);
 static void DropReplicationSlot(DropReplicationSlotCmd *cmd);
 static void StartReplication(StartReplicationCmd *cmd);
 static void StartLogicalReplication(StartReplicationCmd *cmd);
+static void CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd);
 static void ProcessStandbyMessage(void);
 static void ProcessStandbyReplyMessage(void);
 static void ProcessStandbyHSFeedbackMessage(void);
@@ -1280,7 +1281,7 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	 * are reported early.
 	 */
 	logical_decoding_ctx =
-		CreateDecodingContext(cmd->startpoint, cmd->options, false,
+		CreateDecodingContext(cmd->startpoint, cmd->options, false, false,
 							  XL_ROUTINE(.page_read = logical_read_xlog_page,
 										 .segment_open = WalSndSegmentOpen,
 										 .segment_close = wal_segment_close),
@@ -1332,6 +1333,96 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	EndCommand(&qc, DestRemote, false);
 }
 
+/*
+ * Create a snapshot from an existing replication slot.
+ */
+static void
+CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd)
+{
+	Snapshot	snap;
+	LogicalDecodingContext *ctx;
+	char		xloc[MAXFNAMELEN];
+	DestReceiver *dest;
+	TupOutputState *tstate;
+	TupleDesc	tupdesc;
+	Datum		values[1];
+	bool		nulls[1] = {0};
+
+	Assert(!MyReplicationSlot);
+
+	if (!IsTransactionBlock())
+		ereport(ERROR,
+				(errmsg("%s must be called inside a transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (XactIsoLevel != XACT_REPEATABLE_READ)
+		ereport(ERROR,
+				(errmsg("%s must be called in REPEATABLE READ isolation mode transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (!XactReadOnly)
+		ereport(ERROR,
+				(errmsg("%s must be called in a read only transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (FirstSnapshotSet)
+		ereport(ERROR,
+				(errmsg("%s must be called before any query",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (IsSubTransaction())
+		ereport(ERROR,
+				(errmsg("%s must not be called in a subtransaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	ReplicationSlotAcquire(cmd->slotname, false);
+
+	ctx = CreateDecodingContext(MyReplicationSlot->data.restart_lsn,
+								cmd->options,
+								false,
+								true,
+								XL_ROUTINE(.page_read = logical_read_xlog_page,
+										   .segment_open = WalSndSegmentOpen,
+										   .segment_close = wal_segment_close),
+								WalSndPrepareWrite, WalSndWriteData,
+								WalSndUpdateProgress);
+
+	/*
+	 * Signal that we don't need the timeout mechanism. We're just creating
+	 * the snapshot with the replication slot and don't yet accept feedback
+	 * messages or send keepalives. As we possibly need to wait for further
+	 * WAL the walsender would otherwise possibly be killed too soon.
+	 */
+	last_reply_timestamp = 0;
+
+	/* build initial snapshot, might take a while */
+	DecodingContextFindStartpoint(ctx);
+
+	snap = SnapBuildInitialSnapshot(ctx->snapshot_builder);
+	RestoreTransactionSnapshot(snap, MyProc);
+
+	/* Don't need the decoding context anymore */
+	FreeDecodingContext(ctx);
+
+	/* Create a tuple to send consisten WAL location */
+	snprintf(xloc, sizeof(xloc), "%X/%X",
+			 LSN_FORMAT_ARGS(MyReplicationSlot->data.confirmed_flush));
+
+	dest = CreateDestReceiver(DestRemoteSimple);
+	tupdesc = CreateTemplateTupleDesc(1);
+	TupleDescInitBuiltinEntry(tupdesc, (AttrNumber) 1, "consistent_point",
+							  TEXTOID, -1, 0);
+	tstate = begin_tup_output_tupdesc(dest, tupdesc, &TTSOpsVirtual);
+
+	/* consistent wal location */
+	values[0] = CStringGetTextDatum(xloc);
+
+	do_tup_output(tstate, values, nulls);
+	end_tup_output(tstate);
+
+	ReplicationSlotRelease();
+}
+
 /*
  * LogicalDecodingContext 'prepare_write' callback.
  *
@@ -1860,6 +1951,15 @@ exec_replication_command(const char *cmd_string)
 			}
 			break;
 
+		case T_CreateReplicationSnapshotCmd:
+			{
+				cmdtag = "CREATE_REPLICATION_SNAPSHOT";
+				set_ps_display(cmdtag);
+				CreateReplicationSnapshot((CreateReplicationSnapshotCmd *) cmd_node);
+				EndReplicationCommand(cmdtag);
+				break;
+			}
+
 		default:
 			elog(ERROR, "unrecognized replication command node tag: %u",
 				 cmd_node->type);
diff --git a/src/include/nodes/replnodes.h b/src/include/nodes/replnodes.h
index 4321ba8f86..154ab74f33 100644
--- a/src/include/nodes/replnodes.h
+++ b/src/include/nodes/replnodes.h
@@ -108,4 +108,15 @@ typedef struct TimeLineHistoryCmd
 	TimeLineID	timeline;
 } TimeLineHistoryCmd;
 
+/* ----------------------
+ *		CREATE_REPLICATION_SNAPSHOT command
+ * ----------------------
+ */
+typedef struct CreateReplicationSnapshotCmd
+{
+	NodeTag		type;
+	char	   *slotname;
+	List	   *options;
+} CreateReplicationSnapshotCmd;
+
 #endif							/* REPLNODES_H */
diff --git a/src/include/replication/logical.h b/src/include/replication/logical.h
index 5f49554ea0..6535786a0e 100644
--- a/src/include/replication/logical.h
+++ b/src/include/replication/logical.h
@@ -125,6 +125,7 @@ extern LogicalDecodingContext *CreateInitDecodingContext(const char *plugin,
 extern LogicalDecodingContext *CreateDecodingContext(XLogRecPtr start_lsn,
 													 List *output_plugin_options,
 													 bool fast_forward,
+													 bool need_full_snapshot,
 													 XLogReaderRoutine *xl_routine,
 													 LogicalOutputPluginWriterPrepareWrite prepare_write,
 													 LogicalOutputPluginWriterWrite do_write,
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index decffe352d..bd11f9f31e 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -384,6 +384,16 @@ typedef WalRcvExecResult *(*walrcv_exec_fn) (WalReceiverConn *conn,
  */
 typedef void (*walrcv_disconnect_fn) (WalReceiverConn *conn);
 
+/*
+ * walrcv_slot_snapshot_fn
+ *
+ * Create a snapshot by an existing replication slot
+ */
+typedef void (*walrcv_slot_snapshot_fn) (WalReceiverConn *conn,
+										 char *slotname,
+										 const WalRcvStreamOptions *options,
+										 XLogRecPtr *lsn);
+
 typedef struct WalReceiverFunctionsType
 {
 	walrcv_connect_fn walrcv_connect;
@@ -401,6 +411,7 @@ typedef struct WalReceiverFunctionsType
 	walrcv_get_backend_pid_fn walrcv_get_backend_pid;
 	walrcv_exec_fn walrcv_exec;
 	walrcv_disconnect_fn walrcv_disconnect;
+	walrcv_slot_snapshot_fn walrcv_slot_snapshot;
 } WalReceiverFunctionsType;
 
 extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
@@ -435,6 +446,8 @@ extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
 	WalReceiverFunctions->walrcv_exec(conn, exec, nRetTypes, retTypes)
 #define walrcv_disconnect(conn) \
 	WalReceiverFunctions->walrcv_disconnect(conn)
+#define walrcv_slot_snapshot(conn, slotname, options, lsn) \
+	WalReceiverFunctions->walrcv_slot_snapshot(conn, slotname, options, lsn)
 
 static inline void
 walrcv_clear_result(WalRcvExecResult *walres)
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 07fbb7ccf6..a3de441b7e 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2323,6 +2323,7 @@ ReplicationSlotCtlData
 ReplicationSlotOnDisk
 ReplicationSlotPersistency
 ReplicationSlotPersistentData
+CreateReplicationSnapshotCmd
 ReplicationState
 ReplicationStateCtl
 ReplicationStateOnDisk
@@ -3796,6 +3797,7 @@ walrcv_receive_fn
 walrcv_send_fn
 walrcv_server_version_fn
 walrcv_startstreaming_fn
+walrcv_slot_snapshot_fn
 wchar2mb_with_len_converter
 wchar_t
 win32_deadchild_waitinfo
-- 
2.25.1

v10-0002-Reuse-Logical-Replication-Background-worker.patchapplication/octet-stream; name=v10-0002-Reuse-Logical-Replication-Background-worker.patchDownload
From 129c9a1ed564714539a6421f14ab6538444ca9d3 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH 2/2] Reuse Logical Replication Background worker

This commit allows reusing tablesync workers for syncing more than one relation sequantially during their lifetime, instead of exiting after only syncing one relation.

Before this commit, tablesync workers were capable of syncing only one
relation. For each table, a sync worker was launched, then a new
replication slot and new origin were created.

Now, tablesync workers are not only limited with one relation and can move to another relation in the same subscription and reuse existing
replication slots and origins

This reduces the overhead of launching/killing a new background worker for each relation.
By reusing tablesync workers, replication slots and origins created for tablesync can be reused as well.
Removing the burden of creating/dropping replication slot/origin improves tablesync speed significantly especially for empty or small tables.

A new tablesync worker gets launched only if the number of tablesync
workers for the subscription does not exceed
max_sync_workers_per_subscription. If there is a table needs to be synced, a tablesync worker picks that up and processes it.The worker continues to picking new tables to sync until there is no table left for synchronization in the subscription.

If the state of the current table is INIT or DATASYNC, tablesync worker needs a
repliation slot/origin. If the worker has not created slot and origin in
its previous runs, it will create them. Otherwise the worker reuses
slot and origin created by itself earlier. Tables in FINISHEDCOPY are
expected to have a replication slot and origin. Tablesync worker proceed
with existing slot and origin of FINISHEDCOPY tables and do not need to
create new ones.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/catalogs.sgml                    |  31 ++
 src/backend/catalog/pg_subscription.c         | 288 +++++++++++-
 src/backend/commands/subscriptioncmds.c       | 226 ++++++----
 .../replication/logical/applyparallelworker.c |   3 +-
 src/backend/replication/logical/launcher.c    |   8 +-
 src/backend/replication/logical/tablesync.c   | 424 +++++++++++++-----
 src/backend/replication/logical/worker.c      | 387 ++++++++++------
 src/include/catalog/pg_subscription.h         |   6 +
 src/include/catalog/pg_subscription_rel.h     |  15 +-
 src/include/replication/slot.h                |   3 +-
 src/include/replication/worker_internal.h     |  26 +-
 11 files changed, 1083 insertions(+), 334 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index c1e4048054..31b4cfbbe4 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8002,6 +8002,19 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        origin.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>sublastusedid</structfield> <type>int8</type>
+      </para>
+      <para>
+      The last used ID for tablesync workers. It acts as an unique identifier
+      for replication slots which are created by tablesync workers.
+      The last used ID needs to be persisted to make logical replication safely
+      proceed after any interruption. If sublastusedid is 0, then no table has
+      been synced yet.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
@@ -8086,6 +8099,24 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        otherwise null
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srrelslotname</structfield> <type>name</type>
+      </para>
+      <para>
+       Replication slot name that is used for synchronization of relation
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srreloriginname</structfield> <type>name</type>
+      </para>
+      <para>
+       Origin name that is used for tracking synchronization of relation
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index a56ae311c3..29a8fc6919 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -114,6 +114,14 @@ GetSubscription(Oid subid, bool missing_ok)
 	Assert(!isnull);
 	sub->origin = TextDatumGetCString(datum);
 
+	/* Get last used id */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
+							tup,
+							Anum_pg_subscription_sublastusedid,
+							&isnull);
+	Assert(!isnull);
+	sub->lastusedid = DatumGetInt64(datum);
+
 	ReleaseSysCache(tup);
 
 	return sub;
@@ -205,6 +213,44 @@ DisableSubscription(Oid subid)
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update the last used replication slot ID for the given subscription.
+ */
+void
+UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid)
+{
+	Relation	rel;
+	bool		nulls[Natts_pg_subscription];
+	bool		replaces[Natts_pg_subscription];
+	Datum		values[Natts_pg_subscription];
+	HeapTuple	tup;
+
+	/* Look up the subscription in the catalog */
+	rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+	tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	/* Form a new tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	replaces[Anum_pg_subscription_sublastusedid - 1] = true;
+	values[Anum_pg_subscription_sublastusedid- 1] = Int64GetDatum(lastusedid);
+
+	/* Update the catalog */
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+	heap_freetuple(tup);
+
+	table_close(rel, NoLock);
+}
+
 /*
  * Convert text array to list of strings.
  *
@@ -234,7 +280,7 @@ textarray_to_stringlist(ArrayType *textarray)
  */
 void
 AddSubscriptionRelState(Oid subid, Oid relid, char state,
-						XLogRecPtr sublsn)
+						XLogRecPtr sublsn, char *relslotname, char *reloriginname)
 {
 	Relation	rel;
 	HeapTuple	tup;
@@ -263,6 +309,16 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
 	else
 		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] = 
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -275,6 +331,60 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Internal function to modify columns for relation state update
+ */
+static void
+UpdateSubscriptionRelState_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char state,
+									XLogRecPtr sublsn)
+{
+	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
+	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+
+	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (sublsn != InvalidXLogRecPtr)
+		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
+	else
+		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication slot update
+ */
+static void
+UpdateSubscriptionRelReplicationSlot_internal(Datum *values,
+											bool *nulls,
+											bool *replaces,
+											char *relslotname)
+{
+	replaces[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication origin update
+ */
+static void
+UpdateSubscriptionRelOrigin_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char *reloriginname)
+{
+	replaces[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+}
+
 /*
  * Update the state of a subscription table.
  */
@@ -305,14 +415,48 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	memset(nulls, false, sizeof(nulls));
 	memset(replaces, false, sizeof(replaces));
 
-	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
-	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
 
-	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
-	if (sublsn != InvalidXLogRecPtr)
-		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
-	else
-		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update the replication slot name of a subscription table.
+ */
+void
+UpdateSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *relslotname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
 
 	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
 							replaces);
@@ -324,6 +468,134 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update replication slot name, origin name and state of
+ * a subscription table in one transaction.
+ */
+void
+UpdateSubscriptionRel(Oid subid,
+					  Oid relid,
+					  char state,
+					  XLogRecPtr sublsn,
+					  char *relslotname,
+					  char *reloriginname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
+	UpdateSubscriptionRelOrigin_internal(values, nulls, replaces, reloriginname);
+
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Get origin name of subscription table.
+ *
+ * reloriginname's value has the replication origin name if the origin exists.
+ */
+void
+GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*originname;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srreloriginname, isnull);
+	if (!*isnull)
+	{
+		originname = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(reloriginname, originname, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
+/*
+ * Get replication slot name of subscription table.
+ *
+ * slotname's value has the replication slot name if the subscription has any.
+ */
+void
+GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*relrepslot;
+	bool		isnull;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srrelslotname, &isnull);
+	if (!isnull)
+	{
+		relrepslot = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(slotname, relrepslot, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
 /*
  * Get state of subscription table.
  *
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 464db6d247..b580e59ffc 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -649,6 +649,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 		publicationListToArray(publications);
 	values[Anum_pg_subscription_suborigin - 1] =
 		CStringGetTextDatum(opts.origin);
+	values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(0);
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -709,7 +710,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 										 rv->schemaname, rv->relname);
 
 				AddSubscriptionRelState(subid, relid, table_state,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 			}
 
 			/*
@@ -799,6 +800,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	} SubRemoveRels;
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -876,7 +879,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				AddSubscriptionRelState(sub->oid, relid,
 										copy_data ? SUBREL_STATE_INIT : SUBREL_STATE_READY,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 				ereport(DEBUG1,
 						(errmsg_internal("table \"%s.%s\" added to subscription \"%s\"",
 										 rv->schemaname, rv->relname, sub->name)));
@@ -900,6 +903,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				char		state;
 				XLogRecPtr	statelsn;
+				char		slotname[NAMEDATALEN] = {0};
 
 				/*
 				 * Lock pg_subscription_rel with AccessExclusiveLock to
@@ -926,7 +930,29 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/*
+				 * Find the logical replication sync worker. If exists, store
+				 * the slot number for dropping associated replication slots
+				 * later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot_name);
+				}
+				else
+				{
+					/*
+					 * Sync of this relation might be failed in an earlier
+					 * attempt, but the replication slot might still exist.
+					 */
+					GetSubscriptionRelReplicationSlot(sub->oid, relid, slotname);
+					if (strlen(slotname) > 0)
+						sub_remove_slots = lappend(sub_remove_slots, slotname);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
@@ -960,31 +986,24 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
-		 * the database operations we won't be able to rollback dropped slots.
+		 * Drop the replication slots associated with tablesync workers for
+		 * removed tables. This has to be at the end because otherwise if
+		 * there is an error while doing the database operations we won't be
+		 * able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			memcpy(syncslotname, lfirst(lc), sizeof(NAMEDATALEN));
+
+			/*
+			 * There is no certainty, maybe the slot does not exist yet. Also,
+			 * if we fail after removing some of the slots, next time, it will
+			 * again try to drop already dropped slots and fail. For these
+			 * reasons, we allow missing_ok = true for the drop.
+			 */
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1384,6 +1403,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	char	   *subname;
 	char	   *conninfo;
 	char	   *slotname;
+	int64		lastusedid;
 	List	   *subworkers;
 	ListCell   *lc;
 	char		originname[NAMEDATALEN];
@@ -1455,6 +1475,14 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	else
 		slotname = NULL;
 
+	/* Get the last used identifier by the subscription */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID, tup,
+							Anum_pg_subscription_sublastusedid, &isnull);
+	if (!isnull)
+		lastusedid = DatumGetInt64(datum);
+	else
+		lastusedid = 0;
+
 	/*
 	 * Since dropping a replication slot is not transactional, the replication
 	 * slot stays dropped even if the transaction rolls back.  So we cannot
@@ -1504,6 +1532,8 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	}
 	list_free(subworkers);
 
+	rstates = GetSubscriptionRelations(subid, true);
+
 	/*
 	 * Remove the no-longer-useful entry in the launcher's table of apply
 	 * worker start times.
@@ -1515,36 +1545,26 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	ApplyLauncherForgetWorkerStartTime(subid);
 
 	/*
-	 * Cleanup of tablesync replication origins.
-	 *
-	 * Any READY-state relations would already have dealt with clean-ups.
+	 * Cleanup of tablesync replication origins associated with the
+	 * subscription, if exists. Try to drop origins by creating all origin
+	 * names created for this subscription.
 	 *
 	 * Note that the state can't change because we have already stopped both
 	 * the apply and tablesync workers and they can't restart because of
 	 * exclusive lock on the subscription.
+	 *
+	 * XXX: This can be handled better instead of looping through all possible
 	 */
-	rstates = GetSubscriptionRelations(subid, true);
-	foreach(lc, rstates)
+	for (int64 i = 1; i <= lastusedid; i++)
 	{
-		SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-		Oid			relid = rstate->relid;
-
-		/* Only cleanup resources of tablesync workers */
-		if (!OidIsValid(relid))
-			continue;
+		char		originname_to_drop[NAMEDATALEN] = {0};
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * It is possible that the origin is not yet created for tablesync
-		 * worker so passing missing_ok = true. This can happen for the states
-		 * before SUBREL_STATE_FINISHEDCOPY.
-		 */
-		ReplicationOriginNameForLogicalRep(subid, relid, originname,
-										   sizeof(originname));
-		replorigin_drop_by_name(originname, true, false);
+		snprintf(originname_to_drop, sizeof(originname_to_drop), "pg_%u_%lld", subid, (long long) i);
+		/* missing_ok = true, since the origin might be already dropped. */
+		replorigin_drop_by_name(originname_to_drop, true, false);
 	}
 
+
 	/* Clean up dependencies */
 	deleteSharedDependencyRecordsFor(SubscriptionRelationId, subid, 0);
 
@@ -1596,39 +1616,17 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
+		List	   *slots = NULL;
 
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
 
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char	   *syncslotname = (char *) lfirst(lc);
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1651,6 +1649,71 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * Get the replication slot names associated with the subscription.
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok)
+{
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+					 " FROM pg_replication_slots"
+					 " WHERE slot_name LIKE 'pg_%i_sync_%%';",
+					 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+                    errmsg("could not receive list of slots associated with the subscription %u, error: %s",
+					subid, res->err));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();
+		return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -2005,6 +2068,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -2015,18 +2079,20 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
-			char		syncslotname[NAMEDATALEN] = {0};
-
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
-				 syncslotname);
+				 worker->slot_name);
 		}
 	}
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 3579e704fe..1cab625dfb 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -440,7 +440,8 @@ pa_launch_parallel_worker(void)
 										MySubscription->name,
 										MyLogicalRepWorker->userid,
 										InvalidOid,
-										dsm_segment_handle(winfo->dsm_seg));
+										dsm_segment_handle(winfo->dsm_seg),
+										0);
 
 	if (launched)
 	{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 970d170e73..5ea88cedce 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -304,7 +304,7 @@ logicalrep_workers_find(Oid subid, bool only_running)
  */
 bool
 logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
-						 Oid relid, dsm_handle subworker_dsm)
+						 Oid relid, dsm_handle subworker_dsm, int64 slotid)
 {
 	BackgroundWorker bgw;
 	BackgroundWorkerHandle *bgw_handle;
@@ -430,6 +430,9 @@ retry:
 	worker->launch_time = now;
 	worker->in_use = true;
 	worker->generation++;
+	worker->created_slot = false;
+	worker->rep_slot_id = slotid;
+	worker->slot_name = (char *) palloc(NAMEDATALEN);
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -437,6 +440,7 @@ retry:
 	worker->relid = relid;
 	worker->relstate = SUBREL_STATE_UNKNOWN;
 	worker->relstate_lsn = InvalidXLogRecPtr;
+	worker->ready_to_reuse = false;
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
@@ -1155,7 +1159,7 @@ ApplyLauncherMain(Datum main_arg)
 				ApplyLauncherSetWorkerStartTime(sub->oid, now);
 				logicalrep_worker_launch(sub->dbid, sub->oid, sub->name,
 										 sub->owner, InvalidOid,
-										 DSM_HANDLE_INVALID);
+										 DSM_HANDLE_INVALID, 0);
 			}
 			else
 			{
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 07eea504ba..ae49c729e7 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -127,11 +127,10 @@ static bool FetchTableStates(bool *started_tx);
 static StringInfo copybuf = NULL;
 
 /*
- * Exit routine for synchronization worker.
+ * Prepares the synchronization worker for reuse or exit.
  */
 static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+clean_sync_worker(void)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -143,18 +142,28 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
-	/* And flush all writes. */
-	XLogFlush(GetXLogWriteRecPtr());
-
-	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
-	CommitTransactionCommand();
+	/*
+	 * Disconnect from publisher. Otherwise reused sync workers causes
+	 * exceeding max_wal_senders
+	 */
+	walrcv_disconnect(LogRepWorkerWalRcvConn);
+	LogRepWorkerWalRcvConn = NULL;
 
 	/* Find the leader apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
+}
+
+/*
+ * Exit routine for synchronization worker.
+ */
+static void
+pg_attribute_noreturn()
+finish_sync_worker(void)
+{
+	clean_sync_worker();
+
+	/* And flush all writes. */
+	XLogFlush(GetXLogWriteRecPtr());
 
 	/* Stop gracefully */
 	proc_exit(0);
@@ -284,6 +293,10 @@ invalidate_syncing_table_states(Datum arg, int cacheid, uint32 hashvalue)
 static void
 process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 {
+	List	   *rstates;
+	SubscriptionRelState *rstate;
+	ListCell   *lc;
+
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_CATCHUP &&
@@ -292,6 +305,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		TimeLineID	tli;
 		char		syncslotname[NAMEDATALEN] = {0};
 		char		originname[NAMEDATALEN] = {0};
+		bool		is_streaming_ended = false;
 
 		MyLogicalRepWorker->relstate = SUBREL_STATE_SYNCDONE;
 		MyLogicalRepWorker->relstate_lsn = current_lsn;
@@ -308,40 +322,29 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 								   MyLogicalRepWorker->relid,
 								   MyLogicalRepWorker->relstate,
 								   MyLogicalRepWorker->relstate_lsn);
+		CommitTransactionCommand();
 
 		/*
-		 * End streaming so that LogRepWorkerWalRcvConn can be used to drop
-		 * the slot.
-		 */
-		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
-
-		/*
-		 * Cleanup the tablesync slot.
+		 * Cleanup the tablesync slot. If the slot name used by this worker is
+		 * different from the default slot name for the worker, this means the
+		 * current table had started to being synchronized by another worker
+		 * and replication slot. And this worker is reusing a replication slot
+		 * from a previous attempt. We do not need that replication slot
+		 * anymore.
 		 *
 		 * This has to be done after updating the state because otherwise if
 		 * there is an error while doing the database operations we won't be
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
+										MyLogicalRepWorker->rep_slot_id,
 										syncslotname,
 										sizeof(syncslotname));
 
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
-		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
-
-		CommitTransactionCommand();
-		pgstat_report_stat(false);
-
-		/*
-		 * Start a new transaction to clean up the tablesync origin tracking.
-		 * This transaction will be ended within the finish_sync_worker().
-		 * Now, even, if we fail to remove this here, the apply worker will
-		 * ensure to clean it up afterward.
+		 * We are safe to drop the replication trackin origin after this
+		 * point. Now, even, if we fail to remove this here, the apply worker
+		 * will ensure to clean it up afterward.
 		 *
 		 * We need to do this after the table state is set to SYNCDONE.
 		 * Otherwise, if an error occurs while performing the database
@@ -350,34 +353,129 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 * have been cleared before restart. So, the restarted worker will use
 		 * invalid replication progress state resulting in replay of
 		 * transactions that have already been applied.
+		 *
+		 * Firstly reset the origin session to remove the ownership of the
+		 * slot. This is needed to allow the origin to be dropped or reused
+		 * later.
 		 */
+		replorigin_session_reset();
+		replorigin_session_origin = InvalidRepOriginId;
+		replorigin_session_origin_lsn = InvalidXLogRecPtr;
+		replorigin_session_origin_timestamp = 0;
+
 		StartTransactionCommand();
+		if (MyLogicalRepWorker->slot_name && strcmp(syncslotname, MyLogicalRepWorker->slot_name) != 0)
+		{
+			/*
+			 * End streaming so that LogRepWorkerWalRcvConn can be used to
+			 * drop the slot.
+			 */
+			walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+			is_streaming_ended = true;
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
 
+			ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+											   MyLogicalRepWorker->relid,
+											   originname,
+											   sizeof(originname));
+
+			/* Drop replication origin */
+			replorigin_drop_by_name(originname, true, false);
+		}
+
+		/*
+		 * We are safe to remove persisted replication slot and origin data,
+		 * since it's already in SYNCDONE state. They will not be needed
+		 * anymore.
+		 */
+		UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+							  MyLogicalRepWorker->relid,
+							  MyLogicalRepWorker->relstate,
+							  MyLogicalRepWorker->relstate_lsn,
+							  NULL,
+							  NULL);
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid))));
+
+		CommitTransactionCommand();
+		pgstat_report_stat(false);
+
+		StartTransactionCommand();
+
+		/*
+		 * This should return the default origin name for the worker. Even if
+		 * the worker used a different origin for this table, it should be
+		 * dropped and removed from the catalog so far.
+		 */
 		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
 										   MyLogicalRepWorker->relid,
 										   originname,
 										   sizeof(originname));
 
 		/*
-		 * Resetting the origin session removes the ownership of the slot.
-		 * This is needed to allow the origin to be dropped.
+		 * Check if any table whose relation state is still INIT. If a table
+		 * in INIT state is found, the worker will not be finished, it will be
+		 * reused instead.
 		 */
-		replorigin_session_reset();
-		replorigin_session_origin = InvalidRepOriginId;
-		replorigin_session_origin_lsn = InvalidXLogRecPtr;
-		replorigin_session_origin_timestamp = 0;
+		rstates = GetSubscriptionRelations(MySubscription->oid, true);
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * There is a chance that the user is concurrently performing refresh
-		 * for the subscription where we remove the table state and its origin
-		 * or the apply worker would have removed this origin. So passing
-		 * missing_ok = true.
+		foreach(lc, rstates)
+		{
+			rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
+			memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
+
+			/*
+			 * Pick the table for the next run if it is not already picked up
+			 * by another worker.
+			 */
+			LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+			if (rstate->state != SUBREL_STATE_SYNCDONE &&
+				!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+			{
+				/* Update worker state for the next table */
+				MyLogicalRepWorker->relid = rstate->relid;
+				MyLogicalRepWorker->relstate = rstate->state;
+				MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+				MyLogicalRepWorker->ready_to_reuse = true;
+				LWLockRelease(LogicalRepWorkerLock);
+				break;
+			}
+			LWLockRelease(LogicalRepWorkerLock);
+		}
+
+		/* 
+		 * If the worker is ready to be reused, clean up the worker for next
+		 * relations. 
+		 * If there is no more work left for this worker, drop replication slot
+		 * and origin. Then stop the worker gracefully. 
 		 */
-		replorigin_drop_by_name(originname, true, false);
+		if (!MyLogicalRepWorker->ready_to_reuse)
+		{
+			/*
+			 * It is important to give an error if we are unable to drop the
+			 * slot, otherwise, it won't be dropped till the corresponding
+			 * subscription is dropped. So passing missing_ok = false.
+			 */
+			if (MyLogicalRepWorker->created_slot)
+			{
+				/* End streaming if it's not already ended. */
+				if (!is_streaming_ended)
+					walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+				ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
+			}
+
+			/* Drop replication origin before exiting. */
+			replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
+			finish_sync_worker();
+		}
+		else
+		{
+			clean_sync_worker();
+		}
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -464,6 +562,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 			if (current_lsn >= rstate->lsn)
 			{
 				char		originname[NAMEDATALEN];
+				bool		is_origin_null = true;
 
 				rstate->state = SUBREL_STATE_READY;
 				rstate->lsn = current_lsn;
@@ -484,18 +583,27 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				 * error while dropping we won't restart it to drop the
 				 * origin. So passing missing_ok = true.
 				 */
-				ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-												   rstate->relid,
-												   originname,
-												   sizeof(originname));
-				replorigin_drop_by_name(originname, true, false);
+				GetSubscriptionRelOrigin(MyLogicalRepWorker->subid,
+										 rstate->relid, originname,
+										 &is_origin_null);
+
+				if (!is_origin_null)
+				{
+					replorigin_drop_by_name(originname, true, false);
+				}
 
 				/*
 				 * Update the state to READY only after the origin cleanup.
 				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-										   rstate->relid, rstate->state,
-										   rstate->lsn);
+				UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+									  rstate->relid,
+									  rstate->state,
+									  rstate->lsn,
+									  NULL,
+									  NULL);
+
+				CommitTransactionCommand();
+				started_tx = false;
 			}
 		}
 		else
@@ -584,12 +692,22 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						TimestampDifferenceExceeds(hentry->last_start_time, now,
 												   wal_retrieve_retry_interval))
 					{
+						if (IsTransactionState())
+							CommitTransactionCommand();
+						StartTransactionCommand();
+						started_tx = true;
+
+						MySubscription->lastusedid++;
+						UpdateSubscriptionLastSlotId(MyLogicalRepWorker->subid,
+													 MySubscription->lastusedid);
+
 						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
 												 MySubscription->oid,
 												 MySubscription->name,
 												 MyLogicalRepWorker->userid,
 												 rstate->relid,
-												 DSM_HANDLE_INVALID);
+												 DSM_HANDLE_INVALID,
+												 MySubscription->lastusedid);
 						hentry->last_start_time = now;
 					}
 				}
@@ -1198,8 +1316,8 @@ copy_table(Relation rel)
  * The name must not exceed NAMEDATALEN - 1 because of remote node constraints
  * on slot name length. We append system_identifier to avoid slot_name
  * collision with subscriptions in other clusters. With the current scheme
- * pg_%u_sync_%u_UINT64_FORMAT (3 + 10 + 6 + 10 + 20 + '\0'), the maximum
- * length of slot_name will be 50.
+ * pg_%u_sync_%lu_UINT64_FORMAT (3 + 10 + 6 + 20 + 20 + '\0'), the maximum
+ * length of slot_name will be 45.
  *
  * The returned slot name is stored in the supplied buffer (syncslotname) with
  * the given size.
@@ -1210,11 +1328,11 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
+ReplicationSlotNameForTablesync(Oid suboid, int64 slotid,
 								char *syncslotname, Size szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%lld_" UINT64_FORMAT, suboid,
+			(long long) slotid, GetSystemIdentifier());
 }
 
 /*
@@ -1237,6 +1355,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	WalRcvExecResult *res;
 	char		originname[NAMEDATALEN];
 	RepOriginId originid;
+	char	   *prev_slotname;
 
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
@@ -1265,7 +1384,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
+									MyLogicalRepWorker->rep_slot_id,
 									slotname,
 									NAMEDATALEN);
 
@@ -1285,12 +1404,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC ||
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY);
 
+	/*
+	 * See if tablesync of the current relation has been started with another
+	 * replication slot.
+	 *
+	 * Read previous slot name from the catalog, if exists.
+	 */
+	prev_slotname = (char *) palloc(NAMEDATALEN);
+	StartTransactionCommand();
+	GetSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+									  MyLogicalRepWorker->relid,
+									  prev_slotname);
+
 	/* Assign the origin tracking record name. */
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   sizeof(originname));
 
+	CommitTransactionCommand();
+
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC)
 	{
 		/*
@@ -1304,10 +1437,49 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		 * breakdown then it wouldn't have succeeded so trying it next time
 		 * seems like a better bet.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, slotname, true);
+		if (strlen(prev_slotname) > 0)
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);
+
+			StartTransactionCommand();
+			/* Replication origin might still exist. Try to drop */
+			replorigin_drop_by_name(originname, true, false);
+
+			/*
+			 * Remove replication slot and origin name from the relation's
+			 * catalog record
+			 */
+			UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+								  MyLogicalRepWorker->relid,
+								  MyLogicalRepWorker->relstate,
+								  MyLogicalRepWorker->relstate_lsn,
+								  NULL,
+								  NULL);
+			CommitTransactionCommand();
+		}
 	}
 	else if (MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY)
 	{
+		/*
+		 * At this point, the table that is currently being synchronized
+		 * should have its replication slot name filled in the catalog. The
+		 * tablesync process was started with another sync worker and
+		 * replication slot. We need to continue using the same replication
+		 * slot in this worker too.
+		 */
+		if (strlen(prev_slotname) == 0)
+		{
+			elog(ERROR, "Replication slot could not be found for subscription %u, relation %u",
+				 MyLogicalRepWorker->subid,
+				 MyLogicalRepWorker->relid);
+		}
+
+		/*
+		 * Proceed with the correct replication slot. Use previously created
+		 * replication slot to sync this table.
+		 */
+		slotname = prev_slotname;
+
 		/*
 		 * The COPY phase was previously done, but tablesync then crashed
 		 * before it was able to finish normally.
@@ -1327,7 +1499,9 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 		goto copy_table_done;
 	}
+	pfree(prev_slotname);
 
+	/* Preparing for table copy operation */
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 	MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
 	MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
@@ -1335,10 +1509,12 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   MyLogicalRepWorker->relstate,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  MyLogicalRepWorker->relstate,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 	CommitTransactionCommand();
 	pgstat_report_stat(true);
 
@@ -1377,6 +1553,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						GetUserNameFromId(GetUserId(), true),
 						RelationGetRelationName(rel))));
 
+
 	/*
 	 * Start a transaction in the remote node in REPEATABLE READ mode.  This
 	 * ensures that both the replication slot we create (see below) and the
@@ -1392,48 +1569,91 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						res->err)));
 	walrcv_clear_result(res);
 
+	originid = replorigin_by_name(originname, true);
+
 	/*
 	 * Create a new permanent logical decoding slot. This slot will be used
 	 * for the catchup phase after COPY is done, so tell it to use the
 	 * snapshot to make the final data consistent.
+	 *
+	 * Replication slot will only be created if either this is the first run
+	 * of the worker or we're not using a previous replication slot.
 	 */
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-
-	/*
-	 * Setup replication origin tracking. The purpose of doing this before the
-	 * copy is to avoid doing the copy again due to any error in setting up
-	 * origin tracking.
-	 */
-	originid = replorigin_by_name(originname, true);
-	if (!OidIsValid(originid))
+	if (!MyLogicalRepWorker->created_slot)
 	{
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						   slotname, false /* permanent */ , false /* two_phase */ ,
+						   CRS_USE_SNAPSHOT, origin_startpos);
+
 		/*
-		 * Origin tracking does not exist, so create it now.
-		 *
-		 * Then advance to the LSN got from walrcv_create_slot. This is WAL
-		 * logged for the purpose of recovery. Locks are to prevent the
-		 * replication origin from vanishing while advancing.
+		 * Remember that we created the slot so that we will not try to create
+		 * it again.
 		 */
-		originid = replorigin_create(originname);
-
-		LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
-		replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
-						   true /* go backward */ , true /* WAL log */ );
-		UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->created_slot = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
+		/*
+		 * Setup replication origin tracking. The purpose of doing this before
+		 * the copy is to avoid doing the copy again due to any error in
+		 * setting up origin tracking.
+		 */
+		if (!OidIsValid(originid))
+		{
+			/*
+			 * Origin tracking does not exist, so create it now.
+			 */
+			originid = replorigin_create(originname);
+		}
+		else
+		{
+			/*
+			 * At this point, there shouldn't be any existing replication
+			 * origin with the same name.
+			 */
+			ereport(ERROR,
+					(errcode(ERRCODE_DUPLICATE_OBJECT),
+					 errmsg("replication origin \"%s\" already exists",
+							originname)));
+		}
 	}
 	else
 	{
-		ereport(ERROR,
-				(errcode(ERRCODE_DUPLICATE_OBJECT),
-				 errmsg("replication origin \"%s\" already exists",
-						originname)));
+		/*
+		 * Do not create a new replication slot, reuse the existing one
+		 * instead. Use a new snapshot for the replication slot to ensure that
+		 * tablesync and apply proceses are consistent with each other.
+		 */
+		WalRcvStreamOptions options;
+		int			server_version;
+
+		server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+		options.proto.logical.proto_version =
+			server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+			server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+			LOGICALREP_PROTO_VERSION_NUM;
+		options.proto.logical.publication_names = MySubscription->publications;
+
+		HOLD_INTERRUPTS();
+		walrcv_slot_snapshot(LogRepWorkerWalRcvConn, slotname, &options, origin_startpos);
+		RESUME_INTERRUPTS();
 	}
 
+	/*
+	 * Advance to the LSN got from walrcv_create_slot or walrcv_slot_snapshot.
+	 * This is WAL logged for the purpose of recovery. Locks are to prevent
+	 * the replication origin from vanishing while advancing.
+	 *
+	 * Then setup replication origin tracking.
+	 */
+	LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+	replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
+					   true /* go backward */ , true /* WAL log */ );
+	UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+
 	/* Now do the initial data copy */
 	PushActiveSnapshot(GetTransactionSnapshot());
 	copy_table(rel);
@@ -1456,10 +1676,12 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	 * Update the persisted state to indicate the COPY phase is done; make it
 	 * visible to others.
 	 */
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   SUBREL_STATE_FINISHEDCOPY,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  SUBREL_STATE_FINISHEDCOPY,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 
 	CommitTransactionCommand();
 
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index cfb2ab6248..fd39254ecc 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -388,6 +388,7 @@ static void stream_open_file(Oid subid, TransactionId xid,
 static void stream_write_change(char action, StringInfo s);
 static void stream_open_and_write_change(TransactionId xid, char action, StringInfo s);
 static void stream_close_file(void);
+static void stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
@@ -456,8 +457,16 @@ ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 {
 	if (OidIsValid(relid))
 	{
-		/* Replication origin name for tablesync workers. */
-		snprintf(originname, szoriginname, "pg_%u_%u", suboid, relid);
+		bool		is_null = true;
+
+		/*
+		 * Replication origin name for tablesync workers. First, look into the
+		 * catalog. If originname does not exist, then use the default name.
+		 */
+		GetSubscriptionRelOrigin(suboid, relid,
+								 originname, &is_null);
+		if (is_null)
+			snprintf(originname, szoriginname, "pg_%u_%lld", suboid, (long long) MyLogicalRepWorker->rep_slot_id);
 	}
 	else
 	{
@@ -3576,6 +3585,23 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can move to next table while running
+				 * process_syncing_tables_for_sync. Before we were able to
+				 * reuse tablesync workers, that
+				 * process_syncing_tables_for_sync call would exit the worker
+				 * instead of moving to next table. Now that tablesync workers
+				 * can be reused, we need to take care of memory contexts here
+				 * before moving to sync a table.
+				 */
+				if (MyLogicalRepWorker->ready_to_reuse)
+				{
+					MemoryContextResetAndDeleteChildren(ApplyMessageContext);
+					MemoryContextSwitchTo(TopMemoryContext);
+					return;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -3595,6 +3621,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+			if (MyLogicalRepWorker->ready_to_reuse)
+			{
+				endofstream = true;
+			}
 		}
 
 		/* Cleanup the memory. */
@@ -3697,8 +3727,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * If it's moving to next relation, this is a sync worker. Sync workers
+	 * end the streaming during process_syncing_tables_for_sync. Calling
+	 * endstreaming twice causes "no COPY in progress" errors.
+	 */
+	if (!MyLogicalRepWorker->ready_to_reuse)
+	{
+		/* All done */
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	}
 }
 
 /*
@@ -4282,6 +4320,56 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+ /* stream_build_options
+  * 	Build logical replication streaming options.
+  *
+  * This function sets streaming options including replication slot name
+  * and origin start position. Workers need these options for logical replication.
+  */
+static void
+stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4356,6 +4444,9 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 
 	/* allocate slot name in long-lived context */
 	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+
+	/* Keep the replication slot name used for this sync. */
+	MyLogicalRepWorker->slot_name = *myslotname;
 	pfree(syncslotname);
 }
 
@@ -4393,6 +4484,135 @@ start_apply(XLogRecPtr origin_startpos)
 	PG_END_TRY();
 }
 
+/*
+ * Runs the tablesync worker.
+ * It starts table sync. After successful sync,
+ * builds streaming options and starts streaming.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Set this to false for safety, in case we're already reusing the worker */
+	MyLogicalRepWorker->ready_to_reuse = false;
+
+	start_table_sync(origin_startpos, &slotname);
+
+	/*
+	 * Allocate the origin name in long-lived context for error context
+	 * message.
+	 */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   originname_size);
+	CommitTransactionCommand();
+
+	set_apply_error_context_origin(originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/* Start normal logical streaming replication. */
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+}
+
+/*
+ * Runs the apply worker.
+ * It sets up replication origin, the streaming options
+ * and then starts streaming.
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
+{
+	/* This is the leader apply worker */
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+
+	slotname = MySubscription->slotname;
+
+	/*
+		* This shouldn't happen if the subscription is enabled, but guard
+		* against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+		* crash if slot is NULL.)
+		*/
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											MySubscription->name, &err);
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+		* We don't really use the output identify_system for anything but it
+		* does some initializations on the upstream so let's still call it.
+		*/
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+	{
+		/* Start streaming with two_phase enabled */
+		options->proto.logical.twophase = true;
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
+	}
+	else
+	{
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+	}
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+}
+
 /*
  * Common initialization for leader apply worker and parallel apply worker.
  *
@@ -4485,7 +4705,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	/* Attach to slot */
 	logicalrep_worker_attach(worker_slot);
@@ -4513,156 +4732,46 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
-	 * the subscription relation state.
+	 * the subscription relation state. Do this outside the loop to avoid
+	 * exceeding MAX_SYSCACHE_CALLBACKS
 	 */
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
-
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
-
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-
 	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 * The loop where worker does its job. It loops until the worker is not
+	 * reused.
 	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
-
-	if (!am_tablesync_worker())
-	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
+	do {
+		if (am_tablesync_worker())
 		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
+			/*
+			 * This is a tablesync worker. Start syncing tables before
+			 * starting the apply loop.
+			 */
+			run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 		}
 		else
 		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+			/* This is main apply worker */
+			run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 		}
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-						MySubscription->name,
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-						"?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
-
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+		/* Run the main loop. */
+		start_apply(origin_startpos);
 
+		if (MyLogicalRepWorker->ready_to_reuse)
+		{
+			StartTransactionCommand();
+			ereport(LOG,
+					(errmsg("logical replication table synchronization worker for subscription \"%s\" has moved to sync table \"%s\".",
+							MySubscription->name, get_rel_name(MyLogicalRepWorker->relid))));
+			CommitTransactionCommand();
+		}
+	} while (MyLogicalRepWorker->ready_to_reuse);
 	proc_exit(0);
 }
 
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index b0f2a1705d..a0ee12e259 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -103,6 +103,9 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
 
 	/* Only publish data originating from the specified origin */
 	text		suborigin BKI_DEFAULT(LOGICALREP_ORIGIN_ANY);
+
+	/* The last used ID to create a replication slot for tablesync */
+	int64		sublastusedid BKI_DEFAULT(0);
 #endif
 } FormData_pg_subscription;
 
@@ -137,6 +140,8 @@ typedef struct Subscription
 	List	   *publications;	/* List of publication names to subscribe to */
 	char	   *origin;			/* Only publish data originating from the
 								 * specified origin */
+	int64		lastusedid;		/* Last used unique ID to create replication
+								 * slots in tablesync */
 } Subscription;
 
 /* Disallow streaming in-progress transactions. */
@@ -157,6 +162,7 @@ typedef struct Subscription
 extern Subscription *GetSubscription(Oid subid, bool missing_ok);
 extern void FreeSubscription(Subscription *sub);
 extern void DisableSubscription(Oid subid);
+extern void UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid);
 
 extern int	CountDBSubscriptions(Oid dbid);
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 60a2bcca23..a528866012 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -44,6 +44,12 @@ CATALOG(pg_subscription_rel,6102,SubscriptionRelRelationId)
 											 * used for synchronization
 											 * coordination, or NULL if not
 											 * valid */
+	NameData	srrelslotname BKI_FORCE_NULL;	/* name of the replication
+												 * slot for relation in
+												 * subscription */
+	NameData	srreloriginname BKI_FORCE_NULL; /* origin name for relation in
+												 * subscription */
+
 #endif
 } FormData_pg_subscription_rel;
 
@@ -81,10 +87,17 @@ typedef struct SubscriptionRelState
 } SubscriptionRelState;
 
 extern void AddSubscriptionRelState(Oid subid, Oid relid, char state,
-									XLogRecPtr sublsn);
+									XLogRecPtr sublsn, char *relslotname, char *reloriginname);
 extern void UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 									   XLogRecPtr sublsn);
+extern void UpdateSubscriptionRel(Oid subid, Oid relid, char state,
+								  XLogRecPtr sublsn, char *relslotname, char *reloriginname);
+extern void UpdateSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *relslotname);
+
 extern char GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn);
+extern void GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname);
+extern void GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull);
+
 extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 
 extern bool HasSubscriptionRelations(Oid subid);
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 8872c80cdf..3547daaaec 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -219,8 +219,9 @@ extern bool InvalidateObsoleteReplicationSlots(XLogSegNo oldestSegno);
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
 extern bool ReplicationSlotName(int index, Name name);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, Size szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, int64 slotid, char *syncslotname, Size szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List *GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index dc87a4edd1..7f7cb75c81 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -35,6 +35,23 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/*
+	 * Indicates if the sync worker created a replication slot for itself
+	 * in any point of its lifetime.
+	 * False means that the worker has not created a slot yet, and has been
+	 * reusing replication slots created by other workers so far.
+	 */
+	bool		created_slot;
+
+	/*
+	 * Unique identifier for replication slot to be created by tablesnync
+	 * workers, if needed.
+	 */
+	int64		rep_slot_id;
+
+	/* Replication slot name used by the worker. */
+	char	   *slot_name;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
@@ -56,6 +73,12 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/*
+	 * Used to indicate whether sync worker will be reused for another
+	 * relation
+	 */
+	bool		ready_to_reuse;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction or
@@ -231,7 +254,8 @@ extern LogicalRepWorker *logicalrep_worker_find(Oid subid, Oid relid,
 extern List *logicalrep_workers_find(Oid subid, bool only_running);
 extern bool logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname,
 									 Oid userid, Oid relid,
-									 dsm_handle subworker_dsm);
+									 dsm_handle subworker_dsm,
+									 int64 slotid);
 extern void logicalrep_worker_stop(Oid subid, Oid relid);
 extern void logicalrep_pa_worker_stop(int slot_no, uint16 generation);
 extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
-- 
2.25.1

#39shveta malik
shveta.malik@gmail.com
In reply to: Melih Mutlu (#37)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Wed, Feb 1, 2023 at 5:05 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi,

Please see attached patches for the below changes.

Thanks for reviewing,
--
Melih Mutlu
Microsoft

Hello Melih,
Thank you for making the changes.

I have few more comments:
1)
src/backend/replication/logical/worker.c: (errmsg("logical replication
table synchronization worker for subscription \"%s\", table \"%s\" has
started",
src/backend/replication/logical/worker.c: (errmsg("logical replication
table synchronization worker for subscription \"%s\" has moved to sync
table \"%s\".",
src/backend/replication/logical/tablesync.c: (errmsg("logical
replication table synchronization worker for subscription \"%s\",
table \"%s\" has finished",

In above can we have rep_slot_id as well in trace message, else it is
not clear which worker moved to next relation. We may have:
logical replication table synchronization worker_%d for subscription
\"%s\" has moved to sync table, rep_slot_id,....

Overall we need to improve the tracing. I will give my suggestions on
this later (in detail).

2) I found a crash in the previous patch (v9), but have not tested it
on the latest yet. Crash happens when all the replication slots are
consumed and we are trying to create new. I tweaked the settings like
below so that it can be reproduced easily:
max_sync_workers_per_subscription=3
max_replication_slots = 2
and then ran the test case shared by you. I think there is some memory
corruption happening. (I did test in debug mode, have not tried in
release mode). I tried to put some traces to identify the root-cause.
I observed that worker_1 keeps on moving from 1 table to another table
correctly, but at some point, it gets corrupted i.e. origin-name
obtained for it is wrong and it tries to advance that and since that
origin does not exist, it asserts and then something else crashes.
From log: (new trace lines added by me are prefixed by shveta, also
tweaked code to have my comment 1 fixed to have clarity on worker-id).

form below traces, it is clear that worker_1 was moving from one
relation to another, always getting correct origin 'pg_16688_1', but
at the end it got 'pg_16688_49' which does not exist. Second part of
trace shows who updated 'pg_16688_49', it was done by worker_49 which
even did not get chance to create this origin due to max_rep_slot
reached.
==============================
..............
2023-02-01 16:01:38.041 IST [9243] LOG: logical replication table
synchronization worker_1 for subscription "mysub", table "table_93"
has finished
2023-02-01 16:01:38.047 IST [9243] LOG: logical replication table
synchronization worker_1 for subscription "mysub" has moved to sync
table "table_98".
2023-02-01 16:01:38.055 IST [9243] LOG: shveta-
LogicalRepSyncTableStart- worker_1 get-origin-id2 originid:2,
originname:pg_16688_1
2023-02-01 16:01:38.055 IST [9243] LOG: shveta-
LogicalRepSyncTableStart- Worker_1 reusing
slot:pg_16688_sync_1_7195132648087016333, originid:2,
originname:pg_16688_1
2023-02-01 16:01:38.094 IST [9243] LOG: shveta-
LogicalRepSyncTableStart- worker_1 updated-origin2
originname:pg_16688_1
2023-02-01 16:01:38.096 IST [9243] LOG: logical replication table
synchronization worker_1 for subscription "mysub", table "table_98"
has finished
2023-02-01 16:01:38.096 IST [9243] LOG: logical replication table
synchronization worker_1 for subscription "mysub" has moved to sync
table "table_60".
2023-02-01 16:01:38.099 IST [9243] LOG: shveta-
LogicalRepSyncTableStart- worker_1 get-origin originid:0,
originname:pg_16688_49
2023-02-01 16:01:38.099 IST [9243] LOG: could not drop replication
slot "pg_16688_sync_49_7195132648087016333" on publisher: ERROR:
replication slot "pg_16688_sync_49_7195132648087016333" does not exist
2023-02-01 16:01:38.103 IST [9243] LOG: shveta-
LogicalRepSyncTableStart- Worker_1 reusing
slot:pg_16688_sync_1_7195132648087016333, originid:0,
originname:pg_16688_49
TRAP: failed Assert("node != InvalidRepOriginId"), File: "origin.c",
Line: 892, PID: 9243
postgres: logical replication worker for subscription 16688 sync 16384
(ExceptionalCondition+0xbb)[0x56019194d3b7]
postgres: logical replication worker for subscription 16688 sync 16384
(replorigin_advance+0x6d)[0x5601916b53d4]
postgres: logical replication worker for subscription 16688 sync 16384
(LogicalRepSyncTableStart+0xbb4)[0x5601916cb648]
postgres: logical replication worker for subscription 16688 sync 16384
(+0x5d25e2)[0x5601916d35e2]
postgres: logical replication worker for subscription 16688 sync 16384
(+0x5d282c)[0x5601916d382c]
postgres: logical replication worker for subscription 16688 sync 16384
(ApplyWorkerMain+0x17b)[0x5601916d4078]
postgres: logical replication worker for subscription 16688 sync 16384
(StartBackgroundWorker+0x248)[0x56019167f943]
postgres: logical replication worker for subscription 16688 sync 16384
(+0x589ad3)[0x56019168aad3]
postgres: logical replication worker for subscription 16688 sync 16384
(+0x589ee3)[0x56019168aee3]
postgres: logical replication worker for subscription 16688 sync 16384
(+0x588d8d)[0x560191689d8d]
postgres: logical replication worker for subscription 16688 sync 16384
(+0x584604)[0x560191685604]
postgres: logical replication worker for subscription 16688 sync 16384
(PostmasterMain+0x14f1)[0x560191684f1e]
postgres: logical replication worker for subscription 16688 sync 16384
(+0x446e05)[0x560191547e05]
/lib/x86_64-linux-gnu/libc.so.6(+0x29d90)[0x7f048cc58d90]
==============================

'pg_16688_49' updated by worker_49:
2023-02-01 16:01:37.083 IST [9348] LOG: shveta-
LogicalRepSyncTableStart- worker_49 get-origin originid:0,
originname:pg_16688_49
2023-02-01 16:01:37.083 IST [9348] LOG: shveta-
LogicalRepSyncTableStart- worker_49 updated-origin
originname:pg_16688_49
2023-02-01 16:01:37.083 IST [9348] LOG: shveta-
LogicalRepSyncTableStart- worker_49 get-origin-id2 originid:0,
originname:pg_16688_49
2023-02-01 16:01:37.083 IST [9348] ERROR: could not create
replication slot "pg_16688_sync_49_7195132648087016333": ERROR: all
replication slots are in use
HINT: Free one or increase max_replication_slots.
==============================

Rest of the workers keep on exiting and getting recreated since they
could not create slot. The last_used_id kept on increasing on every
restart of subscriber until we kill it. In my case it reached 2k+.

thanks
Shveta

#40Melih Mutlu
m.melihmutlu@gmail.com
In reply to: wangw.fnst@fujitsu.com (#35)
2 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi,

wangw.fnst@fujitsu.com <wangw.fnst@fujitsu.com>, 31 Oca 2023 Sal, 13:40
tarihinde şunu yazdı:

Sorry, I forgot to add the link to the email. Please refer to [1].

[1] -
/messages/by-id/OSZPR01MB631013C833C98E826B3CFCB9FDC69@OSZPR01MB6310.jpnprd01.prod.outlook.com

Thanks for pointing out this review. I somehow skipped that, sorry.

Please see attached patches.

shiy.fnst@fujitsu.com <shiy.fnst@fujitsu.com>, 17 Oca 2023 Sal, 10:46
tarihinde şunu yazdı:

On Wed, Jan 11, 2023 4:31 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:
0001 patch
============
1. walsender.c
+ /* Create a tuple to send consisten WAL location */

"consisten" should be "consistent" I think.

Done.

2. logical.c
+       if (need_full_snapshot)
+       {
+               LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+               SpinLockAcquire(&slot->mutex);
+               slot->effective_catalog_xmin = xmin_horizon;
+               slot->data.catalog_xmin = xmin_horizon;
+               slot->effective_xmin = xmin_horizon;
+               SpinLockRelease(&slot->mutex);
+
+               xmin_horizon =
GetOldestSafeDecodingTransactionId(!need_full_snapshot);
+               ReplicationSlotsComputeRequiredXmin(true);
+
+               LWLockRelease(ProcArrayLock);
+       }

It seems that we should first get the safe decoding xid, then inform the
slot
machinery about the new limit, right? Otherwise the limit will be
InvalidTransactionId and that seems inconsistent with the comment.

You're right. Moved that call before assigning xmin_horizon.

3. doc/src/sgml/protocol.sgml
+       is used in the currenct transaction. This command is currently
only supported
+       for logical replication.
+       slots.

We don't need to put "slots" in a new line.

Done.

0002 patch
============
1.
In pg_subscription_rel.h, I think the type of "srrelslotname" can be
changed to
NameData, see "subslotname" in pg_subscription.h.

2.
+                                * Find the logical replication sync
worker if exists store
+                                * the slot number for dropping associated
replication slots
+                                * later.

Should we add comma after "if exists"?

Done.

3.

+       PG_FINALLY();
+       {
+               pfree(cmd.data);
+       }
+       PG_END_TRY();
+       \
+               return tablelist;
+}

Do we need the backslash?

Removed it.

4.
+       /*
+        * Advance to the LSN got from walrcv_create_slot. This is WAL
+        * logged for the purpose of recovery. Locks are to prevent the
+        * replication origin from vanishing while advancing.

"walrcv_create_slot" should be changed to
"walrcv_create_slot/walrcv_slot_snapshot" I think.

Right, done.

5.

+                       /* Replication drop might still exist. Try to drop
*/
+                       replorigin_drop_by_name(originname, true, false);

Should "Replication drop" be "Replication origin"?

Done.

6.
I saw an assertion failure in the following case, could you please look
into it?
The backtrace is attached.

-- pub
CREATE TABLE tbl1 (a int, b text);
CREATE TABLE tbl2 (a int primary key, b text);
create publication pub for table tbl1, tbl2;
insert into tbl1 values (1, 'a');
insert into tbl1 values (1, 'a');

-- sub
CREATE TABLE tbl1 (a int primary key, b text);
CREATE TABLE tbl2 (a int primary key, b text);
create subscription sub connection 'dbname=postgres port=5432' publication
pub;

Subscriber log:
2023-01-17 14:47:10.054 CST [1980841] LOG: logical replication apply
worker for subscription "sub" has started
2023-01-17 14:47:10.060 CST [1980843] LOG: logical replication table
synchronization worker for subscription "sub", table "tbl1" has started
2023-01-17 14:47:10.070 CST [1980845] LOG: logical replication table
synchronization worker for subscription "sub", table "tbl2" has started
2023-01-17 14:47:10.073 CST [1980843] ERROR: duplicate key value violates
unique constraint "tbl1_pkey"
2023-01-17 14:47:10.073 CST [1980843] DETAIL: Key (a)=(1) already exists.
2023-01-17 14:47:10.073 CST [1980843] CONTEXT: COPY tbl1, line 2
2023-01-17 14:47:10.074 CST [1980821] LOG: background worker "logical
replication worker" (PID 1980843) exited with exit code 1
2023-01-17 14:47:10.083 CST [1980845] LOG: logical replication table
synchronization worker for subscription "sub", table "tbl2" has finished
2023-01-17 14:47:10.083 CST [1980845] LOG: logical replication table
synchronization worker for subscription "sub" has moved to sync table
"tbl1".
TRAP: failed Assert("node != InvalidRepOriginId"), File: "origin.c", Line:
892, PID: 1980845

I'm not able to reproduce this yet. Will look into it further.

Thanks,
--
Melih Mutlu
Microsoft

Attachments:

v10-0002-Reuse-Logical-Replication-Background-worker.patchapplication/octet-stream; name=v10-0002-Reuse-Logical-Replication-Background-worker.patchDownload
From 129c9a1ed564714539a6421f14ab6538444ca9d3 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH 2/2] Reuse Logical Replication Background worker

This commit allows reusing tablesync workers for syncing more than one relation sequantially during their lifetime, instead of exiting after only syncing one relation.

Before this commit, tablesync workers were capable of syncing only one
relation. For each table, a sync worker was launched, then a new
replication slot and new origin were created.

Now, tablesync workers are not only limited with one relation and can move to another relation in the same subscription and reuse existing
replication slots and origins

This reduces the overhead of launching/killing a new background worker for each relation.
By reusing tablesync workers, replication slots and origins created for tablesync can be reused as well.
Removing the burden of creating/dropping replication slot/origin improves tablesync speed significantly especially for empty or small tables.

A new tablesync worker gets launched only if the number of tablesync
workers for the subscription does not exceed
max_sync_workers_per_subscription. If there is a table needs to be synced, a tablesync worker picks that up and processes it.The worker continues to picking new tables to sync until there is no table left for synchronization in the subscription.

If the state of the current table is INIT or DATASYNC, tablesync worker needs a
repliation slot/origin. If the worker has not created slot and origin in
its previous runs, it will create them. Otherwise the worker reuses
slot and origin created by itself earlier. Tables in FINISHEDCOPY are
expected to have a replication slot and origin. Tablesync worker proceed
with existing slot and origin of FINISHEDCOPY tables and do not need to
create new ones.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/catalogs.sgml                    |  31 ++
 src/backend/catalog/pg_subscription.c         | 288 +++++++++++-
 src/backend/commands/subscriptioncmds.c       | 226 ++++++----
 .../replication/logical/applyparallelworker.c |   3 +-
 src/backend/replication/logical/launcher.c    |   8 +-
 src/backend/replication/logical/tablesync.c   | 424 +++++++++++++-----
 src/backend/replication/logical/worker.c      | 387 ++++++++++------
 src/include/catalog/pg_subscription.h         |   6 +
 src/include/catalog/pg_subscription_rel.h     |  15 +-
 src/include/replication/slot.h                |   3 +-
 src/include/replication/worker_internal.h     |  26 +-
 11 files changed, 1083 insertions(+), 334 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index c1e4048054..31b4cfbbe4 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8002,6 +8002,19 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        origin.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>sublastusedid</structfield> <type>int8</type>
+      </para>
+      <para>
+      The last used ID for tablesync workers. It acts as an unique identifier
+      for replication slots which are created by tablesync workers.
+      The last used ID needs to be persisted to make logical replication safely
+      proceed after any interruption. If sublastusedid is 0, then no table has
+      been synced yet.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
@@ -8086,6 +8099,24 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        otherwise null
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srrelslotname</structfield> <type>name</type>
+      </para>
+      <para>
+       Replication slot name that is used for synchronization of relation
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srreloriginname</structfield> <type>name</type>
+      </para>
+      <para>
+       Origin name that is used for tracking synchronization of relation
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index a56ae311c3..29a8fc6919 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -114,6 +114,14 @@ GetSubscription(Oid subid, bool missing_ok)
 	Assert(!isnull);
 	sub->origin = TextDatumGetCString(datum);
 
+	/* Get last used id */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
+							tup,
+							Anum_pg_subscription_sublastusedid,
+							&isnull);
+	Assert(!isnull);
+	sub->lastusedid = DatumGetInt64(datum);
+
 	ReleaseSysCache(tup);
 
 	return sub;
@@ -205,6 +213,44 @@ DisableSubscription(Oid subid)
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update the last used replication slot ID for the given subscription.
+ */
+void
+UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid)
+{
+	Relation	rel;
+	bool		nulls[Natts_pg_subscription];
+	bool		replaces[Natts_pg_subscription];
+	Datum		values[Natts_pg_subscription];
+	HeapTuple	tup;
+
+	/* Look up the subscription in the catalog */
+	rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+	tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	/* Form a new tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	replaces[Anum_pg_subscription_sublastusedid - 1] = true;
+	values[Anum_pg_subscription_sublastusedid- 1] = Int64GetDatum(lastusedid);
+
+	/* Update the catalog */
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+	heap_freetuple(tup);
+
+	table_close(rel, NoLock);
+}
+
 /*
  * Convert text array to list of strings.
  *
@@ -234,7 +280,7 @@ textarray_to_stringlist(ArrayType *textarray)
  */
 void
 AddSubscriptionRelState(Oid subid, Oid relid, char state,
-						XLogRecPtr sublsn)
+						XLogRecPtr sublsn, char *relslotname, char *reloriginname)
 {
 	Relation	rel;
 	HeapTuple	tup;
@@ -263,6 +309,16 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
 	else
 		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] = 
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -275,6 +331,60 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Internal function to modify columns for relation state update
+ */
+static void
+UpdateSubscriptionRelState_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char state,
+									XLogRecPtr sublsn)
+{
+	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
+	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+
+	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (sublsn != InvalidXLogRecPtr)
+		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
+	else
+		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication slot update
+ */
+static void
+UpdateSubscriptionRelReplicationSlot_internal(Datum *values,
+											bool *nulls,
+											bool *replaces,
+											char *relslotname)
+{
+	replaces[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication origin update
+ */
+static void
+UpdateSubscriptionRelOrigin_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char *reloriginname)
+{
+	replaces[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+}
+
 /*
  * Update the state of a subscription table.
  */
@@ -305,14 +415,48 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	memset(nulls, false, sizeof(nulls));
 	memset(replaces, false, sizeof(replaces));
 
-	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
-	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
 
-	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
-	if (sublsn != InvalidXLogRecPtr)
-		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
-	else
-		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update the replication slot name of a subscription table.
+ */
+void
+UpdateSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *relslotname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
 
 	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
 							replaces);
@@ -324,6 +468,134 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update replication slot name, origin name and state of
+ * a subscription table in one transaction.
+ */
+void
+UpdateSubscriptionRel(Oid subid,
+					  Oid relid,
+					  char state,
+					  XLogRecPtr sublsn,
+					  char *relslotname,
+					  char *reloriginname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
+	UpdateSubscriptionRelOrigin_internal(values, nulls, replaces, reloriginname);
+
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Get origin name of subscription table.
+ *
+ * reloriginname's value has the replication origin name if the origin exists.
+ */
+void
+GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*originname;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srreloriginname, isnull);
+	if (!*isnull)
+	{
+		originname = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(reloriginname, originname, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
+/*
+ * Get replication slot name of subscription table.
+ *
+ * slotname's value has the replication slot name if the subscription has any.
+ */
+void
+GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*relrepslot;
+	bool		isnull;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srrelslotname, &isnull);
+	if (!isnull)
+	{
+		relrepslot = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(slotname, relrepslot, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
 /*
  * Get state of subscription table.
  *
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 464db6d247..b580e59ffc 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -649,6 +649,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 		publicationListToArray(publications);
 	values[Anum_pg_subscription_suborigin - 1] =
 		CStringGetTextDatum(opts.origin);
+	values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(0);
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -709,7 +710,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 										 rv->schemaname, rv->relname);
 
 				AddSubscriptionRelState(subid, relid, table_state,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 			}
 
 			/*
@@ -799,6 +800,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	} SubRemoveRels;
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -876,7 +879,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				AddSubscriptionRelState(sub->oid, relid,
 										copy_data ? SUBREL_STATE_INIT : SUBREL_STATE_READY,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 				ereport(DEBUG1,
 						(errmsg_internal("table \"%s.%s\" added to subscription \"%s\"",
 										 rv->schemaname, rv->relname, sub->name)));
@@ -900,6 +903,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				char		state;
 				XLogRecPtr	statelsn;
+				char		slotname[NAMEDATALEN] = {0};
 
 				/*
 				 * Lock pg_subscription_rel with AccessExclusiveLock to
@@ -926,7 +930,29 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/*
+				 * Find the logical replication sync worker. If exists, store
+				 * the slot number for dropping associated replication slots
+				 * later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot_name);
+				}
+				else
+				{
+					/*
+					 * Sync of this relation might be failed in an earlier
+					 * attempt, but the replication slot might still exist.
+					 */
+					GetSubscriptionRelReplicationSlot(sub->oid, relid, slotname);
+					if (strlen(slotname) > 0)
+						sub_remove_slots = lappend(sub_remove_slots, slotname);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
@@ -960,31 +986,24 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
-		 * the database operations we won't be able to rollback dropped slots.
+		 * Drop the replication slots associated with tablesync workers for
+		 * removed tables. This has to be at the end because otherwise if
+		 * there is an error while doing the database operations we won't be
+		 * able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			memcpy(syncslotname, lfirst(lc), sizeof(NAMEDATALEN));
+
+			/*
+			 * There is no certainty, maybe the slot does not exist yet. Also,
+			 * if we fail after removing some of the slots, next time, it will
+			 * again try to drop already dropped slots and fail. For these
+			 * reasons, we allow missing_ok = true for the drop.
+			 */
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1384,6 +1403,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	char	   *subname;
 	char	   *conninfo;
 	char	   *slotname;
+	int64		lastusedid;
 	List	   *subworkers;
 	ListCell   *lc;
 	char		originname[NAMEDATALEN];
@@ -1455,6 +1475,14 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	else
 		slotname = NULL;
 
+	/* Get the last used identifier by the subscription */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID, tup,
+							Anum_pg_subscription_sublastusedid, &isnull);
+	if (!isnull)
+		lastusedid = DatumGetInt64(datum);
+	else
+		lastusedid = 0;
+
 	/*
 	 * Since dropping a replication slot is not transactional, the replication
 	 * slot stays dropped even if the transaction rolls back.  So we cannot
@@ -1504,6 +1532,8 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	}
 	list_free(subworkers);
 
+	rstates = GetSubscriptionRelations(subid, true);
+
 	/*
 	 * Remove the no-longer-useful entry in the launcher's table of apply
 	 * worker start times.
@@ -1515,36 +1545,26 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	ApplyLauncherForgetWorkerStartTime(subid);
 
 	/*
-	 * Cleanup of tablesync replication origins.
-	 *
-	 * Any READY-state relations would already have dealt with clean-ups.
+	 * Cleanup of tablesync replication origins associated with the
+	 * subscription, if exists. Try to drop origins by creating all origin
+	 * names created for this subscription.
 	 *
 	 * Note that the state can't change because we have already stopped both
 	 * the apply and tablesync workers and they can't restart because of
 	 * exclusive lock on the subscription.
+	 *
+	 * XXX: This can be handled better instead of looping through all possible
 	 */
-	rstates = GetSubscriptionRelations(subid, true);
-	foreach(lc, rstates)
+	for (int64 i = 1; i <= lastusedid; i++)
 	{
-		SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-		Oid			relid = rstate->relid;
-
-		/* Only cleanup resources of tablesync workers */
-		if (!OidIsValid(relid))
-			continue;
+		char		originname_to_drop[NAMEDATALEN] = {0};
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * It is possible that the origin is not yet created for tablesync
-		 * worker so passing missing_ok = true. This can happen for the states
-		 * before SUBREL_STATE_FINISHEDCOPY.
-		 */
-		ReplicationOriginNameForLogicalRep(subid, relid, originname,
-										   sizeof(originname));
-		replorigin_drop_by_name(originname, true, false);
+		snprintf(originname_to_drop, sizeof(originname_to_drop), "pg_%u_%lld", subid, (long long) i);
+		/* missing_ok = true, since the origin might be already dropped. */
+		replorigin_drop_by_name(originname_to_drop, true, false);
 	}
 
+
 	/* Clean up dependencies */
 	deleteSharedDependencyRecordsFor(SubscriptionRelationId, subid, 0);
 
@@ -1596,39 +1616,17 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
+		List	   *slots = NULL;
 
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
 
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char	   *syncslotname = (char *) lfirst(lc);
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1651,6 +1649,71 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * Get the replication slot names associated with the subscription.
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok)
+{
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+					 " FROM pg_replication_slots"
+					 " WHERE slot_name LIKE 'pg_%i_sync_%%';",
+					 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+                    errmsg("could not receive list of slots associated with the subscription %u, error: %s",
+					subid, res->err));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();
+		return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -2005,6 +2068,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -2015,18 +2079,20 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
-			char		syncslotname[NAMEDATALEN] = {0};
-
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
-				 syncslotname);
+				 worker->slot_name);
 		}
 	}
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 3579e704fe..1cab625dfb 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -440,7 +440,8 @@ pa_launch_parallel_worker(void)
 										MySubscription->name,
 										MyLogicalRepWorker->userid,
 										InvalidOid,
-										dsm_segment_handle(winfo->dsm_seg));
+										dsm_segment_handle(winfo->dsm_seg),
+										0);
 
 	if (launched)
 	{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 970d170e73..5ea88cedce 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -304,7 +304,7 @@ logicalrep_workers_find(Oid subid, bool only_running)
  */
 bool
 logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
-						 Oid relid, dsm_handle subworker_dsm)
+						 Oid relid, dsm_handle subworker_dsm, int64 slotid)
 {
 	BackgroundWorker bgw;
 	BackgroundWorkerHandle *bgw_handle;
@@ -430,6 +430,9 @@ retry:
 	worker->launch_time = now;
 	worker->in_use = true;
 	worker->generation++;
+	worker->created_slot = false;
+	worker->rep_slot_id = slotid;
+	worker->slot_name = (char *) palloc(NAMEDATALEN);
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -437,6 +440,7 @@ retry:
 	worker->relid = relid;
 	worker->relstate = SUBREL_STATE_UNKNOWN;
 	worker->relstate_lsn = InvalidXLogRecPtr;
+	worker->ready_to_reuse = false;
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
@@ -1155,7 +1159,7 @@ ApplyLauncherMain(Datum main_arg)
 				ApplyLauncherSetWorkerStartTime(sub->oid, now);
 				logicalrep_worker_launch(sub->dbid, sub->oid, sub->name,
 										 sub->owner, InvalidOid,
-										 DSM_HANDLE_INVALID);
+										 DSM_HANDLE_INVALID, 0);
 			}
 			else
 			{
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 07eea504ba..ae49c729e7 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -127,11 +127,10 @@ static bool FetchTableStates(bool *started_tx);
 static StringInfo copybuf = NULL;
 
 /*
- * Exit routine for synchronization worker.
+ * Prepares the synchronization worker for reuse or exit.
  */
 static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+clean_sync_worker(void)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -143,18 +142,28 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
-	/* And flush all writes. */
-	XLogFlush(GetXLogWriteRecPtr());
-
-	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
-	CommitTransactionCommand();
+	/*
+	 * Disconnect from publisher. Otherwise reused sync workers causes
+	 * exceeding max_wal_senders
+	 */
+	walrcv_disconnect(LogRepWorkerWalRcvConn);
+	LogRepWorkerWalRcvConn = NULL;
 
 	/* Find the leader apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
+}
+
+/*
+ * Exit routine for synchronization worker.
+ */
+static void
+pg_attribute_noreturn()
+finish_sync_worker(void)
+{
+	clean_sync_worker();
+
+	/* And flush all writes. */
+	XLogFlush(GetXLogWriteRecPtr());
 
 	/* Stop gracefully */
 	proc_exit(0);
@@ -284,6 +293,10 @@ invalidate_syncing_table_states(Datum arg, int cacheid, uint32 hashvalue)
 static void
 process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 {
+	List	   *rstates;
+	SubscriptionRelState *rstate;
+	ListCell   *lc;
+
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_CATCHUP &&
@@ -292,6 +305,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		TimeLineID	tli;
 		char		syncslotname[NAMEDATALEN] = {0};
 		char		originname[NAMEDATALEN] = {0};
+		bool		is_streaming_ended = false;
 
 		MyLogicalRepWorker->relstate = SUBREL_STATE_SYNCDONE;
 		MyLogicalRepWorker->relstate_lsn = current_lsn;
@@ -308,40 +322,29 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 								   MyLogicalRepWorker->relid,
 								   MyLogicalRepWorker->relstate,
 								   MyLogicalRepWorker->relstate_lsn);
+		CommitTransactionCommand();
 
 		/*
-		 * End streaming so that LogRepWorkerWalRcvConn can be used to drop
-		 * the slot.
-		 */
-		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
-
-		/*
-		 * Cleanup the tablesync slot.
+		 * Cleanup the tablesync slot. If the slot name used by this worker is
+		 * different from the default slot name for the worker, this means the
+		 * current table had started to being synchronized by another worker
+		 * and replication slot. And this worker is reusing a replication slot
+		 * from a previous attempt. We do not need that replication slot
+		 * anymore.
 		 *
 		 * This has to be done after updating the state because otherwise if
 		 * there is an error while doing the database operations we won't be
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
+										MyLogicalRepWorker->rep_slot_id,
 										syncslotname,
 										sizeof(syncslotname));
 
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
-		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
-
-		CommitTransactionCommand();
-		pgstat_report_stat(false);
-
-		/*
-		 * Start a new transaction to clean up the tablesync origin tracking.
-		 * This transaction will be ended within the finish_sync_worker().
-		 * Now, even, if we fail to remove this here, the apply worker will
-		 * ensure to clean it up afterward.
+		 * We are safe to drop the replication trackin origin after this
+		 * point. Now, even, if we fail to remove this here, the apply worker
+		 * will ensure to clean it up afterward.
 		 *
 		 * We need to do this after the table state is set to SYNCDONE.
 		 * Otherwise, if an error occurs while performing the database
@@ -350,34 +353,129 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 * have been cleared before restart. So, the restarted worker will use
 		 * invalid replication progress state resulting in replay of
 		 * transactions that have already been applied.
+		 *
+		 * Firstly reset the origin session to remove the ownership of the
+		 * slot. This is needed to allow the origin to be dropped or reused
+		 * later.
 		 */
+		replorigin_session_reset();
+		replorigin_session_origin = InvalidRepOriginId;
+		replorigin_session_origin_lsn = InvalidXLogRecPtr;
+		replorigin_session_origin_timestamp = 0;
+
 		StartTransactionCommand();
+		if (MyLogicalRepWorker->slot_name && strcmp(syncslotname, MyLogicalRepWorker->slot_name) != 0)
+		{
+			/*
+			 * End streaming so that LogRepWorkerWalRcvConn can be used to
+			 * drop the slot.
+			 */
+			walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+			is_streaming_ended = true;
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
 
+			ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+											   MyLogicalRepWorker->relid,
+											   originname,
+											   sizeof(originname));
+
+			/* Drop replication origin */
+			replorigin_drop_by_name(originname, true, false);
+		}
+
+		/*
+		 * We are safe to remove persisted replication slot and origin data,
+		 * since it's already in SYNCDONE state. They will not be needed
+		 * anymore.
+		 */
+		UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+							  MyLogicalRepWorker->relid,
+							  MyLogicalRepWorker->relstate,
+							  MyLogicalRepWorker->relstate_lsn,
+							  NULL,
+							  NULL);
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid))));
+
+		CommitTransactionCommand();
+		pgstat_report_stat(false);
+
+		StartTransactionCommand();
+
+		/*
+		 * This should return the default origin name for the worker. Even if
+		 * the worker used a different origin for this table, it should be
+		 * dropped and removed from the catalog so far.
+		 */
 		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
 										   MyLogicalRepWorker->relid,
 										   originname,
 										   sizeof(originname));
 
 		/*
-		 * Resetting the origin session removes the ownership of the slot.
-		 * This is needed to allow the origin to be dropped.
+		 * Check if any table whose relation state is still INIT. If a table
+		 * in INIT state is found, the worker will not be finished, it will be
+		 * reused instead.
 		 */
-		replorigin_session_reset();
-		replorigin_session_origin = InvalidRepOriginId;
-		replorigin_session_origin_lsn = InvalidXLogRecPtr;
-		replorigin_session_origin_timestamp = 0;
+		rstates = GetSubscriptionRelations(MySubscription->oid, true);
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * There is a chance that the user is concurrently performing refresh
-		 * for the subscription where we remove the table state and its origin
-		 * or the apply worker would have removed this origin. So passing
-		 * missing_ok = true.
+		foreach(lc, rstates)
+		{
+			rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
+			memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
+
+			/*
+			 * Pick the table for the next run if it is not already picked up
+			 * by another worker.
+			 */
+			LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+			if (rstate->state != SUBREL_STATE_SYNCDONE &&
+				!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+			{
+				/* Update worker state for the next table */
+				MyLogicalRepWorker->relid = rstate->relid;
+				MyLogicalRepWorker->relstate = rstate->state;
+				MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+				MyLogicalRepWorker->ready_to_reuse = true;
+				LWLockRelease(LogicalRepWorkerLock);
+				break;
+			}
+			LWLockRelease(LogicalRepWorkerLock);
+		}
+
+		/* 
+		 * If the worker is ready to be reused, clean up the worker for next
+		 * relations. 
+		 * If there is no more work left for this worker, drop replication slot
+		 * and origin. Then stop the worker gracefully. 
 		 */
-		replorigin_drop_by_name(originname, true, false);
+		if (!MyLogicalRepWorker->ready_to_reuse)
+		{
+			/*
+			 * It is important to give an error if we are unable to drop the
+			 * slot, otherwise, it won't be dropped till the corresponding
+			 * subscription is dropped. So passing missing_ok = false.
+			 */
+			if (MyLogicalRepWorker->created_slot)
+			{
+				/* End streaming if it's not already ended. */
+				if (!is_streaming_ended)
+					walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+				ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
+			}
+
+			/* Drop replication origin before exiting. */
+			replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
+			finish_sync_worker();
+		}
+		else
+		{
+			clean_sync_worker();
+		}
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -464,6 +562,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 			if (current_lsn >= rstate->lsn)
 			{
 				char		originname[NAMEDATALEN];
+				bool		is_origin_null = true;
 
 				rstate->state = SUBREL_STATE_READY;
 				rstate->lsn = current_lsn;
@@ -484,18 +583,27 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				 * error while dropping we won't restart it to drop the
 				 * origin. So passing missing_ok = true.
 				 */
-				ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-												   rstate->relid,
-												   originname,
-												   sizeof(originname));
-				replorigin_drop_by_name(originname, true, false);
+				GetSubscriptionRelOrigin(MyLogicalRepWorker->subid,
+										 rstate->relid, originname,
+										 &is_origin_null);
+
+				if (!is_origin_null)
+				{
+					replorigin_drop_by_name(originname, true, false);
+				}
 
 				/*
 				 * Update the state to READY only after the origin cleanup.
 				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-										   rstate->relid, rstate->state,
-										   rstate->lsn);
+				UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+									  rstate->relid,
+									  rstate->state,
+									  rstate->lsn,
+									  NULL,
+									  NULL);
+
+				CommitTransactionCommand();
+				started_tx = false;
 			}
 		}
 		else
@@ -584,12 +692,22 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						TimestampDifferenceExceeds(hentry->last_start_time, now,
 												   wal_retrieve_retry_interval))
 					{
+						if (IsTransactionState())
+							CommitTransactionCommand();
+						StartTransactionCommand();
+						started_tx = true;
+
+						MySubscription->lastusedid++;
+						UpdateSubscriptionLastSlotId(MyLogicalRepWorker->subid,
+													 MySubscription->lastusedid);
+
 						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
 												 MySubscription->oid,
 												 MySubscription->name,
 												 MyLogicalRepWorker->userid,
 												 rstate->relid,
-												 DSM_HANDLE_INVALID);
+												 DSM_HANDLE_INVALID,
+												 MySubscription->lastusedid);
 						hentry->last_start_time = now;
 					}
 				}
@@ -1198,8 +1316,8 @@ copy_table(Relation rel)
  * The name must not exceed NAMEDATALEN - 1 because of remote node constraints
  * on slot name length. We append system_identifier to avoid slot_name
  * collision with subscriptions in other clusters. With the current scheme
- * pg_%u_sync_%u_UINT64_FORMAT (3 + 10 + 6 + 10 + 20 + '\0'), the maximum
- * length of slot_name will be 50.
+ * pg_%u_sync_%lu_UINT64_FORMAT (3 + 10 + 6 + 20 + 20 + '\0'), the maximum
+ * length of slot_name will be 45.
  *
  * The returned slot name is stored in the supplied buffer (syncslotname) with
  * the given size.
@@ -1210,11 +1328,11 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
+ReplicationSlotNameForTablesync(Oid suboid, int64 slotid,
 								char *syncslotname, Size szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%lld_" UINT64_FORMAT, suboid,
+			(long long) slotid, GetSystemIdentifier());
 }
 
 /*
@@ -1237,6 +1355,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	WalRcvExecResult *res;
 	char		originname[NAMEDATALEN];
 	RepOriginId originid;
+	char	   *prev_slotname;
 
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
@@ -1265,7 +1384,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
+									MyLogicalRepWorker->rep_slot_id,
 									slotname,
 									NAMEDATALEN);
 
@@ -1285,12 +1404,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC ||
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY);
 
+	/*
+	 * See if tablesync of the current relation has been started with another
+	 * replication slot.
+	 *
+	 * Read previous slot name from the catalog, if exists.
+	 */
+	prev_slotname = (char *) palloc(NAMEDATALEN);
+	StartTransactionCommand();
+	GetSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+									  MyLogicalRepWorker->relid,
+									  prev_slotname);
+
 	/* Assign the origin tracking record name. */
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   sizeof(originname));
 
+	CommitTransactionCommand();
+
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC)
 	{
 		/*
@@ -1304,10 +1437,49 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		 * breakdown then it wouldn't have succeeded so trying it next time
 		 * seems like a better bet.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, slotname, true);
+		if (strlen(prev_slotname) > 0)
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);
+
+			StartTransactionCommand();
+			/* Replication origin might still exist. Try to drop */
+			replorigin_drop_by_name(originname, true, false);
+
+			/*
+			 * Remove replication slot and origin name from the relation's
+			 * catalog record
+			 */
+			UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+								  MyLogicalRepWorker->relid,
+								  MyLogicalRepWorker->relstate,
+								  MyLogicalRepWorker->relstate_lsn,
+								  NULL,
+								  NULL);
+			CommitTransactionCommand();
+		}
 	}
 	else if (MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY)
 	{
+		/*
+		 * At this point, the table that is currently being synchronized
+		 * should have its replication slot name filled in the catalog. The
+		 * tablesync process was started with another sync worker and
+		 * replication slot. We need to continue using the same replication
+		 * slot in this worker too.
+		 */
+		if (strlen(prev_slotname) == 0)
+		{
+			elog(ERROR, "Replication slot could not be found for subscription %u, relation %u",
+				 MyLogicalRepWorker->subid,
+				 MyLogicalRepWorker->relid);
+		}
+
+		/*
+		 * Proceed with the correct replication slot. Use previously created
+		 * replication slot to sync this table.
+		 */
+		slotname = prev_slotname;
+
 		/*
 		 * The COPY phase was previously done, but tablesync then crashed
 		 * before it was able to finish normally.
@@ -1327,7 +1499,9 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 		goto copy_table_done;
 	}
+	pfree(prev_slotname);
 
+	/* Preparing for table copy operation */
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 	MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
 	MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
@@ -1335,10 +1509,12 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   MyLogicalRepWorker->relstate,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  MyLogicalRepWorker->relstate,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 	CommitTransactionCommand();
 	pgstat_report_stat(true);
 
@@ -1377,6 +1553,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						GetUserNameFromId(GetUserId(), true),
 						RelationGetRelationName(rel))));
 
+
 	/*
 	 * Start a transaction in the remote node in REPEATABLE READ mode.  This
 	 * ensures that both the replication slot we create (see below) and the
@@ -1392,48 +1569,91 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						res->err)));
 	walrcv_clear_result(res);
 
+	originid = replorigin_by_name(originname, true);
+
 	/*
 	 * Create a new permanent logical decoding slot. This slot will be used
 	 * for the catchup phase after COPY is done, so tell it to use the
 	 * snapshot to make the final data consistent.
+	 *
+	 * Replication slot will only be created if either this is the first run
+	 * of the worker or we're not using a previous replication slot.
 	 */
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-
-	/*
-	 * Setup replication origin tracking. The purpose of doing this before the
-	 * copy is to avoid doing the copy again due to any error in setting up
-	 * origin tracking.
-	 */
-	originid = replorigin_by_name(originname, true);
-	if (!OidIsValid(originid))
+	if (!MyLogicalRepWorker->created_slot)
 	{
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						   slotname, false /* permanent */ , false /* two_phase */ ,
+						   CRS_USE_SNAPSHOT, origin_startpos);
+
 		/*
-		 * Origin tracking does not exist, so create it now.
-		 *
-		 * Then advance to the LSN got from walrcv_create_slot. This is WAL
-		 * logged for the purpose of recovery. Locks are to prevent the
-		 * replication origin from vanishing while advancing.
+		 * Remember that we created the slot so that we will not try to create
+		 * it again.
 		 */
-		originid = replorigin_create(originname);
-
-		LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
-		replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
-						   true /* go backward */ , true /* WAL log */ );
-		UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->created_slot = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
+		/*
+		 * Setup replication origin tracking. The purpose of doing this before
+		 * the copy is to avoid doing the copy again due to any error in
+		 * setting up origin tracking.
+		 */
+		if (!OidIsValid(originid))
+		{
+			/*
+			 * Origin tracking does not exist, so create it now.
+			 */
+			originid = replorigin_create(originname);
+		}
+		else
+		{
+			/*
+			 * At this point, there shouldn't be any existing replication
+			 * origin with the same name.
+			 */
+			ereport(ERROR,
+					(errcode(ERRCODE_DUPLICATE_OBJECT),
+					 errmsg("replication origin \"%s\" already exists",
+							originname)));
+		}
 	}
 	else
 	{
-		ereport(ERROR,
-				(errcode(ERRCODE_DUPLICATE_OBJECT),
-				 errmsg("replication origin \"%s\" already exists",
-						originname)));
+		/*
+		 * Do not create a new replication slot, reuse the existing one
+		 * instead. Use a new snapshot for the replication slot to ensure that
+		 * tablesync and apply proceses are consistent with each other.
+		 */
+		WalRcvStreamOptions options;
+		int			server_version;
+
+		server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+		options.proto.logical.proto_version =
+			server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+			server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+			LOGICALREP_PROTO_VERSION_NUM;
+		options.proto.logical.publication_names = MySubscription->publications;
+
+		HOLD_INTERRUPTS();
+		walrcv_slot_snapshot(LogRepWorkerWalRcvConn, slotname, &options, origin_startpos);
+		RESUME_INTERRUPTS();
 	}
 
+	/*
+	 * Advance to the LSN got from walrcv_create_slot or walrcv_slot_snapshot.
+	 * This is WAL logged for the purpose of recovery. Locks are to prevent
+	 * the replication origin from vanishing while advancing.
+	 *
+	 * Then setup replication origin tracking.
+	 */
+	LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+	replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
+					   true /* go backward */ , true /* WAL log */ );
+	UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+
 	/* Now do the initial data copy */
 	PushActiveSnapshot(GetTransactionSnapshot());
 	copy_table(rel);
@@ -1456,10 +1676,12 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	 * Update the persisted state to indicate the COPY phase is done; make it
 	 * visible to others.
 	 */
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   SUBREL_STATE_FINISHEDCOPY,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  SUBREL_STATE_FINISHEDCOPY,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 
 	CommitTransactionCommand();
 
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index cfb2ab6248..fd39254ecc 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -388,6 +388,7 @@ static void stream_open_file(Oid subid, TransactionId xid,
 static void stream_write_change(char action, StringInfo s);
 static void stream_open_and_write_change(TransactionId xid, char action, StringInfo s);
 static void stream_close_file(void);
+static void stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
@@ -456,8 +457,16 @@ ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 {
 	if (OidIsValid(relid))
 	{
-		/* Replication origin name for tablesync workers. */
-		snprintf(originname, szoriginname, "pg_%u_%u", suboid, relid);
+		bool		is_null = true;
+
+		/*
+		 * Replication origin name for tablesync workers. First, look into the
+		 * catalog. If originname does not exist, then use the default name.
+		 */
+		GetSubscriptionRelOrigin(suboid, relid,
+								 originname, &is_null);
+		if (is_null)
+			snprintf(originname, szoriginname, "pg_%u_%lld", suboid, (long long) MyLogicalRepWorker->rep_slot_id);
 	}
 	else
 	{
@@ -3576,6 +3585,23 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can move to next table while running
+				 * process_syncing_tables_for_sync. Before we were able to
+				 * reuse tablesync workers, that
+				 * process_syncing_tables_for_sync call would exit the worker
+				 * instead of moving to next table. Now that tablesync workers
+				 * can be reused, we need to take care of memory contexts here
+				 * before moving to sync a table.
+				 */
+				if (MyLogicalRepWorker->ready_to_reuse)
+				{
+					MemoryContextResetAndDeleteChildren(ApplyMessageContext);
+					MemoryContextSwitchTo(TopMemoryContext);
+					return;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -3595,6 +3621,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+			if (MyLogicalRepWorker->ready_to_reuse)
+			{
+				endofstream = true;
+			}
 		}
 
 		/* Cleanup the memory. */
@@ -3697,8 +3727,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * If it's moving to next relation, this is a sync worker. Sync workers
+	 * end the streaming during process_syncing_tables_for_sync. Calling
+	 * endstreaming twice causes "no COPY in progress" errors.
+	 */
+	if (!MyLogicalRepWorker->ready_to_reuse)
+	{
+		/* All done */
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	}
 }
 
 /*
@@ -4282,6 +4320,56 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+ /* stream_build_options
+  * 	Build logical replication streaming options.
+  *
+  * This function sets streaming options including replication slot name
+  * and origin start position. Workers need these options for logical replication.
+  */
+static void
+stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4356,6 +4444,9 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 
 	/* allocate slot name in long-lived context */
 	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+
+	/* Keep the replication slot name used for this sync. */
+	MyLogicalRepWorker->slot_name = *myslotname;
 	pfree(syncslotname);
 }
 
@@ -4393,6 +4484,135 @@ start_apply(XLogRecPtr origin_startpos)
 	PG_END_TRY();
 }
 
+/*
+ * Runs the tablesync worker.
+ * It starts table sync. After successful sync,
+ * builds streaming options and starts streaming.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Set this to false for safety, in case we're already reusing the worker */
+	MyLogicalRepWorker->ready_to_reuse = false;
+
+	start_table_sync(origin_startpos, &slotname);
+
+	/*
+	 * Allocate the origin name in long-lived context for error context
+	 * message.
+	 */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   originname_size);
+	CommitTransactionCommand();
+
+	set_apply_error_context_origin(originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/* Start normal logical streaming replication. */
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+}
+
+/*
+ * Runs the apply worker.
+ * It sets up replication origin, the streaming options
+ * and then starts streaming.
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
+{
+	/* This is the leader apply worker */
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+
+	slotname = MySubscription->slotname;
+
+	/*
+		* This shouldn't happen if the subscription is enabled, but guard
+		* against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+		* crash if slot is NULL.)
+		*/
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											MySubscription->name, &err);
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+		* We don't really use the output identify_system for anything but it
+		* does some initializations on the upstream so let's still call it.
+		*/
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+	{
+		/* Start streaming with two_phase enabled */
+		options->proto.logical.twophase = true;
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
+	}
+	else
+	{
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+	}
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+}
+
 /*
  * Common initialization for leader apply worker and parallel apply worker.
  *
@@ -4485,7 +4705,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	/* Attach to slot */
 	logicalrep_worker_attach(worker_slot);
@@ -4513,156 +4732,46 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
-	 * the subscription relation state.
+	 * the subscription relation state. Do this outside the loop to avoid
+	 * exceeding MAX_SYSCACHE_CALLBACKS
 	 */
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
-
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
-
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-
 	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 * The loop where worker does its job. It loops until the worker is not
+	 * reused.
 	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
-
-	if (!am_tablesync_worker())
-	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
+	do {
+		if (am_tablesync_worker())
 		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
+			/*
+			 * This is a tablesync worker. Start syncing tables before
+			 * starting the apply loop.
+			 */
+			run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 		}
 		else
 		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+			/* This is main apply worker */
+			run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 		}
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-						MySubscription->name,
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-						"?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
-
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+		/* Run the main loop. */
+		start_apply(origin_startpos);
 
+		if (MyLogicalRepWorker->ready_to_reuse)
+		{
+			StartTransactionCommand();
+			ereport(LOG,
+					(errmsg("logical replication table synchronization worker for subscription \"%s\" has moved to sync table \"%s\".",
+							MySubscription->name, get_rel_name(MyLogicalRepWorker->relid))));
+			CommitTransactionCommand();
+		}
+	} while (MyLogicalRepWorker->ready_to_reuse);
 	proc_exit(0);
 }
 
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index b0f2a1705d..a0ee12e259 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -103,6 +103,9 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
 
 	/* Only publish data originating from the specified origin */
 	text		suborigin BKI_DEFAULT(LOGICALREP_ORIGIN_ANY);
+
+	/* The last used ID to create a replication slot for tablesync */
+	int64		sublastusedid BKI_DEFAULT(0);
 #endif
 } FormData_pg_subscription;
 
@@ -137,6 +140,8 @@ typedef struct Subscription
 	List	   *publications;	/* List of publication names to subscribe to */
 	char	   *origin;			/* Only publish data originating from the
 								 * specified origin */
+	int64		lastusedid;		/* Last used unique ID to create replication
+								 * slots in tablesync */
 } Subscription;
 
 /* Disallow streaming in-progress transactions. */
@@ -157,6 +162,7 @@ typedef struct Subscription
 extern Subscription *GetSubscription(Oid subid, bool missing_ok);
 extern void FreeSubscription(Subscription *sub);
 extern void DisableSubscription(Oid subid);
+extern void UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid);
 
 extern int	CountDBSubscriptions(Oid dbid);
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 60a2bcca23..a528866012 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -44,6 +44,12 @@ CATALOG(pg_subscription_rel,6102,SubscriptionRelRelationId)
 											 * used for synchronization
 											 * coordination, or NULL if not
 											 * valid */
+	NameData	srrelslotname BKI_FORCE_NULL;	/* name of the replication
+												 * slot for relation in
+												 * subscription */
+	NameData	srreloriginname BKI_FORCE_NULL; /* origin name for relation in
+												 * subscription */
+
 #endif
 } FormData_pg_subscription_rel;
 
@@ -81,10 +87,17 @@ typedef struct SubscriptionRelState
 } SubscriptionRelState;
 
 extern void AddSubscriptionRelState(Oid subid, Oid relid, char state,
-									XLogRecPtr sublsn);
+									XLogRecPtr sublsn, char *relslotname, char *reloriginname);
 extern void UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 									   XLogRecPtr sublsn);
+extern void UpdateSubscriptionRel(Oid subid, Oid relid, char state,
+								  XLogRecPtr sublsn, char *relslotname, char *reloriginname);
+extern void UpdateSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *relslotname);
+
 extern char GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn);
+extern void GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname);
+extern void GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull);
+
 extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 
 extern bool HasSubscriptionRelations(Oid subid);
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 8872c80cdf..3547daaaec 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -219,8 +219,9 @@ extern bool InvalidateObsoleteReplicationSlots(XLogSegNo oldestSegno);
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
 extern bool ReplicationSlotName(int index, Name name);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, Size szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, int64 slotid, char *syncslotname, Size szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List *GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index dc87a4edd1..7f7cb75c81 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -35,6 +35,23 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/*
+	 * Indicates if the sync worker created a replication slot for itself
+	 * in any point of its lifetime.
+	 * False means that the worker has not created a slot yet, and has been
+	 * reusing replication slots created by other workers so far.
+	 */
+	bool		created_slot;
+
+	/*
+	 * Unique identifier for replication slot to be created by tablesnync
+	 * workers, if needed.
+	 */
+	int64		rep_slot_id;
+
+	/* Replication slot name used by the worker. */
+	char	   *slot_name;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
@@ -56,6 +73,12 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/*
+	 * Used to indicate whether sync worker will be reused for another
+	 * relation
+	 */
+	bool		ready_to_reuse;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction or
@@ -231,7 +254,8 @@ extern LogicalRepWorker *logicalrep_worker_find(Oid subid, Oid relid,
 extern List *logicalrep_workers_find(Oid subid, bool only_running);
 extern bool logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname,
 									 Oid userid, Oid relid,
-									 dsm_handle subworker_dsm);
+									 dsm_handle subworker_dsm,
+									 int64 slotid);
 extern void logicalrep_worker_stop(Oid subid, Oid relid);
 extern void logicalrep_pa_worker_stop(int slot_no, uint16 generation);
 extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
-- 
2.25.1

v7-0001-Add-replication-protocol-cmd-to-create-a-snapshot.patchapplication/octet-stream; name=v7-0001-Add-replication-protocol-cmd-to-create-a-snapshot.patchDownload
From 2959b1ae74318e6b110050b452ca06ad87f9cb6a Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 13 Oct 2022 17:05:45 +0300
Subject: [PATCH 1/2] Add replication protocol cmd to create a snapshot

Introduced CREATE_REPLICATION_SNAPSHOT to be able to create and use a
snapshot without creating a new replication slot, but by using an
existing slot.

CREATE_REPLICATION_SNAPSHOT simply does what CREATE_REPLICATION_SLOT does
without creating a new replication slot.

CREATE_REPLICATION_SNAPSHOT command imports the snapshot into the current
transaction and returns consistent_point. The changes earlier than the
consistent_point will be applied by importing the snapshot. All changes
later than the consistent_point will be available to be consumed from
the replication slot.

This is useful for reusing replication slots in logical replication.
Otherwise, tablesync workers cannot start from a consistent point to copy a relation and then apply changes by consuming from replication slot.
---
 doc/src/sgml/protocol.sgml                    |  31 ++++++
 .../libpqwalreceiver/libpqwalreceiver.c       |  69 +++++++++++-
 src/backend/replication/logical/logical.c     |  40 ++++++-
 .../replication/logical/logicalfuncs.c        |   1 +
 src/backend/replication/repl_gram.y           |  18 +++-
 src/backend/replication/repl_scanner.l        |   2 +
 src/backend/replication/slotfuncs.c           |   1 +
 src/backend/replication/walsender.c           | 102 +++++++++++++++++-
 src/include/nodes/replnodes.h                 |  11 ++
 src/include/replication/logical.h             |   1 +
 src/include/replication/walreceiver.h         |  13 +++
 src/tools/pgindent/typedefs.list              |   2 +
 12 files changed, 287 insertions(+), 4 deletions(-)

diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 93fc7167d4..948a5aabd4 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2613,6 +2613,37 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
      </listitem>
     </varlistentry>
 
+    <varlistentry id="protocol-replication-replication-slot-snapshot">
+     <term><literal>CREATE_REPLICATION_SNAPSHOT</literal> <replaceable class="parameter">slot_name</replaceable> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
+      <indexterm><primary>CREATE_REPLICATION_SNAPSHOT</primary></indexterm>
+     </term>
+     <listitem>
+      <para>
+       Creates a snapshot including all the changes from the replication slot until
+       the point at which the replication slot becomes consistent. Then the snapshot
+       is used in the current transaction. This command is currently only supported
+       for logical replication slots.
+      </para>
+
+      <para>
+       In response to this command, the server will return a one-row result set,
+       containing the following field:
+       <variablelist>
+        <varlistentry>
+         <term><literal>consistent_point</literal> (<type>text</type>)</term>
+         <listitem>
+          <para>
+           The WAL location at which the slot became consistent.  This is the
+           earliest location from which streaming can start on this replication
+           slot.
+          </para>
+         </listitem>
+        </varlistentry>
+       </variablelist>
+      </para>
+     </listitem>
+    </varlistentry>
+
     <varlistentry id="protocol-replication-base-backup" xreflabel="BASE_BACKUP">
      <term><literal>BASE_BACKUP</literal> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
       <indexterm><primary>BASE_BACKUP</primary></indexterm>
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index 560ec974fa..9173163a07 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -81,6 +81,8 @@ static WalRcvExecResult *libpqrcv_exec(WalReceiverConn *conn,
 									   const int nRetTypes,
 									   const Oid *retTypes);
 static void libpqrcv_disconnect(WalReceiverConn *conn);
+static void libpqrcv_slot_snapshot(WalReceiverConn *conn, char *slotname,
+								   const WalRcvStreamOptions *options, XLogRecPtr *lsn);
 
 static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_connect = libpqrcv_connect,
@@ -97,7 +99,8 @@ static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_create_slot = libpqrcv_create_slot,
 	.walrcv_get_backend_pid = libpqrcv_get_backend_pid,
 	.walrcv_exec = libpqrcv_exec,
-	.walrcv_disconnect = libpqrcv_disconnect
+	.walrcv_disconnect = libpqrcv_disconnect,
+	.walrcv_slot_snapshot = libpqrcv_slot_snapshot
 };
 
 /* Prototypes for private functions */
@@ -933,6 +936,70 @@ libpqrcv_create_slot(WalReceiverConn *conn, const char *slotname,
 	return snapshot;
 }
 
+/*
+ * TODO
+ */
+static void
+libpqrcv_slot_snapshot(WalReceiverConn *conn,
+					   char *slotname,
+					   const WalRcvStreamOptions *options,
+					   XLogRecPtr *lsn)
+{
+	StringInfoData cmd;
+	PGresult   *res;
+	char	   *pubnames_str;
+	List	   *pubnames;
+	char	   *pubnames_literal;
+
+	initStringInfo(&cmd);
+
+	/* Build the command. */
+	appendStringInfo(&cmd, "CREATE_REPLICATION_SNAPSHOT \"%s\"", slotname);
+	appendStringInfoString(&cmd, " (");
+	appendStringInfo(&cmd, " proto_version '%u'",
+					 options->proto.logical.proto_version);
+
+	/* Add publication names. */
+	pubnames = options->proto.logical.publication_names;
+	pubnames_str = stringlist_to_identifierstr(conn->streamConn, pubnames);
+	if (!pubnames_str)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	pubnames_literal = PQescapeLiteral(conn->streamConn, pubnames_str,
+									   strlen(pubnames_str));
+	if (!pubnames_literal)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	appendStringInfo(&cmd, ", publication_names %s", pubnames_literal);
+	PQfreemem(pubnames_literal);
+	pfree(pubnames_str);
+
+	appendStringInfoString(&cmd, " )");
+
+	/* Execute the command. */
+	res = libpqrcv_PQexec(conn->streamConn, cmd.data);
+	pfree(cmd.data);
+
+	if (PQresultStatus(res) != PGRES_TUPLES_OK)
+	{
+		PQclear(res);
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg("Could not create a snapshot by replication slot \"%s\": %s",
+						slotname, pchomp(PQerrorMessage(conn->streamConn)))));
+	}
+
+	if (lsn)
+		*lsn = DatumGetLSN(DirectFunctionCall1Coll(pg_lsn_in, InvalidOid,
+												   CStringGetDatum(PQgetvalue(res, 0, 0))));
+
+	PQclear(res);
+}
+
 /*
  * Return PID of remote backend process.
  */
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 1a58dd7649..d1504f7ec6 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -461,6 +461,10 @@ CreateInitDecodingContext(const char *plugin,
  * fast_forward
  *		bypass the generation of logical changes.
  *
+ * need_full_snapshot
+ * 		if true, create a snapshot able to read all tables,
+ * 		otherwise do not create any snapshot.
+ *
  * xl_routine
  *		XLogReaderRoutine used by underlying xlogreader
  *
@@ -479,6 +483,7 @@ LogicalDecodingContext *
 CreateDecodingContext(XLogRecPtr start_lsn,
 					  List *output_plugin_options,
 					  bool fast_forward,
+					  bool need_full_snapshot,
 					  XLogReaderRoutine *xl_routine,
 					  LogicalOutputPluginWriterPrepareWrite prepare_write,
 					  LogicalOutputPluginWriterWrite do_write,
@@ -487,6 +492,7 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 	LogicalDecodingContext *ctx;
 	ReplicationSlot *slot;
 	MemoryContext old_context;
+	TransactionId xmin_horizon = InvalidTransactionId;
 
 	/* shorter lines... */
 	slot = MyReplicationSlot;
@@ -533,8 +539,40 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 		start_lsn = slot->data.confirmed_flush;
 	}
 
+
+	/*
+	 * We need to determine a safe xmin horizon to start decoding from if we
+	 * want to create a snapshot too. Otherwise we would end up with a
+	 * snapshot that cannot be imported since xmin value from the snapshot may
+	 * be less than the oldest safe xmin. To avoid this call
+	 * GetOldestSafeDecodingTransactionId() to return a safe xmin value, which
+	 * can be used while exporting/importing the snapshot.
+	 *
+	 * So we have to acquire the ProcArrayLock to prevent computation of new
+	 * xmin horizons by other backends, get the safe decoding xid, and inform
+	 * the slot machinery about the new limit. Once that's done the
+	 * ProcArrayLock can be released as the slot machinery now is protecting
+	 * against vacuum.
+	 */
+	if (need_full_snapshot)
+	{
+		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+		xmin_horizon = GetOldestSafeDecodingTransactionId(!need_full_snapshot);
+
+		SpinLockAcquire(&slot->mutex);
+		slot->effective_catalog_xmin = xmin_horizon;
+		slot->data.catalog_xmin = xmin_horizon;
+		slot->effective_xmin = xmin_horizon;
+		SpinLockRelease(&slot->mutex);
+
+		ReplicationSlotsComputeRequiredXmin(true);
+
+		LWLockRelease(ProcArrayLock);
+	}
+
 	ctx = StartupDecodingContext(output_plugin_options,
-								 start_lsn, InvalidTransactionId, false,
+								 start_lsn, xmin_horizon, need_full_snapshot,
 								 fast_forward, xl_routine, prepare_write,
 								 do_write, update_progress);
 
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index fa1b641a2b..1191c70eb0 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -208,6 +208,7 @@ pg_logical_slot_get_changes_guts(FunctionCallInfo fcinfo, bool confirm, bool bin
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									options,
 									false,
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/repl_gram.y b/src/backend/replication/repl_gram.y
index 0c874e33cf..957a5cc653 100644
--- a/src/backend/replication/repl_gram.y
+++ b/src/backend/replication/repl_gram.y
@@ -65,6 +65,7 @@ Node *replication_parse_result;
 %token K_CREATE_REPLICATION_SLOT
 %token K_DROP_REPLICATION_SLOT
 %token K_TIMELINE_HISTORY
+%token K_CREATE_REPLICATION_SNAPSHOT
 %token K_WAIT
 %token K_TIMELINE
 %token K_PHYSICAL
@@ -80,7 +81,7 @@ Node *replication_parse_result;
 %type <node>	command
 %type <node>	base_backup start_replication start_logical_replication
 				create_replication_slot drop_replication_slot identify_system
-				read_replication_slot timeline_history show
+				read_replication_slot timeline_history show create_replication_snapshot
 %type <list>	generic_option_list
 %type <defelt>	generic_option
 %type <uintval>	opt_timeline
@@ -114,6 +115,7 @@ command:
 			| read_replication_slot
 			| timeline_history
 			| show
+			| create_replication_snapshot
 			;
 
 /*
@@ -307,6 +309,19 @@ timeline_history:
 				}
 			;
 
+/*
+ * CREATE_REPLICATION_SNAPSHOT %s options
+ */
+create_replication_snapshot:
+			K_CREATE_REPLICATION_SNAPSHOT var_name plugin_options
+				{
+					CreateReplicationSnapshotCmd *n = makeNode(CreateReplicationSnapshotCmd);
+					n->slotname = $2;
+					n->options = $3;
+					$$ = (Node *) n;
+				}
+			;
+
 opt_physical:
 			K_PHYSICAL
 			| /* EMPTY */
@@ -400,6 +415,7 @@ ident_or_keyword:
 			| K_CREATE_REPLICATION_SLOT	{ $$ = "create_replication_slot"; }
 			| K_DROP_REPLICATION_SLOT		{ $$ = "drop_replication_slot"; }
 			| K_TIMELINE_HISTORY			{ $$ = "timeline_history"; }
+			| K_CREATE_REPLICATION_SNAPSHOT	{ $$ = "create_replication_snapshot"; }
 			| K_WAIT						{ $$ = "wait"; }
 			| K_TIMELINE					{ $$ = "timeline"; }
 			| K_PHYSICAL					{ $$ = "physical"; }
diff --git a/src/backend/replication/repl_scanner.l b/src/backend/replication/repl_scanner.l
index cb467ca46f..5ba2e9c54b 100644
--- a/src/backend/replication/repl_scanner.l
+++ b/src/backend/replication/repl_scanner.l
@@ -126,6 +126,7 @@ START_REPLICATION	{ return K_START_REPLICATION; }
 CREATE_REPLICATION_SLOT		{ return K_CREATE_REPLICATION_SLOT; }
 DROP_REPLICATION_SLOT		{ return K_DROP_REPLICATION_SLOT; }
 TIMELINE_HISTORY	{ return K_TIMELINE_HISTORY; }
+CREATE_REPLICATION_SNAPSHOT	{ return K_CREATE_REPLICATION_SNAPSHOT; }
 PHYSICAL			{ return K_PHYSICAL; }
 RESERVE_WAL			{ return K_RESERVE_WAL; }
 LOGICAL				{ return K_LOGICAL; }
@@ -303,6 +304,7 @@ replication_scanner_is_replication_command(void)
 		case K_DROP_REPLICATION_SLOT:
 		case K_READ_REPLICATION_SLOT:
 		case K_TIMELINE_HISTORY:
+		case K_CREATE_REPLICATION_SNAPSHOT:
 		case K_SHOW:
 			/* Yes; push back the first token so we can parse later. */
 			repl_pushed_back_token = first_token;
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 2f3c964824..b3ae11b2c8 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -478,6 +478,7 @@ pg_logical_replication_slot_advance(XLogRecPtr moveto)
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									NIL,
 									true,	/* fast_forward */
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 4ed3747e3f..b10ae3eada 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -238,6 +238,7 @@ static void CreateReplicationSlot(CreateReplicationSlotCmd *cmd);
 static void DropReplicationSlot(DropReplicationSlotCmd *cmd);
 static void StartReplication(StartReplicationCmd *cmd);
 static void StartLogicalReplication(StartReplicationCmd *cmd);
+static void CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd);
 static void ProcessStandbyMessage(void);
 static void ProcessStandbyReplyMessage(void);
 static void ProcessStandbyHSFeedbackMessage(void);
@@ -1280,7 +1281,7 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	 * are reported early.
 	 */
 	logical_decoding_ctx =
-		CreateDecodingContext(cmd->startpoint, cmd->options, false,
+		CreateDecodingContext(cmd->startpoint, cmd->options, false, false,
 							  XL_ROUTINE(.page_read = logical_read_xlog_page,
 										 .segment_open = WalSndSegmentOpen,
 										 .segment_close = wal_segment_close),
@@ -1332,6 +1333,96 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	EndCommand(&qc, DestRemote, false);
 }
 
+/*
+ * Create a snapshot from an existing replication slot.
+ */
+static void
+CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd)
+{
+	Snapshot	snap;
+	LogicalDecodingContext *ctx;
+	char		xloc[MAXFNAMELEN];
+	DestReceiver *dest;
+	TupOutputState *tstate;
+	TupleDesc	tupdesc;
+	Datum		values[1];
+	bool		nulls[1] = {0};
+
+	Assert(!MyReplicationSlot);
+
+	if (!IsTransactionBlock())
+		ereport(ERROR,
+				(errmsg("%s must be called inside a transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (XactIsoLevel != XACT_REPEATABLE_READ)
+		ereport(ERROR,
+				(errmsg("%s must be called in REPEATABLE READ isolation mode transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (!XactReadOnly)
+		ereport(ERROR,
+				(errmsg("%s must be called in a read only transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (FirstSnapshotSet)
+		ereport(ERROR,
+				(errmsg("%s must be called before any query",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (IsSubTransaction())
+		ereport(ERROR,
+				(errmsg("%s must not be called in a subtransaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	ReplicationSlotAcquire(cmd->slotname, false);
+
+	ctx = CreateDecodingContext(MyReplicationSlot->data.restart_lsn,
+								cmd->options,
+								false,
+								true,
+								XL_ROUTINE(.page_read = logical_read_xlog_page,
+										   .segment_open = WalSndSegmentOpen,
+										   .segment_close = wal_segment_close),
+								WalSndPrepareWrite, WalSndWriteData,
+								WalSndUpdateProgress);
+
+	/*
+	 * Signal that we don't need the timeout mechanism. We're just creating
+	 * the snapshot with the replication slot and don't yet accept feedback
+	 * messages or send keepalives. As we possibly need to wait for further
+	 * WAL the walsender would otherwise possibly be killed too soon.
+	 */
+	last_reply_timestamp = 0;
+
+	/* build initial snapshot, might take a while */
+	DecodingContextFindStartpoint(ctx);
+
+	snap = SnapBuildInitialSnapshot(ctx->snapshot_builder);
+	RestoreTransactionSnapshot(snap, MyProc);
+
+	/* Don't need the decoding context anymore */
+	FreeDecodingContext(ctx);
+
+	/* Create a tuple to send consistent WAL location */
+	snprintf(xloc, sizeof(xloc), "%X/%X",
+			 LSN_FORMAT_ARGS(MyReplicationSlot->data.confirmed_flush));
+
+	dest = CreateDestReceiver(DestRemoteSimple);
+	tupdesc = CreateTemplateTupleDesc(1);
+	TupleDescInitBuiltinEntry(tupdesc, (AttrNumber) 1, "consistent_point",
+							  TEXTOID, -1, 0);
+	tstate = begin_tup_output_tupdesc(dest, tupdesc, &TTSOpsVirtual);
+
+	/* consistent wal location */
+	values[0] = CStringGetTextDatum(xloc);
+
+	do_tup_output(tstate, values, nulls);
+	end_tup_output(tstate);
+
+	ReplicationSlotRelease();
+}
+
 /*
  * LogicalDecodingContext 'prepare_write' callback.
  *
@@ -1860,6 +1951,15 @@ exec_replication_command(const char *cmd_string)
 			}
 			break;
 
+		case T_CreateReplicationSnapshotCmd:
+			{
+				cmdtag = "CREATE_REPLICATION_SNAPSHOT";
+				set_ps_display(cmdtag);
+				CreateReplicationSnapshot((CreateReplicationSnapshotCmd *) cmd_node);
+				EndReplicationCommand(cmdtag);
+				break;
+			}
+
 		default:
 			elog(ERROR, "unrecognized replication command node tag: %u",
 				 cmd_node->type);
diff --git a/src/include/nodes/replnodes.h b/src/include/nodes/replnodes.h
index 4321ba8f86..154ab74f33 100644
--- a/src/include/nodes/replnodes.h
+++ b/src/include/nodes/replnodes.h
@@ -108,4 +108,15 @@ typedef struct TimeLineHistoryCmd
 	TimeLineID	timeline;
 } TimeLineHistoryCmd;
 
+/* ----------------------
+ *		CREATE_REPLICATION_SNAPSHOT command
+ * ----------------------
+ */
+typedef struct CreateReplicationSnapshotCmd
+{
+	NodeTag		type;
+	char	   *slotname;
+	List	   *options;
+} CreateReplicationSnapshotCmd;
+
 #endif							/* REPLNODES_H */
diff --git a/src/include/replication/logical.h b/src/include/replication/logical.h
index 5f49554ea0..6535786a0e 100644
--- a/src/include/replication/logical.h
+++ b/src/include/replication/logical.h
@@ -125,6 +125,7 @@ extern LogicalDecodingContext *CreateInitDecodingContext(const char *plugin,
 extern LogicalDecodingContext *CreateDecodingContext(XLogRecPtr start_lsn,
 													 List *output_plugin_options,
 													 bool fast_forward,
+													 bool need_full_snapshot,
 													 XLogReaderRoutine *xl_routine,
 													 LogicalOutputPluginWriterPrepareWrite prepare_write,
 													 LogicalOutputPluginWriterWrite do_write,
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index decffe352d..bd11f9f31e 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -384,6 +384,16 @@ typedef WalRcvExecResult *(*walrcv_exec_fn) (WalReceiverConn *conn,
  */
 typedef void (*walrcv_disconnect_fn) (WalReceiverConn *conn);
 
+/*
+ * walrcv_slot_snapshot_fn
+ *
+ * Create a snapshot by an existing replication slot
+ */
+typedef void (*walrcv_slot_snapshot_fn) (WalReceiverConn *conn,
+										 char *slotname,
+										 const WalRcvStreamOptions *options,
+										 XLogRecPtr *lsn);
+
 typedef struct WalReceiverFunctionsType
 {
 	walrcv_connect_fn walrcv_connect;
@@ -401,6 +411,7 @@ typedef struct WalReceiverFunctionsType
 	walrcv_get_backend_pid_fn walrcv_get_backend_pid;
 	walrcv_exec_fn walrcv_exec;
 	walrcv_disconnect_fn walrcv_disconnect;
+	walrcv_slot_snapshot_fn walrcv_slot_snapshot;
 } WalReceiverFunctionsType;
 
 extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
@@ -435,6 +446,8 @@ extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
 	WalReceiverFunctions->walrcv_exec(conn, exec, nRetTypes, retTypes)
 #define walrcv_disconnect(conn) \
 	WalReceiverFunctions->walrcv_disconnect(conn)
+#define walrcv_slot_snapshot(conn, slotname, options, lsn) \
+	WalReceiverFunctions->walrcv_slot_snapshot(conn, slotname, options, lsn)
 
 static inline void
 walrcv_clear_result(WalRcvExecResult *walres)
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 07fbb7ccf6..a3de441b7e 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2323,6 +2323,7 @@ ReplicationSlotCtlData
 ReplicationSlotOnDisk
 ReplicationSlotPersistency
 ReplicationSlotPersistentData
+CreateReplicationSnapshotCmd
 ReplicationState
 ReplicationStateCtl
 ReplicationStateOnDisk
@@ -3796,6 +3797,7 @@ walrcv_receive_fn
 walrcv_send_fn
 walrcv_server_version_fn
 walrcv_startstreaming_fn
+walrcv_slot_snapshot_fn
 wchar2mb_with_len_converter
 wchar_t
 win32_deadchild_waitinfo
-- 
2.25.1

#41Melih Mutlu
m.melihmutlu@gmail.com
In reply to: shveta malik (#39)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Shveta,

shveta malik <shveta.malik@gmail.com>, 1 Şub 2023 Çar, 15:01 tarihinde şunu
yazdı:

On Wed, Feb 1, 2023 at 5:05 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:
2) I found a crash in the previous patch (v9), but have not tested it
on the latest yet. Crash happens when all the replication slots are
consumed and we are trying to create new. I tweaked the settings like
below so that it can be reproduced easily:
max_sync_workers_per_subscription=3
max_replication_slots = 2
and then ran the test case shared by you. I think there is some memory
corruption happening. (I did test in debug mode, have not tried in
release mode). I tried to put some traces to identify the root-cause.
I observed that worker_1 keeps on moving from 1 table to another table
correctly, but at some point, it gets corrupted i.e. origin-name
obtained for it is wrong and it tries to advance that and since that
origin does not exist, it asserts and then something else crashes.
From log: (new trace lines added by me are prefixed by shveta, also
tweaked code to have my comment 1 fixed to have clarity on worker-id).

form below traces, it is clear that worker_1 was moving from one
relation to another, always getting correct origin 'pg_16688_1', but
at the end it got 'pg_16688_49' which does not exist. Second part of
trace shows who updated 'pg_16688_49', it was done by worker_49 which
even did not get chance to create this origin due to max_rep_slot
reached.

Thanks for investigating this error. I think it's the same error as the one
Shi reported earlier. [1]/messages/by-id/OSZPR01MB631013C833C98E826B3CFCB9FDC69@OSZPR01MB6310.jpnprd01.prod.outlook.com
I couldn't reproduce it yet but will apply your tweaks and try again.
Looking into this.

[1]: /messages/by-id/OSZPR01MB631013C833C98E826B3CFCB9FDC69@OSZPR01MB6310.jpnprd01.prod.outlook.com
/messages/by-id/OSZPR01MB631013C833C98E826B3CFCB9FDC69@OSZPR01MB6310.jpnprd01.prod.outlook.com

Thanks,
--
Melih Mutlu
Microsoft

#42shveta malik
shveta.malik@gmail.com
In reply to: Melih Mutlu (#41)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Wed, Feb 1, 2023 at 5:42 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Thanks for investigating this error. I think it's the same error as the one Shi reported earlier. [1]
I couldn't reproduce it yet but will apply your tweaks and try again.
Looking into this.

[1] /messages/by-id/OSZPR01MB631013C833C98E826B3CFCB9FDC69@OSZPR01MB6310.jpnprd01.prod.outlook.com

I tried Shi-san's testcase earlier but I too could not reproduce it,
so I assumed that it is fixed in one of your patches already and thus
thought that the current issue is a new one.

thanks
Shveta

#43shveta malik
shveta.malik@gmail.com
In reply to: Melih Mutlu (#41)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Wed, Feb 1, 2023 at 5:42 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi Shveta,

shveta malik <shveta.malik@gmail.com>, 1 Şub 2023 Çar, 15:01 tarihinde şunu yazdı:

On Wed, Feb 1, 2023 at 5:05 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:
2) I found a crash in the previous patch (v9), but have not tested it
on the latest yet. Crash happens when all the replication slots are
consumed and we are trying to create new. I tweaked the settings like
below so that it can be reproduced easily:
max_sync_workers_per_subscription=3
max_replication_slots = 2
and then ran the test case shared by you. I think there is some memory
corruption happening. (I did test in debug mode, have not tried in
release mode). I tried to put some traces to identify the root-cause.
I observed that worker_1 keeps on moving from 1 table to another table
correctly, but at some point, it gets corrupted i.e. origin-name
obtained for it is wrong and it tries to advance that and since that
origin does not exist, it asserts and then something else crashes.
From log: (new trace lines added by me are prefixed by shveta, also
tweaked code to have my comment 1 fixed to have clarity on worker-id).

form below traces, it is clear that worker_1 was moving from one
relation to another, always getting correct origin 'pg_16688_1', but
at the end it got 'pg_16688_49' which does not exist. Second part of
trace shows who updated 'pg_16688_49', it was done by worker_49 which
even did not get chance to create this origin due to max_rep_slot
reached.

Thanks for investigating this error. I think it's the same error as the one Shi reported earlier. [1]
I couldn't reproduce it yet but will apply your tweaks and try again.
Looking into this.

[1] /messages/by-id/OSZPR01MB631013C833C98E826B3CFCB9FDC69@OSZPR01MB6310.jpnprd01.prod.outlook.com

Hi Melih,
I think I am able to identify the root cause. It is not memory
corruption, but the way origin-names are stored in system-catalog
mapped to a particular relation-id before even those are created.

After adding few more logs:

[4227]: ERROR: could not create replication slot "pg_16684_sync_49_7195149685251088378": ERROR: all replication slots are in use HINT: Free one or increase max_replication_slots.
originname :pg_16684_49, relid:16540
[4227]: ERROR: could not create replication slot "pg_16684_sync_49_7195149685251088378": ERROR: all replication slots are in use HINT: Free one or increase max_replication_slots.
updated-origin in system catalog:pg_16684_49,
slot:pg_16684_sync_49_7195149685251088378, relid:16540
[4227]: ERROR: could not create replication slot "pg_16684_sync_49_7195149685251088378": ERROR: all replication slots are in use HINT: Free one or increase max_replication_slots.
get-origin-id2 originid:0, originname:pg_16684_49
[4227]: ERROR: could not create replication slot "pg_16684_sync_49_7195149685251088378": ERROR: all replication slots are in use HINT: Free one or increase max_replication_slots.
"pg_16684_sync_49_7195149685251088378": ERROR: all replication slots
are in use
HINT: Free one or increase max_replication_slots.

[4428]: LOG: shveta- LogicalRepSyncTableStart- worker_148 updated-origin:pg_16684_49, slot:pg_16684_sync_148_7195149685251088378, relid:16540
originname :pg_16684_49, relid:16540
[4428]: LOG: shveta- LogicalRepSyncTableStart- worker_148 updated-origin:pg_16684_49, slot:pg_16684_sync_148_7195149685251088378, relid:16540
"pg_16684_sync_49_7195149685251088378" on publisher: ERROR:
replication slot "pg_16684_sync_49_7195149 685251088378" does not
exist
[4428]: LOG: shveta- LogicalRepSyncTableStart- worker_148 updated-origin:pg_16684_49, slot:pg_16684_sync_148_7195149685251088378, relid:16540
originname:pg_16684_49
[4428]: LOG: shveta- LogicalRepSyncTableStart- worker_148 updated-origin:pg_16684_49, slot:pg_16684_sync_148_7195149685251088378, relid:16540
updated-origin:pg_16684_49,
slot:pg_16684_sync_148_7195149685251088378, relid:16540

So from above, worker_49 came and picked up relid:16540, constructed
origin-name and slot-name and updated in system-catalog and then it
tried to actually create that slot and origin but since max-slot count
was reached, it failed and exited, but did not do cleanup from system
catalog for that relid.

Then worker_148 came and also picked up table 16540 since it was not
completed/started by previous worker, but this time it found that
origin and slot entry present in system-catalog against this relid, so
it picked the same names and started processing, but since those do
not exist, it asserted while advancing the origin.

The assert is only reproduced when an already running worker (say
worker_1) who has 'created=true' set, gets to sync the relid for which
a previously failed worker has tried and updated origin-name w/o
creating it. In such a case worker_1 (with created=true) will try to
reuse the origin and thus will try to advance it and will end up
asserting. That is why you might not see the assertion always. The
condition 'created' is set to true for that worker and it goes to
reuse the origin updated by the previous worker.

So to fix this, I think either we update origin and slot entries in
the system catalog after the creation has passed or we clean-up the
system catalog in case of failure. What do you think?

thanks
Shveta

#44shveta malik
shveta.malik@gmail.com
In reply to: shveta malik (#43)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Thu, Feb 2, 2023 at 9:18 AM shveta malik <shveta.malik@gmail.com> wrote:

Hi Melih,
I think I am able to identify the root cause. It is not memory
corruption, but the way origin-names are stored in system-catalog
mapped to a particular relation-id before even those are created.

Apart from the problem mentioned in my earlier email, I think there is
one more issue here as seen by the same assert causing testcase. The
'lastusedid' stored in system-catalog kept on increasing w/o even slot
and origin getting created. 2 workers worked well with
max_replication_slots=2 and then since all slots were consumed 3rd one
could not create any slot and exited but it increased lastusedid. Then
another worker came, incremented lastusedId in system-catalog and
failed to create slot and exited and so on. This makes lastUsedId
incremented continuously until you kill the subscriber or free any
slot used previously. If you keep subscriber running long enough, it
will make lastUsedId go beyond its limit.
Shouldn't lastUsedId be incremented only after making sure that worker
has created a slot and origin corresponding to that particular
rep_slot_id (derived using lastUsedId). Please let me know if my
understanding is not correct.

thanks
Shveta

#45shveta malik
shveta.malik@gmail.com
In reply to: Melih Mutlu (#40)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Wed, Feb 1, 2023 at 5:37 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi,
Please see attached patches.

Thanks,
--
Melih Mutlu
Microsoft

Hi Melih,

Few suggestions on v10-0002-Reuse patch

1)
for (int64 i = 1; i <= lastusedid; i++)
{
char originname_to_drop[NAMEDATALEN] = {0};
snprintf(originname_to_drop,
sizeof(originname_to_drop), "pg_%u_%lld", subid, (long long) i);
.......
}

--Is it better to use the function
'ReplicationOriginNameForLogicalRep' here instead of sprintf, just to
be consistent everywhere to construct origin-name?

2)
pa_launch_parallel_worker:
launched = logicalrep_worker_launch(MyLogicalRepWorker->dbid,
MySubscription->oid,

MySubscription->name,

MyLogicalRepWorker->userid,
InvalidOid,

dsm_segment_handle(winfo->dsm_seg),
0);

--Can we please define 'InvalidRepSlotId' macro and pass it here as
the last arg to make it more readable.
#define InvalidRepSlotId 0
Same in ApplyLauncherMain where we are passing 0 as last arg to
logicalrep_worker_launch.

3)
We are safe to drop the replication trackin origin after this
--typo: trackin -->tracking

4)
process_syncing_tables_for_sync:
if (MyLogicalRepWorker->slot_name && strcmp(syncslotname,
MyLogicalRepWorker->slot_name) != 0)
{
..............
ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,

MyLogicalRepWorker->relid,
originname,

sizeof(originname));

/* Drop replication origin */
replorigin_drop_by_name(originname, true, false);
}

--Are we passing missing_ok as true (second arg) intentionally here in
replorigin_drop_by_name? Once we fix the issue reported in my earlier
email (ASSERT), do you think it makes sense to pass missing_ok as
false here?

5)
process_syncing_tables_for_sync:
foreach(lc, rstates)
{

rstate = (SubscriptionRelState *)
palloc(sizeof(SubscriptionRelState));
memcpy(rstate, lfirst(lc),
sizeof(SubscriptionRelState));
/*
* Pick the table for the next run if it is
not already picked up
* by another worker.
*/
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
if (rstate->state != SUBREL_STATE_SYNCDONE &&

!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))

{
.........
}
LWLockRelease(LogicalRepWorkerLock);
}

--Do we need to palloc for each relation separately? Shall we do it
once outside the loop and reuse it? Also pfree is not done for rstate
here.

6)
LogicalRepSyncTableStart:
1385 slotname = (char *) palloc(NAMEDATALEN);
1413 prev_slotname = (char *) palloc(NAMEDATALEN);
1481 slotname = prev_slotname;
1502 pfree(prev_slotname);
1512 UpdateSubscriptionRel(MyLogicalRepWorker->subid,
1513
MyLogicalRepWorker->relid,
1514
MyLogicalRepWorker->relstate,
1515
MyLogicalRepWorker->relstate_lsn,
1516 slotname,
1517 originname);

Can you please review the above flow (I have given line# along with),
I think it could be problematic. We alloced prev_slotname, assigned it
to slotname, freed prev_slotname and used slotname after freeing the
prev_slotname.
Also slotname is allocated some memory too, that is not freed.

Reviewing further....

JFYI, I get below while applying patch:

========================
shveta@shveta-vm:~/repos/postgres1/postgres$ git am
~/Desktop/shared/reuse/v10-0002-Reuse-Logical-Replication-Background-worker.patch
Applying: Reuse Logical Replication Background worker
.git/rebase-apply/patch:142: trailing whitespace.
values[Anum_pg_subscription_rel_srrelslotname - 1] =
.git/rebase-apply/patch:692: indent with spaces.
errmsg("could not receive list of slots associated
with the subscription %u, error: %s",
.git/rebase-apply/patch:1055: trailing whitespace.
/*
.git/rebase-apply/patch:1057: trailing whitespace.
* relations.
.git/rebase-apply/patch:1059: trailing whitespace.
* and origin. Then stop the worker gracefully.
warning: 5 lines add whitespace errors.
========================

thanks
Shveta

#46shveta malik
shveta.malik@gmail.com
In reply to: shveta malik (#45)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Thu, Feb 2, 2023 at 5:01 PM shveta malik <shveta.malik@gmail.com> wrote:

Reviewing further....

Few more comments for v10-0002 and v7-0001:

1)
+ * need_full_snapshot
+ * if true, create a snapshot able to read all tables,
+ * otherwise do not create any snapshot.
+ *
CreateDecodingContext(..,CreateDecodingContext,..)

--Is the comment correct? Shall we have same comment here as that of
'CreateDecodingContext'
* need_full_snapshot -- if true, must obtain a snapshot able to read all
* tables; if false, one that can read only catalogs is acceptable.
This function is not going to create a snapshot anyways. It is just a
pre-step and then the caller needs to call 'SnapBuild' functions to
build a snapshot. Here need_full_snapshot decides whether we need all
tables or only catalog tables changes only and thus the comment change
is needed.

==========

2)

Can we please add more logging:

2a)
when lastusedId is incremented and updated in pg_* table
ereport(DEBUG2,
(errmsg("[subid:%d] Incremented lastusedid
to:%ld",MySubscription->oid, MySubscription->lastusedid)));

Comments for LogicalRepSyncTableStart():

2b ) After every UpdateSubscriptionRel:

ereport(DEBUG2,
(errmsg("[subid:%d] LogicalRepSyncWorker_%ld updated origin to %s and
slot to %s for relid %d",
MyLogicalRepWorker->subid, MyLogicalRepWorker->rep_slot_id,
originname, slotname, MyLogicalRepWorker->relid)));

2c )
After walrcv_create_slot:

ereport(DEBUG2,
(errmsg("[subid:%d] LogicalRepSyncWorker_%ld created slot %s",
MyLogicalRepWorker->subid, MyLogicalRepWorker->rep_slot_id, slotname)));

2d)
After replorigin_create:

ereport(DEBUG2,
(errmsg("[subid:%d] LogicalRepSyncWorker_%ld created origin %s [id: %d]",
MyLogicalRepWorker->subid, MyLogicalRepWorker->rep_slot_id,
originname, originid)));

2e)
When it goes to reuse flow (i.e. before walrcv_slot_snapshot), if
needed we can dump newly obtained origin_startpos also:

ereport(DEBUG2,
(errmsg("[subid:%d] LogicalRepSyncWorker_%ld reusing slot %s and origin %s",
MyLogicalRepWorker->subid, MyLogicalRepWorker->rep_slot_id, slotname,
originname)));

2f)
Also in existing comment:

+ (errmsg("logical replication table synchronization worker for
subscription \"%s\" has moved to sync table \"%s\".",
+ MySubscription->name, get_rel_name(MyLogicalRepWorker->relid))));

we can add relid also along with relname. relid is the one stored in
pg_subscription_rel and thus it becomes easy to map. Also we can
change 'logical replication table synchronization worker' to
'LogicalRepSyncWorker_%ld'.
Same change needed in other similar log lines where we say that worker
started and finished.

Please feel free to change the above log lines as you find
appropriate. I have given just a sample sort of thing.

thanks
Shveta

#47shveta malik
shveta.malik@gmail.com
In reply to: shveta malik (#46)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Fri, Feb 3, 2023 at 11:50 AM shveta malik <shveta.malik@gmail.com> wrote:

On Thu, Feb 2, 2023 at 5:01 PM shveta malik <shveta.malik@gmail.com> wrote:

2e)
When it goes to reuse flow (i.e. before walrcv_slot_snapshot), if
needed we can dump newly obtained origin_startpos also:

ereport(DEBUG2,
(errmsg("[subid:%d] LogicalRepSyncWorker_%ld reusing slot %s and origin %s",
MyLogicalRepWorker->subid, MyLogicalRepWorker->rep_slot_id, slotname,
originname)));

One addition, I think it will be good to add relid as well in above so
that we can get info as in we are reusing old slot for which relid.
Once we have all the above in log-file, it makes it very easy to
diagnose reuse-sync worker related problems just by looking at the
logfile.

thanks
Shveta

#48shveta malik
shveta.malik@gmail.com
In reply to: shveta malik (#45)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Thu, Feb 2, 2023 at 5:01 PM shveta malik <shveta.malik@gmail.com> wrote:

Reviewing further....

Hi Melih,

int64 rep_slot_id;
int64 lastusedid;
int64 sublastusedid

--Should all of the above be unsigned integers?

thanks
Shveta

#49shiy.fnst@fujitsu.com
shiy.fnst@fujitsu.com
In reply to: shveta malik (#43)
RE: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Thu, Feb 2, 2023 11:48 AM shveta malik <shveta.malik@gmail.com> wrote:

On Wed, Feb 1, 2023 at 5:42 PM Melih Mutlu <m.melihmutlu@gmail.com>
wrote:

Hi Shveta,

shveta malik <shveta.malik@gmail.com>, 1 Şub 2023 Çar, 15:01 tarihinde

şunu yazdı:

On Wed, Feb 1, 2023 at 5:05 PM Melih Mutlu <m.melihmutlu@gmail.com>

wrote:

2) I found a crash in the previous patch (v9), but have not tested it
on the latest yet. Crash happens when all the replication slots are
consumed and we are trying to create new. I tweaked the settings like
below so that it can be reproduced easily:
max_sync_workers_per_subscription=3
max_replication_slots = 2
and then ran the test case shared by you. I think there is some memory
corruption happening. (I did test in debug mode, have not tried in
release mode). I tried to put some traces to identify the root-cause.
I observed that worker_1 keeps on moving from 1 table to another table
correctly, but at some point, it gets corrupted i.e. origin-name
obtained for it is wrong and it tries to advance that and since that
origin does not exist, it asserts and then something else crashes.
From log: (new trace lines added by me are prefixed by shveta, also
tweaked code to have my comment 1 fixed to have clarity on worker-id).

form below traces, it is clear that worker_1 was moving from one
relation to another, always getting correct origin 'pg_16688_1', but
at the end it got 'pg_16688_49' which does not exist. Second part of
trace shows who updated 'pg_16688_49', it was done by worker_49

which

even did not get chance to create this origin due to max_rep_slot
reached.

Thanks for investigating this error. I think it's the same error as the one Shi

reported earlier. [1]

I couldn't reproduce it yet but will apply your tweaks and try again.
Looking into this.

[1] https://www.postgresql.org/message-

id/OSZPR01MB631013C833C98E826B3CFCB9FDC69%40OSZPR01MB6310.jpn
prd01.prod.outlook.com

Hi Melih,
I think I am able to identify the root cause. It is not memory
corruption, but the way origin-names are stored in system-catalog
mapped to a particular relation-id before even those are created.

After adding few more logs:

[4227] LOG: shveta- LogicalRepSyncTableStart- worker_49 constructed
originname :pg_16684_49, relid:16540
[4227] LOG: shveta- LogicalRepSyncTableStart- worker_49
updated-origin in system catalog:pg_16684_49,
slot:pg_16684_sync_49_7195149685251088378, relid:16540
[4227] LOG: shveta- LogicalRepSyncTableStart- worker_49
get-origin-id2 originid:0, originname:pg_16684_49
[4227] ERROR: could not create replication slot
"pg_16684_sync_49_7195149685251088378": ERROR: all replication slots
are in use
HINT: Free one or increase max_replication_slots.

[4428] LOG: shveta- LogicalRepSyncTableStart- worker_148 constructed
originname :pg_16684_49, relid:16540
[4428] LOG: could not drop replication slot
"pg_16684_sync_49_7195149685251088378" on publisher: ERROR:
replication slot "pg_16684_sync_49_7195149 685251088378" does not
exist
[4428] LOG: shveta- LogicalRepSyncTableStart- worker_148 drop-origin
originname:pg_16684_49
[4428] LOG: shveta- LogicalRepSyncTableStart- worker_148
updated-origin:pg_16684_49,
slot:pg_16684_sync_148_7195149685251088378, relid:16540

So from above, worker_49 came and picked up relid:16540, constructed
origin-name and slot-name and updated in system-catalog and then it
tried to actually create that slot and origin but since max-slot count
was reached, it failed and exited, but did not do cleanup from system
catalog for that relid.

Then worker_148 came and also picked up table 16540 since it was not
completed/started by previous worker, but this time it found that
origin and slot entry present in system-catalog against this relid, so
it picked the same names and started processing, but since those do
not exist, it asserted while advancing the origin.

The assert is only reproduced when an already running worker (say
worker_1) who has 'created=true' set, gets to sync the relid for which
a previously failed worker has tried and updated origin-name w/o
creating it. In such a case worker_1 (with created=true) will try to
reuse the origin and thus will try to advance it and will end up
asserting. That is why you might not see the assertion always. The
condition 'created' is set to true for that worker and it goes to
reuse the origin updated by the previous worker.

So to fix this, I think either we update origin and slot entries in
the system catalog after the creation has passed or we clean-up the
system catalog in case of failure. What do you think?

I think the first way seems better.

I reproduced the problem I reported before with latest patch (v7-0001,
v10-0002), and looked into this problem. It is caused by a similar reason. Here
is some analysis for the problem I reported [1]/messages/by-id/OSZPR01MB631013C833C98E826B3CFCB9FDC69@OSZPR01MB6310.jpnprd01.prod.outlook.com.#6.

First, a tablesync worker (worker-1) started for "tbl1", its originname is
"pg_16398_1". And it exited because of unique constraint. In
LogicalRepSyncTableStart(), originname in pg_subscription_rel is updated when
updating table state to DATASYNC, and the origin is created when updating table
state to FINISHEDCOPY. So when it exited with state DATASYNC , the origin is not
created but the originname has been updated in pg_subscription_rel.

Then a tablesync worker (worker-2) started for "tbl2", its originname is
"pg_16398_2". After tablesync of "tbl2" finished, this worker moved to sync
table "tbl1". In LogicalRepSyncTableStart(), it got the originname of "tbl1" -
"pg_16398_1", by calling ReplicationOriginNameForLogicalRep(), and tried to drop
the origin (although it is not actually created before). After that, it called
replorigin_by_name to get the originid whose name is "pg_16398_1" and the result
is InvalidOid. Origin won't be created in this case because the sync worker has
created a replication slot (when it synced tbl2), so the originid was still
invalid and it caused an assertion failure when calling replorigin_advance().

It seems we don't need to drop previous origin in worker-2 because the previous
origin was not created in worker-1. I think one way to fix it is to not update
originname of pg_subscription_rel when setting state to DATASYNC, and only do
that when setting state to FINISHEDCOPY. If so, the originname in
pg_subscription_rel will be set at the same time the origin is created.
(Besides, the slotname seems need to be updated when setting state to DATASYNC,
because the previous slot might have been created successfully and we need to get
the previous slotname and drop that.)

[1]: /messages/by-id/OSZPR01MB631013C833C98E826B3CFCB9FDC69@OSZPR01MB6310.jpnprd01.prod.outlook.com

Regards,
Shi yu

#50wangw.fnst@fujitsu.com
wangw.fnst@fujitsu.com
In reply to: Melih Mutlu (#40)
RE: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Wed, Feb 1, 2023 20:07 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Thanks for pointing out this review. I somehow skipped that, sorry.

Please see attached patches.

Thanks for updating the patch set.
Here are some comments.

1. In the function ApplyWorkerMain.
+			/* This is main apply worker */
+			run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);

I think we need to keep the worker name as "leader apply worker" in the comment
like the current HEAD.

---

2. In the function LogicalRepApplyLoop.
+				 * can be reused, we need to take care of memory contexts here
+				 * before moving to sync a table.
+				 */
+				if (MyLogicalRepWorker->ready_to_reuse)
+				{
+					MemoryContextResetAndDeleteChildren(ApplyMessageContext);
+					MemoryContextSwitchTo(TopMemoryContext);
+					return;
+				}

I think in this case we also need to pop the error context stack before
returning. Otherwise, I think we might use the wrong callback
(apply error_callback) after we return from this function.

---

3. About the function UpdateSubscriptionRelReplicationSlot.
This newly introduced function UpdateSubscriptionRelReplicationSlot does not
seem to be invoked. Do we need this function?

Regards,
Wang Wei

#51shveta malik
shveta.malik@gmail.com
In reply to: shiy.fnst@fujitsu.com (#49)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Tue, Feb 7, 2023 at 8:18 AM shiy.fnst@fujitsu.com
<shiy.fnst@fujitsu.com> wrote:

On Thu, Feb 2, 2023 11:48 AM shveta malik <shveta.malik@gmail.com> wrote:

So to fix this, I think either we update origin and slot entries in
the system catalog after the creation has passed or we clean-up the
system catalog in case of failure. What do you think?

I think the first way seems better.

Yes, I agree.

I reproduced the problem I reported before with latest patch (v7-0001,
v10-0002), and looked into this problem. It is caused by a similar reason. Here
is some analysis for the problem I reported [1].#6.

First, a tablesync worker (worker-1) started for "tbl1", its originname is
"pg_16398_1". And it exited because of unique constraint. In
LogicalRepSyncTableStart(), originname in pg_subscription_rel is updated when
updating table state to DATASYNC, and the origin is created when updating table
state to FINISHEDCOPY. So when it exited with state DATASYNC , the origin is not
created but the originname has been updated in pg_subscription_rel.

Then a tablesync worker (worker-2) started for "tbl2", its originname is
"pg_16398_2". After tablesync of "tbl2" finished, this worker moved to sync
table "tbl1". In LogicalRepSyncTableStart(), it got the originname of "tbl1" -
"pg_16398_1", by calling ReplicationOriginNameForLogicalRep(), and tried to drop
the origin (although it is not actually created before). After that, it called
replorigin_by_name to get the originid whose name is "pg_16398_1" and the result
is InvalidOid. Origin won't be created in this case because the sync worker has
created a replication slot (when it synced tbl2), so the originid was still
invalid and it caused an assertion failure when calling replorigin_advance().

It seems we don't need to drop previous origin in worker-2 because the previous
origin was not created in worker-1. I think one way to fix it is to not update
originname of pg_subscription_rel when setting state to DATASYNC, and only do
that when setting state to FINISHEDCOPY. If so, the originname in
pg_subscription_rel will be set at the same time the origin is created.

+1. Update of system-catalog needs to be done carefully and only when
origin is created.

thanks
Shveta

#52wangw.fnst@fujitsu.com
wangw.fnst@fujitsu.com
In reply to: wangw.fnst@fujitsu.com (#50)
RE: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Thur, Feb 7, 2023 15:29 PM I wrote:

On Wed, Feb 1, 2023 20:07 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Thanks for pointing out this review. I somehow skipped that, sorry.

Please see attached patches.

Thanks for updating the patch set.
Here are some comments.

Hi, here are some more comments on patch v7-0001*:

1. The new comments atop the function CreateDecodingContext
+ * need_full_snapshot
+ * 		if true, create a snapshot able to read all tables,
+ * 		otherwise do not create any snapshot.

I think if 'need_full_snapshot' is false, it means we will create a snapshot
that can read only catalogs. (see SnapBuild->building_full_snapshot)

===

2. This are two questions I'm not sure about.
2a.
Because pg-doc has the following description in [1]https://www.postgresql.org/docs/devel/protocol-replication.html: (option "SNAPSHOT 'use'")
```
'use' will use the snapshot for the current transaction executing the command.
This option must be used in a transaction, and CREATE_REPLICATION_SLOT must be
the first command run in that transaction.
```
So I think in the function CreateDecodingContext, when "need_full_snapshot" is
true, we seem to need the following check, just like in the function
CreateInitDecodingContext:
```
if (IsTransactionState() &&
GetTopTransactionIdIfAny() != InvalidTransactionId)
ereport(ERROR,
(errcode(ERRCODE_ACTIVE_SQL_TRANSACTION),
errmsg("cannot create logical replication slot in transaction that has performed writes")));
```

2b.
It seems that we also need to invoke the function
CheckLogicalDecodingRequirements in the new function CreateReplicationSnapshot,
just like the function CreateReplicationSlot and the function
StartLogicalReplication.

Is there any reason not to do these two checks? Please let me know if I missed
something.

===

3. The invocation of startup_cb_wrapper in the function CreateDecodingContext.
I think we should change the third input parameter to true when invoke function
startup_cb_wrapper for CREATE_REPLICATION_SNAPSHOT. BTW, after applying patch
v10-0002*, these settings will be inconsistent when sync workers use
"CREATE_REPLICATION_SLOT" and "CREATE_REPLICATION_SNAPSHOT" to take snapshots.
This input parameter (true) will let us disable streaming and two-phase
transactions in function pgoutput_startup. See the last paragraph of the commit
message for 4648243 for more details.

[1]: https://www.postgresql.org/docs/devel/protocol-replication.html

Regards,
Wang Wei

#53Melih Mutlu
m.melihmutlu@gmail.com
In reply to: shveta malik (#51)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Shveta and Shi,

Thanks for your investigations.

shveta malik <shveta.malik@gmail.com>, 8 Şub 2023 Çar, 16:49 tarihinde şunu
yazdı:

On Tue, Feb 7, 2023 at 8:18 AM shiy.fnst@fujitsu.com
<shiy.fnst@fujitsu.com> wrote:

I reproduced the problem I reported before with latest patch (v7-0001,
v10-0002), and looked into this problem. It is caused by a similar

reason. Here

is some analysis for the problem I reported [1].#6.

First, a tablesync worker (worker-1) started for "tbl1", its originname

is

"pg_16398_1". And it exited because of unique constraint. In
LogicalRepSyncTableStart(), originname in pg_subscription_rel is updated

when

updating table state to DATASYNC, and the origin is created when

updating table

state to FINISHEDCOPY. So when it exited with state DATASYNC , the

origin is not

created but the originname has been updated in pg_subscription_rel.

Then a tablesync worker (worker-2) started for "tbl2", its originname is
"pg_16398_2". After tablesync of "tbl2" finished, this worker moved to

sync

table "tbl1". In LogicalRepSyncTableStart(), it got the originname of

"tbl1" -

"pg_16398_1", by calling ReplicationOriginNameForLogicalRep(), and tried

to drop

the origin (although it is not actually created before). After that, it

called

replorigin_by_name to get the originid whose name is "pg_16398_1" and

the result

is InvalidOid. Origin won't be created in this case because the sync

worker has

created a replication slot (when it synced tbl2), so the originid was

still

invalid and it caused an assertion failure when calling

replorigin_advance().

It seems we don't need to drop previous origin in worker-2 because the

previous

origin was not created in worker-1. I think one way to fix it is to not

update

originname of pg_subscription_rel when setting state to DATASYNC, and

only do

that when setting state to FINISHEDCOPY. If so, the originname in
pg_subscription_rel will be set at the same time the origin is created.

+1. Update of system-catalog needs to be done carefully and only when
origin is created.

I see that setting originname in the catalog before actually creating it
causes issues. My concern with setting originname when setting the state to
FINISHEDCOPY is that if worker waits until FINISHEDCOPY to update
slot/origin name but it fails somewhere before reaching FINISHEDCOPY and
after creating slot/origin, then this new created slot/origin will be left
behind. It wouldn't be possible to find and drop them since their names are
not stored in the catalog. Eventually, this might also cause hitting
the max_replication_slots limit in case of such failures between origin
creation and FINISHEDCOPY.

One fix I can think is to update the catalog right after creating a new
origin. But this would also require commiting the current transaction to
actually persist the originname. I guess this action of commiting the
transaction in the middle of initial sync could hurt the copy process.

What do you think?

Also; working on an updated patch to address your other comments. Thanks
again.

Best,
--
Melih Mutlu
Microsoft

#54Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Melih Mutlu (#53)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi,

Melih Mutlu <m.melihmutlu@gmail.com>, 16 Şub 2023 Per, 14:37 tarihinde şunu
yazdı:

I see that setting originname in the catalog before actually creating it
causes issues. My concern with setting originname when setting the state to
FINISHEDCOPY is that if worker waits until FINISHEDCOPY to update
slot/origin name but it fails somewhere before reaching FINISHEDCOPY and
after creating slot/origin, then this new created slot/origin will be left
behind. It wouldn't be possible to find and drop them since their names are
not stored in the catalog. Eventually, this might also cause hitting
the max_replication_slots limit in case of such failures between origin
creation and FINISHEDCOPY.

One fix I can think is to update the catalog right after creating a new
origin. But this would also require commiting the current transaction to
actually persist the originname. I guess this action of commiting the
transaction in the middle of initial sync could hurt the copy process.

Here are more thoughts on this:
I still believe that updating originname when setting state
to FINISHEDCOPY is not a good idea since any failure
before FINISHEDCOPY prevent us to store originname in the catalog. If an
origin or slot is not in the catalog, it's not easily possible to find and
drop origins/slot that are left behind. And we definitely do not want to
keep unnecessary origins/slots since we would hit max_replication_slots
limit.
It's better to be safe and update origin/slot names when setting state
to DATASYNC. At this point, the worker must be sure that it writes correct
origin/slot names into the catalog.
Following part actually cleans up the catalog if a table is left behind in
DATASYNC state and its slot and origin cannot be used for sync.

ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);

StartTransactionCommand();
/* Replication origin might still exist. Try to drop */
replorigin_drop_by_name(originname, true, false);

/*
* Remove replication slot and origin name from the relation's
* catalog record
*/
UpdateSubscriptionRel(MyLogicalRepWorker->subid,
MyLogicalRepWorker->relid,
MyLogicalRepWorker->relstate,
MyLogicalRepWorker->relstate_lsn,
NULL,
NULL);

The patch needs to refresh the origin name before it begins copying the
table. It will try to read from the catalog but won't find any slot/origin
since they are cleaned. Then, it will move on with the correct origin name
which is the one created/will be created for the current sync worker.

I tested refetching originname and seems like it fixes the errors you
reported.

Thanks,
--
Melih Mutlu
Microsoft

#55Melih Mutlu
m.melihmutlu@gmail.com
In reply to: shveta malik (#45)
2 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Shveta,

Thanks for reviewing.
Please see attached patches.

shveta malik <shveta.malik@gmail.com>, 2 Şub 2023 Per, 14:31 tarihinde şunu
yazdı:

On Wed, Feb 1, 2023 at 5:37 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:
for (int64 i = 1; i <= lastusedid; i++)
{
char originname_to_drop[NAMEDATALEN] = {0};
snprintf(originname_to_drop,
sizeof(originname_to_drop), "pg_%u_%lld", subid, (long long) i);
.......
}

--Is it better to use the function
'ReplicationOriginNameForLogicalRep' here instead of sprintf, just to
be consistent everywhere to construct origin-name?

ReplicationOriginNameForLogicalRep creates a slot name with current
"lastusedid" and doesn't accept that id as parameter. Here the patch needs
to check all possible id's.

/* Drop replication origin */
replorigin_drop_by_name(originname, true, false);
}

--Are we passing missing_ok as true (second arg) intentionally here in
replorigin_drop_by_name? Once we fix the issue reported in my earlier
email (ASSERT), do you think it makes sense to pass missing_ok as
false here?

Yes, missing_ok is intentional. The user might be concurrently refreshing
the sub or the apply worker might already drop the origin at that point.
So, missing_ok is set to true.
This is also how origin drops before the worker exits are handled on HEAD
too. I only followed the same approach.

--Do we need to palloc for each relation separately? Shall we do it
once outside the loop and reuse it? Also pfree is not done for rstate
here.

Removed palloc from the loop. No need to pfree here since the memory
context will be deleted with the next CommitTransactionCommand call.

Can you please review the above flow (I have given line# along with),
I think it could be problematic. We alloced prev_slotname, assigned it
to slotname, freed prev_slotname and used slotname after freeing the
prev_slotname.
Also slotname is allocated some memory too, that is not freed.

Right, I used memcpy instead of assigning prev_slotname to slotname.
slotname is returned in the end and pfree'd later [1]https://github.com/postgres/postgres/blob/master/src/backend/replication/logical/worker.c#L4359

I also addressed your other reviews that I didn't explicitly mention in
this email. I simply applied the changes you pointed out. Also added some
more logs as well. I hope it's more useful now.

[1]: https://github.com/postgres/postgres/blob/master/src/backend/replication/logical/worker.c#L4359
https://github.com/postgres/postgres/blob/master/src/backend/replication/logical/worker.c#L4359

Thanks,
--
Melih Mutlu
Microsoft

Attachments:

v8-0001-Add-replication-protocol-cmd-to-create-a-snapsho.patchapplication/octet-stream; name=v8-0001-Add-replication-protocol-cmd-to-create-a-snapsho.patchDownload
From 1cb2c7e2e3efa1abd10f90e38b27220af8bcabf2 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 13 Oct 2022 17:05:45 +0300
Subject: [PATCH v8 1/2] Add replication protocol cmd to create a snapshot

Introduced CREATE_REPLICATION_SNAPSHOT to be able to create and use a
snapshot without creating a new replication slot, but by using an
existing slot.

CREATE_REPLICATION_SNAPSHOT simply does what CREATE_REPLICATION_SLOT does
without creating a new replication slot.

CREATE_REPLICATION_SNAPSHOT command imports the snapshot into the current
transaction and returns consistent_point. The changes earlier than the
consistent_point will be applied by importing the snapshot. All changes
later than the consistent_point will be available to be consumed from
the replication slot.

This is useful for reusing replication slots in logical replication.
Otherwise, tablesync workers cannot start from a consistent point to copy a relation and then apply changes by consuming from replication slot.
---
 doc/src/sgml/protocol.sgml                    |  31 ++++++
 .../libpqwalreceiver/libpqwalreceiver.c       |  69 +++++++++++-
 src/backend/replication/logical/logical.c     |  40 ++++++-
 .../replication/logical/logicalfuncs.c        |   1 +
 src/backend/replication/repl_gram.y           |  18 ++-
 src/backend/replication/repl_scanner.l        |   2 +
 src/backend/replication/slotfuncs.c           |   1 +
 src/backend/replication/walsender.c           | 104 +++++++++++++++++-
 src/include/nodes/replnodes.h                 |  11 ++
 src/include/replication/logical.h             |   1 +
 src/include/replication/walreceiver.h         |  13 +++
 src/tools/pgindent/typedefs.list              |   2 +
 12 files changed, 289 insertions(+), 4 deletions(-)

diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 93fc7167d4..948a5aabd4 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2613,6 +2613,37 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
      </listitem>
     </varlistentry>
 
+    <varlistentry id="protocol-replication-replication-slot-snapshot">
+     <term><literal>CREATE_REPLICATION_SNAPSHOT</literal> <replaceable class="parameter">slot_name</replaceable> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
+      <indexterm><primary>CREATE_REPLICATION_SNAPSHOT</primary></indexterm>
+     </term>
+     <listitem>
+      <para>
+       Creates a snapshot including all the changes from the replication slot until
+       the point at which the replication slot becomes consistent. Then the snapshot
+       is used in the current transaction. This command is currently only supported
+       for logical replication slots.
+      </para>
+
+      <para>
+       In response to this command, the server will return a one-row result set,
+       containing the following field:
+       <variablelist>
+        <varlistentry>
+         <term><literal>consistent_point</literal> (<type>text</type>)</term>
+         <listitem>
+          <para>
+           The WAL location at which the slot became consistent.  This is the
+           earliest location from which streaming can start on this replication
+           slot.
+          </para>
+         </listitem>
+        </varlistentry>
+       </variablelist>
+      </para>
+     </listitem>
+    </varlistentry>
+
     <varlistentry id="protocol-replication-base-backup" xreflabel="BASE_BACKUP">
      <term><literal>BASE_BACKUP</literal> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
       <indexterm><primary>BASE_BACKUP</primary></indexterm>
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index 560ec974fa..9173163a07 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -81,6 +81,8 @@ static WalRcvExecResult *libpqrcv_exec(WalReceiverConn *conn,
 									   const int nRetTypes,
 									   const Oid *retTypes);
 static void libpqrcv_disconnect(WalReceiverConn *conn);
+static void libpqrcv_slot_snapshot(WalReceiverConn *conn, char *slotname,
+								   const WalRcvStreamOptions *options, XLogRecPtr *lsn);
 
 static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_connect = libpqrcv_connect,
@@ -97,7 +99,8 @@ static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_create_slot = libpqrcv_create_slot,
 	.walrcv_get_backend_pid = libpqrcv_get_backend_pid,
 	.walrcv_exec = libpqrcv_exec,
-	.walrcv_disconnect = libpqrcv_disconnect
+	.walrcv_disconnect = libpqrcv_disconnect,
+	.walrcv_slot_snapshot = libpqrcv_slot_snapshot
 };
 
 /* Prototypes for private functions */
@@ -933,6 +936,70 @@ libpqrcv_create_slot(WalReceiverConn *conn, const char *slotname,
 	return snapshot;
 }
 
+/*
+ * TODO
+ */
+static void
+libpqrcv_slot_snapshot(WalReceiverConn *conn,
+					   char *slotname,
+					   const WalRcvStreamOptions *options,
+					   XLogRecPtr *lsn)
+{
+	StringInfoData cmd;
+	PGresult   *res;
+	char	   *pubnames_str;
+	List	   *pubnames;
+	char	   *pubnames_literal;
+
+	initStringInfo(&cmd);
+
+	/* Build the command. */
+	appendStringInfo(&cmd, "CREATE_REPLICATION_SNAPSHOT \"%s\"", slotname);
+	appendStringInfoString(&cmd, " (");
+	appendStringInfo(&cmd, " proto_version '%u'",
+					 options->proto.logical.proto_version);
+
+	/* Add publication names. */
+	pubnames = options->proto.logical.publication_names;
+	pubnames_str = stringlist_to_identifierstr(conn->streamConn, pubnames);
+	if (!pubnames_str)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	pubnames_literal = PQescapeLiteral(conn->streamConn, pubnames_str,
+									   strlen(pubnames_str));
+	if (!pubnames_literal)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	appendStringInfo(&cmd, ", publication_names %s", pubnames_literal);
+	PQfreemem(pubnames_literal);
+	pfree(pubnames_str);
+
+	appendStringInfoString(&cmd, " )");
+
+	/* Execute the command. */
+	res = libpqrcv_PQexec(conn->streamConn, cmd.data);
+	pfree(cmd.data);
+
+	if (PQresultStatus(res) != PGRES_TUPLES_OK)
+	{
+		PQclear(res);
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg("Could not create a snapshot by replication slot \"%s\": %s",
+						slotname, pchomp(PQerrorMessage(conn->streamConn)))));
+	}
+
+	if (lsn)
+		*lsn = DatumGetLSN(DirectFunctionCall1Coll(pg_lsn_in, InvalidOid,
+												   CStringGetDatum(PQgetvalue(res, 0, 0))));
+
+	PQclear(res);
+}
+
 /*
  * Return PID of remote backend process.
  */
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index c3ec97a0a6..93fc0e5143 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -472,6 +472,10 @@ CreateInitDecodingContext(const char *plugin,
  * fast_forward
  *		bypass the generation of logical changes.
  *
+ * need_full_snapshot
+ * 		if true, must obtain a snapshot able to read all tables;
+ *  	if false, one that can read only catalogs is acceptable.
+ *
  * xl_routine
  *		XLogReaderRoutine used by underlying xlogreader
  *
@@ -490,6 +494,7 @@ LogicalDecodingContext *
 CreateDecodingContext(XLogRecPtr start_lsn,
 					  List *output_plugin_options,
 					  bool fast_forward,
+					  bool need_full_snapshot,
 					  XLogReaderRoutine *xl_routine,
 					  LogicalOutputPluginWriterPrepareWrite prepare_write,
 					  LogicalOutputPluginWriterWrite do_write,
@@ -498,6 +503,7 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 	LogicalDecodingContext *ctx;
 	ReplicationSlot *slot;
 	MemoryContext old_context;
+	TransactionId xmin_horizon = InvalidTransactionId;
 
 	/* shorter lines... */
 	slot = MyReplicationSlot;
@@ -544,8 +550,40 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 		start_lsn = slot->data.confirmed_flush;
 	}
 
+
+	/*
+	 * We need to determine a safe xmin horizon to start decoding from if we
+	 * want to create a snapshot too. Otherwise we would end up with a
+	 * snapshot that cannot be imported since xmin value from the snapshot may
+	 * be less than the oldest safe xmin. To avoid this call
+	 * GetOldestSafeDecodingTransactionId() to return a safe xmin value, which
+	 * can be used while exporting/importing the snapshot.
+	 *
+	 * So we have to acquire the ProcArrayLock to prevent computation of new
+	 * xmin horizons by other backends, get the safe decoding xid, and inform
+	 * the slot machinery about the new limit. Once that's done the
+	 * ProcArrayLock can be released as the slot machinery now is protecting
+	 * against vacuum.
+	 */
+	if (need_full_snapshot)
+	{
+		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+		xmin_horizon = GetOldestSafeDecodingTransactionId(!need_full_snapshot);
+
+		SpinLockAcquire(&slot->mutex);
+		slot->effective_catalog_xmin = xmin_horizon;
+		slot->data.catalog_xmin = xmin_horizon;
+		slot->effective_xmin = xmin_horizon;
+		SpinLockRelease(&slot->mutex);
+
+		ReplicationSlotsComputeRequiredXmin(true);
+
+		LWLockRelease(ProcArrayLock);
+	}
+
 	ctx = StartupDecodingContext(output_plugin_options,
-								 start_lsn, InvalidTransactionId, false,
+								 start_lsn, xmin_horizon, need_full_snapshot,
 								 fast_forward, xl_routine, prepare_write,
 								 do_write, update_progress);
 
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index fa1b641a2b..1191c70eb0 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -208,6 +208,7 @@ pg_logical_slot_get_changes_guts(FunctionCallInfo fcinfo, bool confirm, bool bin
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									options,
 									false,
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/repl_gram.y b/src/backend/replication/repl_gram.y
index 0c874e33cf..957a5cc653 100644
--- a/src/backend/replication/repl_gram.y
+++ b/src/backend/replication/repl_gram.y
@@ -65,6 +65,7 @@ Node *replication_parse_result;
 %token K_CREATE_REPLICATION_SLOT
 %token K_DROP_REPLICATION_SLOT
 %token K_TIMELINE_HISTORY
+%token K_CREATE_REPLICATION_SNAPSHOT
 %token K_WAIT
 %token K_TIMELINE
 %token K_PHYSICAL
@@ -80,7 +81,7 @@ Node *replication_parse_result;
 %type <node>	command
 %type <node>	base_backup start_replication start_logical_replication
 				create_replication_slot drop_replication_slot identify_system
-				read_replication_slot timeline_history show
+				read_replication_slot timeline_history show create_replication_snapshot
 %type <list>	generic_option_list
 %type <defelt>	generic_option
 %type <uintval>	opt_timeline
@@ -114,6 +115,7 @@ command:
 			| read_replication_slot
 			| timeline_history
 			| show
+			| create_replication_snapshot
 			;
 
 /*
@@ -307,6 +309,19 @@ timeline_history:
 				}
 			;
 
+/*
+ * CREATE_REPLICATION_SNAPSHOT %s options
+ */
+create_replication_snapshot:
+			K_CREATE_REPLICATION_SNAPSHOT var_name plugin_options
+				{
+					CreateReplicationSnapshotCmd *n = makeNode(CreateReplicationSnapshotCmd);
+					n->slotname = $2;
+					n->options = $3;
+					$$ = (Node *) n;
+				}
+			;
+
 opt_physical:
 			K_PHYSICAL
 			| /* EMPTY */
@@ -400,6 +415,7 @@ ident_or_keyword:
 			| K_CREATE_REPLICATION_SLOT	{ $$ = "create_replication_slot"; }
 			| K_DROP_REPLICATION_SLOT		{ $$ = "drop_replication_slot"; }
 			| K_TIMELINE_HISTORY			{ $$ = "timeline_history"; }
+			| K_CREATE_REPLICATION_SNAPSHOT	{ $$ = "create_replication_snapshot"; }
 			| K_WAIT						{ $$ = "wait"; }
 			| K_TIMELINE					{ $$ = "timeline"; }
 			| K_PHYSICAL					{ $$ = "physical"; }
diff --git a/src/backend/replication/repl_scanner.l b/src/backend/replication/repl_scanner.l
index cb467ca46f..5ba2e9c54b 100644
--- a/src/backend/replication/repl_scanner.l
+++ b/src/backend/replication/repl_scanner.l
@@ -126,6 +126,7 @@ START_REPLICATION	{ return K_START_REPLICATION; }
 CREATE_REPLICATION_SLOT		{ return K_CREATE_REPLICATION_SLOT; }
 DROP_REPLICATION_SLOT		{ return K_DROP_REPLICATION_SLOT; }
 TIMELINE_HISTORY	{ return K_TIMELINE_HISTORY; }
+CREATE_REPLICATION_SNAPSHOT	{ return K_CREATE_REPLICATION_SNAPSHOT; }
 PHYSICAL			{ return K_PHYSICAL; }
 RESERVE_WAL			{ return K_RESERVE_WAL; }
 LOGICAL				{ return K_LOGICAL; }
@@ -303,6 +304,7 @@ replication_scanner_is_replication_command(void)
 		case K_DROP_REPLICATION_SLOT:
 		case K_READ_REPLICATION_SLOT:
 		case K_TIMELINE_HISTORY:
+		case K_CREATE_REPLICATION_SNAPSHOT:
 		case K_SHOW:
 			/* Yes; push back the first token so we can parse later. */
 			repl_pushed_back_token = first_token;
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 2f3c964824..b3ae11b2c8 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -478,6 +478,7 @@ pg_logical_replication_slot_advance(XLogRecPtr moveto)
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									NIL,
 									true,	/* fast_forward */
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 75e8363e24..d2ae504ce5 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -238,6 +238,7 @@ static void CreateReplicationSlot(CreateReplicationSlotCmd *cmd);
 static void DropReplicationSlot(DropReplicationSlotCmd *cmd);
 static void StartReplication(StartReplicationCmd *cmd);
 static void StartLogicalReplication(StartReplicationCmd *cmd);
+static void CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd);
 static void ProcessStandbyMessage(void);
 static void ProcessStandbyReplyMessage(void);
 static void ProcessStandbyHSFeedbackMessage(void);
@@ -1280,7 +1281,7 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	 * are reported early.
 	 */
 	logical_decoding_ctx =
-		CreateDecodingContext(cmd->startpoint, cmd->options, false,
+		CreateDecodingContext(cmd->startpoint, cmd->options, false, false,
 							  XL_ROUTINE(.page_read = logical_read_xlog_page,
 										 .segment_open = WalSndSegmentOpen,
 										 .segment_close = wal_segment_close),
@@ -1332,6 +1333,98 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	EndCommand(&qc, DestRemote, false);
 }
 
+/*
+ * Create a snapshot from an existing replication slot.
+ */
+static void
+CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd)
+{
+	Snapshot	snap;
+	LogicalDecodingContext *ctx;
+	char		xloc[MAXFNAMELEN];
+	DestReceiver *dest;
+	TupOutputState *tstate;
+	TupleDesc	tupdesc;
+	Datum		values[1];
+	bool		nulls[1] = {0};
+
+	Assert(!MyReplicationSlot);
+
+	CheckLogicalDecodingRequirements();
+
+	if (!IsTransactionBlock())
+		ereport(ERROR,
+				(errmsg("%s must be called inside a transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (XactIsoLevel != XACT_REPEATABLE_READ)
+		ereport(ERROR,
+				(errmsg("%s must be called in REPEATABLE READ isolation mode transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (!XactReadOnly)
+		ereport(ERROR,
+				(errmsg("%s must be called in a read only transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (FirstSnapshotSet)
+		ereport(ERROR,
+				(errmsg("%s must be called before any query",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (IsSubTransaction())
+		ereport(ERROR,
+				(errmsg("%s must not be called in a subtransaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	ReplicationSlotAcquire(cmd->slotname, false);
+
+	ctx = CreateDecodingContext(MyReplicationSlot->data.restart_lsn,
+								cmd->options,
+								false,
+								true,
+								XL_ROUTINE(.page_read = logical_read_xlog_page,
+										   .segment_open = WalSndSegmentOpen,
+										   .segment_close = wal_segment_close),
+								WalSndPrepareWrite, WalSndWriteData,
+								WalSndUpdateProgress);
+
+	/*
+	 * Signal that we don't need the timeout mechanism. We're just creating
+	 * the snapshot with the replication slot and don't yet accept feedback
+	 * messages or send keepalives. As we possibly need to wait for further
+	 * WAL the walsender would otherwise possibly be killed too soon.
+	 */
+	last_reply_timestamp = 0;
+
+	/* build initial snapshot, might take a while */
+	DecodingContextFindStartpoint(ctx);
+
+	snap = SnapBuildInitialSnapshot(ctx->snapshot_builder);
+	RestoreTransactionSnapshot(snap, MyProc);
+
+	/* Don't need the decoding context anymore */
+	FreeDecodingContext(ctx);
+
+	/* Create a tuple to send consistent WAL location */
+	snprintf(xloc, sizeof(xloc), "%X/%X",
+			 LSN_FORMAT_ARGS(MyReplicationSlot->data.confirmed_flush));
+
+	dest = CreateDestReceiver(DestRemoteSimple);
+	tupdesc = CreateTemplateTupleDesc(1);
+	TupleDescInitBuiltinEntry(tupdesc, (AttrNumber) 1, "consistent_point",
+							  TEXTOID, -1, 0);
+	tstate = begin_tup_output_tupdesc(dest, tupdesc, &TTSOpsVirtual);
+
+	/* consistent wal location */
+	values[0] = CStringGetTextDatum(xloc);
+
+	do_tup_output(tstate, values, nulls);
+	end_tup_output(tstate);
+
+	ReplicationSlotRelease();
+}
+
 /*
  * LogicalDecodingContext 'prepare_write' callback.
  *
@@ -1860,6 +1953,15 @@ exec_replication_command(const char *cmd_string)
 			}
 			break;
 
+		case T_CreateReplicationSnapshotCmd:
+			{
+				cmdtag = "CREATE_REPLICATION_SNAPSHOT";
+				set_ps_display(cmdtag);
+				CreateReplicationSnapshot((CreateReplicationSnapshotCmd *) cmd_node);
+				EndReplicationCommand(cmdtag);
+				break;
+			}
+
 		default:
 			elog(ERROR, "unrecognized replication command node tag: %u",
 				 cmd_node->type);
diff --git a/src/include/nodes/replnodes.h b/src/include/nodes/replnodes.h
index 4321ba8f86..154ab74f33 100644
--- a/src/include/nodes/replnodes.h
+++ b/src/include/nodes/replnodes.h
@@ -108,4 +108,15 @@ typedef struct TimeLineHistoryCmd
 	TimeLineID	timeline;
 } TimeLineHistoryCmd;
 
+/* ----------------------
+ *		CREATE_REPLICATION_SNAPSHOT command
+ * ----------------------
+ */
+typedef struct CreateReplicationSnapshotCmd
+{
+	NodeTag		type;
+	char	   *slotname;
+	List	   *options;
+} CreateReplicationSnapshotCmd;
+
 #endif							/* REPLNODES_H */
diff --git a/src/include/replication/logical.h b/src/include/replication/logical.h
index 5f49554ea0..6535786a0e 100644
--- a/src/include/replication/logical.h
+++ b/src/include/replication/logical.h
@@ -125,6 +125,7 @@ extern LogicalDecodingContext *CreateInitDecodingContext(const char *plugin,
 extern LogicalDecodingContext *CreateDecodingContext(XLogRecPtr start_lsn,
 													 List *output_plugin_options,
 													 bool fast_forward,
+													 bool need_full_snapshot,
 													 XLogReaderRoutine *xl_routine,
 													 LogicalOutputPluginWriterPrepareWrite prepare_write,
 													 LogicalOutputPluginWriterWrite do_write,
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index decffe352d..bd11f9f31e 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -384,6 +384,16 @@ typedef WalRcvExecResult *(*walrcv_exec_fn) (WalReceiverConn *conn,
  */
 typedef void (*walrcv_disconnect_fn) (WalReceiverConn *conn);
 
+/*
+ * walrcv_slot_snapshot_fn
+ *
+ * Create a snapshot by an existing replication slot
+ */
+typedef void (*walrcv_slot_snapshot_fn) (WalReceiverConn *conn,
+										 char *slotname,
+										 const WalRcvStreamOptions *options,
+										 XLogRecPtr *lsn);
+
 typedef struct WalReceiverFunctionsType
 {
 	walrcv_connect_fn walrcv_connect;
@@ -401,6 +411,7 @@ typedef struct WalReceiverFunctionsType
 	walrcv_get_backend_pid_fn walrcv_get_backend_pid;
 	walrcv_exec_fn walrcv_exec;
 	walrcv_disconnect_fn walrcv_disconnect;
+	walrcv_slot_snapshot_fn walrcv_slot_snapshot;
 } WalReceiverFunctionsType;
 
 extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
@@ -435,6 +446,8 @@ extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
 	WalReceiverFunctions->walrcv_exec(conn, exec, nRetTypes, retTypes)
 #define walrcv_disconnect(conn) \
 	WalReceiverFunctions->walrcv_disconnect(conn)
+#define walrcv_slot_snapshot(conn, slotname, options, lsn) \
+	WalReceiverFunctions->walrcv_slot_snapshot(conn, slotname, options, lsn)
 
 static inline void
 walrcv_clear_result(WalRcvExecResult *walres)
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 22ea42c16b..fb5cc1d599 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2331,6 +2331,7 @@ ReplicationSlotCtlData
 ReplicationSlotOnDisk
 ReplicationSlotPersistency
 ReplicationSlotPersistentData
+CreateReplicationSnapshotCmd
 ReplicationState
 ReplicationStateCtl
 ReplicationStateOnDisk
@@ -3805,6 +3806,7 @@ walrcv_receive_fn
 walrcv_send_fn
 walrcv_server_version_fn
 walrcv_startstreaming_fn
+walrcv_slot_snapshot_fn
 wchar2mb_with_len_converter
 wchar_t
 win32_deadchild_waitinfo
-- 
2.25.1

v11-0002-Reuse-Logical-Replication-Background-worker.patchapplication/octet-stream; name=v11-0002-Reuse-Logical-Replication-Background-worker.patchDownload
From bcfa6abff4feaa9f965bdf31cf6cedf8e53e285d Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH v11 2/2] Reuse Logical Replication Background worker

This commit allows reusing tablesync workers for syncing more than one relation sequantially during their lifetime, instead of exiting after only syncing one relation.

Before this commit, tablesync workers were capable of syncing only one
relation. For each table, a sync worker was launched, then a new
replication slot and new origin were created.

Now, tablesync workers are not only limited with one relation and can move to another relation in the same subscription and reuse existing
replication slots and origins

This reduces the overhead of launching/killing a new background worker for each relation.
By reusing tablesync workers, replication slots and origins created for tablesync can be reused as well.
Removing the burden of creating/dropping replication slot/origin improves tablesync speed significantly especially for empty or small tables.

A new tablesync worker gets launched only if the number of tablesync
workers for the subscription does not exceed
max_sync_workers_per_subscription. If there is a table needs to be synced, a tablesync worker picks that up and processes it.The worker continues to picking new tables to sync until there is no table left for synchronization in the subscription.

If the state of the current table is INIT or DATASYNC, tablesync worker needs a
repliation slot/origin. If the worker has not created slot and origin in
its previous runs, it will create them. Otherwise the worker reuses
slot and origin created by itself earlier. Tables in FINISHEDCOPY are
expected to have a replication slot and origin. Tablesync worker proceed
with existing slot and origin of FINISHEDCOPY tables and do not need to
create new ones.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/catalogs.sgml                    |  31 ++
 src/backend/catalog/pg_subscription.c         | 247 ++++++++-
 src/backend/commands/subscriptioncmds.c       | 226 ++++++---
 .../replication/logical/applyparallelworker.c |   3 +-
 src/backend/replication/logical/launcher.c    |   9 +-
 src/backend/replication/logical/tablesync.c   | 478 ++++++++++++++----
 src/backend/replication/logical/worker.c      | 396 +++++++++------
 src/include/catalog/pg_subscription.h         |   6 +
 src/include/catalog/pg_subscription_rel.h     |  14 +-
 src/include/replication/slot.h                |   3 +-
 src/include/replication/worker_internal.h     |  29 +-
 11 files changed, 1107 insertions(+), 335 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index c1e4048054..31b4cfbbe4 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8002,6 +8002,19 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        origin.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>sublastusedid</structfield> <type>int8</type>
+      </para>
+      <para>
+      The last used ID for tablesync workers. It acts as an unique identifier
+      for replication slots which are created by tablesync workers.
+      The last used ID needs to be persisted to make logical replication safely
+      proceed after any interruption. If sublastusedid is 0, then no table has
+      been synced yet.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
@@ -8086,6 +8099,24 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        otherwise null
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srrelslotname</structfield> <type>name</type>
+      </para>
+      <para>
+       Replication slot name that is used for synchronization of relation
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srreloriginname</structfield> <type>name</type>
+      </para>
+      <para>
+       Origin name that is used for tracking synchronization of relation
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index a56ae311c3..cb4001825f 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -114,6 +114,14 @@ GetSubscription(Oid subid, bool missing_ok)
 	Assert(!isnull);
 	sub->origin = TextDatumGetCString(datum);
 
+	/* Get last used id */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
+							tup,
+							Anum_pg_subscription_sublastusedid,
+							&isnull);
+	Assert(!isnull);
+	sub->lastusedid = DatumGetInt64(datum);
+
 	ReleaseSysCache(tup);
 
 	return sub;
@@ -205,6 +213,44 @@ DisableSubscription(Oid subid)
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update the last used replication slot ID for the given subscription.
+ */
+void
+UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid)
+{
+	Relation	rel;
+	bool		nulls[Natts_pg_subscription];
+	bool		replaces[Natts_pg_subscription];
+	Datum		values[Natts_pg_subscription];
+	HeapTuple	tup;
+
+	/* Look up the subscription in the catalog */
+	rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+	tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	/* Form a new tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	replaces[Anum_pg_subscription_sublastusedid - 1] = true;
+	values[Anum_pg_subscription_sublastusedid- 1] = Int64GetDatum(lastusedid);
+
+	/* Update the catalog */
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+	heap_freetuple(tup);
+
+	table_close(rel, NoLock);
+}
+
 /*
  * Convert text array to list of strings.
  *
@@ -234,7 +280,7 @@ textarray_to_stringlist(ArrayType *textarray)
  */
 void
 AddSubscriptionRelState(Oid subid, Oid relid, char state,
-						XLogRecPtr sublsn)
+						XLogRecPtr sublsn, char *relslotname, char *reloriginname)
 {
 	Relation	rel;
 	HeapTuple	tup;
@@ -263,6 +309,16 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
 	else
 		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -275,6 +331,60 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Internal function to modify columns for relation state update
+ */
+static void
+UpdateSubscriptionRelState_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char state,
+									XLogRecPtr sublsn)
+{
+	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
+	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+
+	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (sublsn != InvalidXLogRecPtr)
+		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
+	else
+		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication slot update
+ */
+static void
+UpdateSubscriptionRelReplicationSlot_internal(Datum *values,
+											bool *nulls,
+											bool *replaces,
+											char *relslotname)
+{
+	replaces[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication origin update
+ */
+static void
+UpdateSubscriptionRelOrigin_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char *reloriginname)
+{
+	replaces[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+}
+
 /*
  * Update the state of a subscription table.
  */
@@ -305,14 +415,56 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	memset(nulls, false, sizeof(nulls));
 	memset(replaces, false, sizeof(replaces));
 
-	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
-	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
 
-	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
-	if (sublsn != InvalidXLogRecPtr)
-		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
-	else
-		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update replication slot name, origin name and state of
+ * a subscription table in one transaction.
+ */
+void
+UpdateSubscriptionRel(Oid subid,
+					  Oid relid,
+					  char state,
+					  XLogRecPtr sublsn,
+					  char *relslotname,
+					  char *reloriginname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
+	UpdateSubscriptionRelOrigin_internal(values, nulls, replaces, reloriginname);
 
 	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
 							replaces);
@@ -324,6 +476,85 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Get origin name of subscription table.
+ *
+ * reloriginname's value has the replication origin name if the origin exists.
+ */
+void
+GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*originname;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srreloriginname, isnull);
+	if (!*isnull)
+	{
+		originname = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(reloriginname, originname, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
+/*
+ * Get replication slot name of subscription table.
+ *
+ * slotname's value has the replication slot name if the subscription has any.
+ */
+void
+GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*relrepslot;
+	bool		isnull;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srrelslotname, &isnull);
+	if (!isnull)
+	{
+		relrepslot = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(slotname, relrepslot, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
 /*
  * Get state of subscription table.
  *
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 464db6d247..49d5ec8eca 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -649,6 +649,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 		publicationListToArray(publications);
 	values[Anum_pg_subscription_suborigin - 1] =
 		CStringGetTextDatum(opts.origin);
+	values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(0);
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -709,7 +710,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 										 rv->schemaname, rv->relname);
 
 				AddSubscriptionRelState(subid, relid, table_state,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 			}
 
 			/*
@@ -799,6 +800,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	} SubRemoveRels;
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -876,7 +879,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				AddSubscriptionRelState(sub->oid, relid,
 										copy_data ? SUBREL_STATE_INIT : SUBREL_STATE_READY,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 				ereport(DEBUG1,
 						(errmsg_internal("table \"%s.%s\" added to subscription \"%s\"",
 										 rv->schemaname, rv->relname, sub->name)));
@@ -900,6 +903,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				char		state;
 				XLogRecPtr	statelsn;
+				char		slotname[NAMEDATALEN] = {0};
 
 				/*
 				 * Lock pg_subscription_rel with AccessExclusiveLock to
@@ -926,7 +930,29 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/*
+				 * Find the logical replication sync worker. If exists, store
+				 * the slot number for dropping associated replication slots
+				 * later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot_name);
+				}
+				else
+				{
+					/*
+					 * Sync of this relation might be failed in an earlier
+					 * attempt, but the replication slot might still exist.
+					 */
+					GetSubscriptionRelReplicationSlot(sub->oid, relid, slotname);
+					if (strlen(slotname) > 0)
+						sub_remove_slots = lappend(sub_remove_slots, slotname);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
@@ -960,31 +986,24 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
-		 * the database operations we won't be able to rollback dropped slots.
+		 * Drop the replication slots associated with tablesync workers for
+		 * removed tables. This has to be at the end because otherwise if
+		 * there is an error while doing the database operations we won't be
+		 * able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			memcpy(syncslotname, lfirst(lc), sizeof(NAMEDATALEN));
+
+			/*
+			 * There is no certainty, maybe the slot does not exist yet. Also,
+			 * if we fail after removing some of the slots, next time, it will
+			 * again try to drop already dropped slots and fail. For these
+			 * reasons, we allow missing_ok = true for the drop.
+			 */
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1384,6 +1403,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	char	   *subname;
 	char	   *conninfo;
 	char	   *slotname;
+	int64		lastusedid;
 	List	   *subworkers;
 	ListCell   *lc;
 	char		originname[NAMEDATALEN];
@@ -1455,6 +1475,14 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	else
 		slotname = NULL;
 
+	/* Get the last used identifier by the subscription */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID, tup,
+							Anum_pg_subscription_sublastusedid, &isnull);
+	if (!isnull)
+		lastusedid = DatumGetInt64(datum);
+	else
+		lastusedid = 0;
+
 	/*
 	 * Since dropping a replication slot is not transactional, the replication
 	 * slot stays dropped even if the transaction rolls back.  So we cannot
@@ -1504,6 +1532,8 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	}
 	list_free(subworkers);
 
+	rstates = GetSubscriptionRelations(subid, true);
+
 	/*
 	 * Remove the no-longer-useful entry in the launcher's table of apply
 	 * worker start times.
@@ -1515,36 +1545,26 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	ApplyLauncherForgetWorkerStartTime(subid);
 
 	/*
-	 * Cleanup of tablesync replication origins.
-	 *
-	 * Any READY-state relations would already have dealt with clean-ups.
+	 * Cleanup of tablesync replication origins associated with the
+	 * subscription, if exists. Try to drop origins by creating all origin
+	 * names created for this subscription.
 	 *
 	 * Note that the state can't change because we have already stopped both
 	 * the apply and tablesync workers and they can't restart because of
 	 * exclusive lock on the subscription.
+	 *
+	 * XXX: This can be handled better instead of looping through all possible
 	 */
-	rstates = GetSubscriptionRelations(subid, true);
-	foreach(lc, rstates)
+	for (int64 i = 1; i <= lastusedid; i++)
 	{
-		SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-		Oid			relid = rstate->relid;
-
-		/* Only cleanup resources of tablesync workers */
-		if (!OidIsValid(relid))
-			continue;
+		char		originname_to_drop[NAMEDATALEN] = {0};
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * It is possible that the origin is not yet created for tablesync
-		 * worker so passing missing_ok = true. This can happen for the states
-		 * before SUBREL_STATE_FINISHEDCOPY.
-		 */
-		ReplicationOriginNameForLogicalRep(subid, relid, originname,
-										   sizeof(originname));
-		replorigin_drop_by_name(originname, true, false);
+		snprintf(originname_to_drop, sizeof(originname_to_drop), "pg_%u_%lld", subid, (long long) i);
+		/* missing_ok = true, since the origin might be already dropped. */
+		replorigin_drop_by_name(originname_to_drop, true, false);
 	}
 
+
 	/* Clean up dependencies */
 	deleteSharedDependencyRecordsFor(SubscriptionRelationId, subid, 0);
 
@@ -1596,39 +1616,17 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
+		List	   *slots = NULL;
 
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
 
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char	   *syncslotname = (char *) lfirst(lc);
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1651,6 +1649,71 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * Get the replication slot names associated with the subscription.
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok)
+{
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+					 " FROM pg_replication_slots"
+					 " WHERE slot_name LIKE 'pg_%i_sync_%%';",
+					 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+					errmsg("could not receive list of slots associated with the subscription %u, error: %s",
+					subid, res->err));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();
+		return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -2005,6 +2068,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -2015,18 +2079,20 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
-			char		syncslotname[NAMEDATALEN] = {0};
-
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
-				 syncslotname);
+				 worker->slot_name);
 		}
 	}
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 4518683779..6114970289 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -440,7 +440,8 @@ pa_launch_parallel_worker(void)
 										MySubscription->name,
 										MyLogicalRepWorker->userid,
 										InvalidOid,
-										dsm_segment_handle(winfo->dsm_seg));
+										dsm_segment_handle(winfo->dsm_seg),
+										InvalidRepSlotId);
 
 	if (launched)
 	{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 970d170e73..226aa3f58f 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -304,7 +304,7 @@ logicalrep_workers_find(Oid subid, bool only_running)
  */
 bool
 logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
-						 Oid relid, dsm_handle subworker_dsm)
+						 Oid relid, dsm_handle subworker_dsm, int64 slotid)
 {
 	BackgroundWorker bgw;
 	BackgroundWorkerHandle *bgw_handle;
@@ -430,6 +430,9 @@ retry:
 	worker->launch_time = now;
 	worker->in_use = true;
 	worker->generation++;
+	worker->created_slot = false;
+	worker->rep_slot_id = slotid;
+	worker->slot_name = (char *) palloc(NAMEDATALEN);
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -437,6 +440,7 @@ retry:
 	worker->relid = relid;
 	worker->relstate = SUBREL_STATE_UNKNOWN;
 	worker->relstate_lsn = InvalidXLogRecPtr;
+	worker->ready_to_reuse = false;
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
@@ -1155,7 +1159,8 @@ ApplyLauncherMain(Datum main_arg)
 				ApplyLauncherSetWorkerStartTime(sub->oid, now);
 				logicalrep_worker_launch(sub->dbid, sub->oid, sub->name,
 										 sub->owner, InvalidOid,
-										 DSM_HANDLE_INVALID);
+										 DSM_HANDLE_INVALID,
+										 InvalidRepSlotId);
 			}
 			else
 			{
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 07eea504ba..6983871e6b 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -127,11 +127,10 @@ static bool FetchTableStates(bool *started_tx);
 static StringInfo copybuf = NULL;
 
 /*
- * Exit routine for synchronization worker.
+ * Prepares the synchronization worker for reuse or exit.
  */
 static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+clean_sync_worker(void)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -143,18 +142,28 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
-	/* And flush all writes. */
-	XLogFlush(GetXLogWriteRecPtr());
-
-	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
-	CommitTransactionCommand();
+	/*
+	 * Disconnect from publisher. Otherwise reused sync workers causes
+	 * exceeding max_wal_senders
+	 */
+	walrcv_disconnect(LogRepWorkerWalRcvConn);
+	LogRepWorkerWalRcvConn = NULL;
 
 	/* Find the leader apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
+}
+
+/*
+ * Exit routine for synchronization worker.
+ */
+static void
+pg_attribute_noreturn()
+finish_sync_worker(void)
+{
+	clean_sync_worker();
+
+	/* And flush all writes. */
+	XLogFlush(GetXLogWriteRecPtr());
 
 	/* Stop gracefully */
 	proc_exit(0);
@@ -284,6 +293,10 @@ invalidate_syncing_table_states(Datum arg, int cacheid, uint32 hashvalue)
 static void
 process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 {
+	List	   *rstates;
+	SubscriptionRelState *rstate;
+	ListCell   *lc;
+
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_CATCHUP &&
@@ -292,6 +305,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		TimeLineID	tli;
 		char		syncslotname[NAMEDATALEN] = {0};
 		char		originname[NAMEDATALEN] = {0};
+		bool		is_streaming_ended = false;
 
 		MyLogicalRepWorker->relstate = SUBREL_STATE_SYNCDONE;
 		MyLogicalRepWorker->relstate_lsn = current_lsn;
@@ -308,40 +322,29 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 								   MyLogicalRepWorker->relid,
 								   MyLogicalRepWorker->relstate,
 								   MyLogicalRepWorker->relstate_lsn);
+		CommitTransactionCommand();
 
 		/*
-		 * End streaming so that LogRepWorkerWalRcvConn can be used to drop
-		 * the slot.
-		 */
-		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
-
-		/*
-		 * Cleanup the tablesync slot.
+		 * Cleanup the tablesync slot. If the slot name used by this worker is
+		 * different from the default slot name for the worker, this means the
+		 * current table had started to being synchronized by another worker
+		 * and replication slot. And this worker is reusing a replication slot
+		 * from a previous attempt. We do not need that replication slot
+		 * anymore.
 		 *
 		 * This has to be done after updating the state because otherwise if
 		 * there is an error while doing the database operations we won't be
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
+										MyLogicalRepWorker->rep_slot_id,
 										syncslotname,
 										sizeof(syncslotname));
 
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
-		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
-
-		CommitTransactionCommand();
-		pgstat_report_stat(false);
-
-		/*
-		 * Start a new transaction to clean up the tablesync origin tracking.
-		 * This transaction will be ended within the finish_sync_worker().
-		 * Now, even, if we fail to remove this here, the apply worker will
-		 * ensure to clean it up afterward.
+		 * We are safe to drop the replication tracking origin after this
+		 * point. Now, even, if we fail to remove this here, the apply worker
+		 * will ensure to clean it up afterward.
 		 *
 		 * We need to do this after the table state is set to SYNCDONE.
 		 * Otherwise, if an error occurs while performing the database
@@ -350,34 +353,148 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 * have been cleared before restart. So, the restarted worker will use
 		 * invalid replication progress state resulting in replay of
 		 * transactions that have already been applied.
+		 *
+		 * Firstly reset the origin session to remove the ownership of the
+		 * slot. This is needed to allow the origin to be dropped or reused
+		 * later.
+		 */
+		replorigin_session_reset();
+		replorigin_session_origin = InvalidRepOriginId;
+		replorigin_session_origin_lsn = InvalidXLogRecPtr;
+		replorigin_session_origin_timestamp = 0;
+
+		StartTransactionCommand();
+		if (MyLogicalRepWorker->slot_name && strcmp(syncslotname, MyLogicalRepWorker->slot_name) != 0)
+		{
+			/*
+			 * End streaming so that LogRepWorkerWalRcvConn can be used to
+			 * drop the slot.
+			 */
+			walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+			is_streaming_ended = true;
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
+
+			ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+											   MyLogicalRepWorker->relid,
+											   originname,
+											   sizeof(originname));
+
+			/*
+			 * Drop replication origin
+			 *
+			 * There is a chance that the user is concurrently performing refresh
+			 * for the subscription where we remove the table state and its origin
+			 * or the apply worker would have removed this origin. So passing
+			 * missing_ok = true.
+			 */
+			replorigin_drop_by_name(originname, true, false);
+		}
+
+		/*
+		 * We are safe to remove persisted replication slot and origin data,
+		 * since it's already in SYNCDONE state. They will not be needed
+		 * anymore.
 		 */
+		UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+							  MyLogicalRepWorker->relid,
+							  MyLogicalRepWorker->relstate,
+							  MyLogicalRepWorker->relstate_lsn,
+							  NULL,
+							  NULL);
+		ereport(DEBUG2,
+			(errmsg("process_syncing_tables_for_sync: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					"NULL", "NULL", MyLogicalRepWorker->relstate,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", relation \"%s\" with relid %u has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
+
+		CommitTransactionCommand();
+		pgstat_report_stat(false);
+
 		StartTransactionCommand();
 
+		/*
+		 * This should return the default origin name for the worker. Even if
+		 * the worker used a different origin for this table, it should be
+		 * dropped and removed from the catalog so far.
+		 */
 		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
 										   MyLogicalRepWorker->relid,
 										   originname,
 										   sizeof(originname));
 
 		/*
-		 * Resetting the origin session removes the ownership of the slot.
-		 * This is needed to allow the origin to be dropped.
+		 * Check if any table whose relation state is still INIT. If a table
+		 * in INIT state is found, the worker will not be finished, it will be
+		 * reused instead.
 		 */
-		replorigin_session_reset();
-		replorigin_session_origin = InvalidRepOriginId;
-		replorigin_session_origin_lsn = InvalidXLogRecPtr;
-		replorigin_session_origin_timestamp = 0;
+		rstates = GetSubscriptionRelations(MySubscription->oid, true);
+		rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
+
+		foreach(lc, rstates)
+		{
+			memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
+
+			/*
+			 * Pick the table for the next run if it is not already picked up
+			 * by another worker.
+			 */
+			LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+			if (rstate->state != SUBREL_STATE_SYNCDONE &&
+				!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+			{
+				/* Update worker state for the next table */
+				MyLogicalRepWorker->relid = rstate->relid;
+				MyLogicalRepWorker->relstate = rstate->state;
+				MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+				MyLogicalRepWorker->ready_to_reuse = true;
+				LWLockRelease(LogicalRepWorkerLock);
+				break;
+			}
+			LWLockRelease(LogicalRepWorkerLock);
+		}
 
 		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * There is a chance that the user is concurrently performing refresh
-		 * for the subscription where we remove the table state and its origin
-		 * or the apply worker would have removed this origin. So passing
-		 * missing_ok = true.
+		 * If the worker is ready to be reused, clean up the worker for next
+		 * relations.
+		 * If there is no more work left for this worker, drop replication slot
+		 * and origin. Then stop the worker gracefully.
 		 */
-		replorigin_drop_by_name(originname, true, false);
+		if (!MyLogicalRepWorker->ready_to_reuse)
+		{
+			/*
+			 * It is important to give an error if we are unable to drop the
+			 * slot, otherwise, it won't be dropped till the corresponding
+			 * subscription is dropped. So passing missing_ok = false.
+			 */
+			if (MyLogicalRepWorker->created_slot)
+			{
+				/* End streaming if it's not already ended. */
+				if (!is_streaming_ended)
+					walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+				ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
+			}
+
+			/*
+			 * Drop replication origin before exiting.
+			 *
+			 * There is a chance that the user is concurrently performing refresh
+			 * for the subscription where we remove the table state and its origin
+			 * or the apply worker would have removed this origin. So passing
+			 * missing_ok = true.
+			 */
+			replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
+			finish_sync_worker();
+		}
+		else
+		{
+			clean_sync_worker();
+		}
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -464,6 +581,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 			if (current_lsn >= rstate->lsn)
 			{
 				char		originname[NAMEDATALEN];
+				bool		is_origin_null = true;
 
 				rstate->state = SUBREL_STATE_READY;
 				rstate->lsn = current_lsn;
@@ -484,18 +602,31 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				 * error while dropping we won't restart it to drop the
 				 * origin. So passing missing_ok = true.
 				 */
-				ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-												   rstate->relid,
-												   originname,
-												   sizeof(originname));
-				replorigin_drop_by_name(originname, true, false);
+				GetSubscriptionRelOrigin(MyLogicalRepWorker->subid,
+										 rstate->relid, originname,
+										 &is_origin_null);
+
+				if (!is_origin_null)
+				{
+					replorigin_drop_by_name(originname, true, false);
+				}
 
 				/*
 				 * Update the state to READY only after the origin cleanup.
 				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-										   rstate->relid, rstate->state,
-										   rstate->lsn);
+				UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+									  rstate->relid,
+									  rstate->state,
+									  rstate->lsn,
+									  NULL,
+									  NULL);
+				ereport(DEBUG2,
+					(errmsg("process_syncing_tables_for_apply: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+							"NULL", "NULL", rstate->state,
+							rstate->relid, MyLogicalRepWorker->subid)));
+
+				CommitTransactionCommand();
+				started_tx = false;
 			}
 		}
 		else
@@ -584,12 +715,25 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						TimestampDifferenceExceeds(hentry->last_start_time, now,
 												   wal_retrieve_retry_interval))
 					{
+						if (IsTransactionState())
+							CommitTransactionCommand();
+						StartTransactionCommand();
+						started_tx = true;
+
+						MySubscription->lastusedid++;
+						UpdateSubscriptionLastSlotId(MyLogicalRepWorker->subid,
+													 MySubscription->lastusedid);
+						ereport(DEBUG2,
+								(errmsg("process_syncing_tables_for_apply: incremented lastusedid to %lld for subscription %u",
+										(long long) MySubscription->lastusedid, MySubscription->oid)));
+
 						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
 												 MySubscription->oid,
 												 MySubscription->name,
 												 MyLogicalRepWorker->userid,
 												 rstate->relid,
-												 DSM_HANDLE_INVALID);
+												 DSM_HANDLE_INVALID,
+												 MySubscription->lastusedid);
 						hentry->last_start_time = now;
 					}
 				}
@@ -1198,8 +1342,8 @@ copy_table(Relation rel)
  * The name must not exceed NAMEDATALEN - 1 because of remote node constraints
  * on slot name length. We append system_identifier to avoid slot_name
  * collision with subscriptions in other clusters. With the current scheme
- * pg_%u_sync_%u_UINT64_FORMAT (3 + 10 + 6 + 10 + 20 + '\0'), the maximum
- * length of slot_name will be 50.
+ * pg_%u_sync_%lu_UINT64_FORMAT (3 + 10 + 6 + 20 + 20 + '\0'), the maximum
+ * length of slot_name will be 45.
  *
  * The returned slot name is stored in the supplied buffer (syncslotname) with
  * the given size.
@@ -1210,11 +1354,11 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
+ReplicationSlotNameForTablesync(Oid suboid, int64 slotid,
 								char *syncslotname, Size szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%lld_" UINT64_FORMAT, suboid,
+			(long long) slotid, GetSystemIdentifier());
 }
 
 /*
@@ -1237,6 +1381,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	WalRcvExecResult *res;
 	char		originname[NAMEDATALEN];
 	RepOriginId originid;
+	char	   *prev_slotname;
 
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
@@ -1265,7 +1410,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
+									MyLogicalRepWorker->rep_slot_id,
 									slotname,
 									NAMEDATALEN);
 
@@ -1285,12 +1430,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC ||
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY);
 
+	/*
+	 * See if tablesync of the current relation has been started with another
+	 * replication slot.
+	 *
+	 * Read previous slot name from the catalog, if exists.
+	 */
+	prev_slotname = (char *) palloc(NAMEDATALEN);
+	StartTransactionCommand();
+	GetSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+									  MyLogicalRepWorker->relid,
+									  prev_slotname);
+
 	/* Assign the origin tracking record name. */
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   sizeof(originname));
 
+	CommitTransactionCommand();
+
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC)
 	{
 		/*
@@ -1304,10 +1463,53 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		 * breakdown then it wouldn't have succeeded so trying it next time
 		 * seems like a better bet.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, slotname, true);
+		if (strlen(prev_slotname) > 0)
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);
+
+			StartTransactionCommand();
+			/* Replication origin might still exist. Try to drop */
+			replorigin_drop_by_name(originname, true, false);
+
+			/*
+			 * Remove replication slot and origin name from the relation's
+			 * catalog record
+			 */
+			UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+								  MyLogicalRepWorker->relid,
+								  MyLogicalRepWorker->relstate,
+								  MyLogicalRepWorker->relstate_lsn,
+								  NULL,
+								  NULL);
+			CommitTransactionCommand();
+			ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+						"NULL", "NULL", MyLogicalRepWorker->relstate,
+						MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+		}
 	}
 	else if (MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY)
 	{
+		/*
+		 * At this point, the table that is currently being synchronized
+		 * should have its replication slot name filled in the catalog. The
+		 * tablesync process was started with another sync worker and
+		 * replication slot. We need to continue using the same replication
+		 * slot in this worker too.
+		 */
+		if (strlen(prev_slotname) == 0)
+		{
+			elog(ERROR, "Replication slot could not be found for subscription %u, relation %u",
+				 MyLogicalRepWorker->subid,
+				 MyLogicalRepWorker->relid);
+		}
+
+		/*
+		 * Proceed with the correct replication slot. Use previously created
+		 * replication slot to sync this table.
+		 */
+		memcpy(slotname, prev_slotname, NAMEDATALEN);
+
 		/*
 		 * The COPY phase was previously done, but tablesync then crashed
 		 * before it was able to finish normally.
@@ -1327,7 +1529,9 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 		goto copy_table_done;
 	}
+	pfree(prev_slotname);
 
+	/* Preparing for table copy operation */
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 	MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
 	MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
@@ -1335,11 +1539,31 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   MyLogicalRepWorker->relstate,
-							   MyLogicalRepWorker->relstate_lsn);
+
+	/*
+	 * Refresh the originname in case of having non-existing origin
+	 * from previous failed sync attempts.
+	 * If that's the case, it should be removed from the catalog so far.
+	 * Then, we can continue by reusing the origin created by the current
+	 * worker instead of .
+	 */
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									MyLogicalRepWorker->relid,
+									originname,
+									sizeof(originname));
+
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  MyLogicalRepWorker->relstate,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					slotname, originname, MyLogicalRepWorker->relstate,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+
 	pgstat_report_stat(true);
 
 	StartTransactionCommand();
@@ -1377,6 +1601,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						GetUserNameFromId(GetUserId(), true),
 						RelationGetRelationName(rel))));
 
+
 	/*
 	 * Start a transaction in the remote node in REPEATABLE READ mode.  This
 	 * ensures that both the replication slot we create (see below) and the
@@ -1392,48 +1617,95 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						res->err)));
 	walrcv_clear_result(res);
 
+	originid = replorigin_by_name(originname, true);
+
 	/*
 	 * Create a new permanent logical decoding slot. This slot will be used
 	 * for the catchup phase after COPY is done, so tell it to use the
 	 * snapshot to make the final data consistent.
+	 *
+	 * Replication slot will only be created if either this is the first run
+	 * of the worker or we're not using a previous replication slot.
 	 */
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-
-	/*
-	 * Setup replication origin tracking. The purpose of doing this before the
-	 * copy is to avoid doing the copy again due to any error in setting up
-	 * origin tracking.
-	 */
-	originid = replorigin_by_name(originname, true);
-	if (!OidIsValid(originid))
+	if (!MyLogicalRepWorker->created_slot)
 	{
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						   slotname, false /* permanent */ , false /* two_phase */ ,
+						   CRS_USE_SNAPSHOT, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: created replication slot %s for subscription %u",
+						slotname, MyLogicalRepWorker->subid)));
+
 		/*
-		 * Origin tracking does not exist, so create it now.
-		 *
-		 * Then advance to the LSN got from walrcv_create_slot. This is WAL
-		 * logged for the purpose of recovery. Locks are to prevent the
-		 * replication origin from vanishing while advancing.
+		 * Remember that we created the slot so that we will not try to create
+		 * it again.
 		 */
-		originid = replorigin_create(originname);
-
-		LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
-		replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
-						   true /* go backward */ , true /* WAL log */ );
-		UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->created_slot = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
+		/*
+		 * Setup replication origin tracking. The purpose of doing this before
+		 * the copy is to avoid doing the copy again due to any error in
+		 * setting up origin tracking.
+		 */
+		if (!OidIsValid(originid))
+		{
+			/*
+			 * Origin tracking does not exist, so create it now.
+			 */
+			originid = replorigin_create(originname);
+		}
+		else
+		{
+			/*
+			 * At this point, there shouldn't be any existing replication
+			 * origin with the same name.
+			 */
+			ereport(ERROR,
+					(errcode(ERRCODE_DUPLICATE_OBJECT),
+					 errmsg("replication origin \"%s\" already exists",
+							originname)));
+		}
 	}
 	else
 	{
-		ereport(ERROR,
-				(errcode(ERRCODE_DUPLICATE_OBJECT),
-				 errmsg("replication origin \"%s\" already exists",
-						originname)));
+		/*
+		 * Do not create a new replication slot, reuse the existing one
+		 * instead. Use a new snapshot for the replication slot to ensure that
+		 * tablesync and apply proceses are consistent with each other.
+		 */
+		WalRcvStreamOptions options;
+		int			server_version;
+
+		server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+		options.proto.logical.proto_version =
+			server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+			server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+			LOGICALREP_PROTO_VERSION_NUM;
+		options.proto.logical.publication_names = MySubscription->publications;
+
+		walrcv_slot_snapshot(LogRepWorkerWalRcvConn, slotname, &options, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: reusing replication slot %s for relation %u in subscription %u",
+						slotname, MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
 	}
 
+	/*
+	 * Advance to the LSN got from walrcv_create_slot or walrcv_slot_snapshot.
+	 * This is WAL logged for the purpose of recovery. Locks are to prevent
+	 * the replication origin from vanishing while advancing.
+	 *
+	 * Then setup replication origin tracking.
+	 */
+	LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+	replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
+					   true /* go backward */ , true /* WAL log */ );
+	UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+
 	/* Now do the initial data copy */
 	PushActiveSnapshot(GetTransactionSnapshot());
 	copy_table(rel);
@@ -1456,12 +1728,18 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	 * Update the persisted state to indicate the COPY phase is done; make it
 	 * visible to others.
 	 */
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   SUBREL_STATE_FINISHEDCOPY,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  SUBREL_STATE_FINISHEDCOPY,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					originname, slotname, SUBREL_STATE_FINISHEDCOPY,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
 
 copy_table_done:
 
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index cfb2ab6248..9651d6f752 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -388,6 +388,7 @@ static void stream_open_file(Oid subid, TransactionId xid,
 static void stream_write_change(char action, StringInfo s);
 static void stream_open_and_write_change(TransactionId xid, char action, StringInfo s);
 static void stream_close_file(void);
+static void stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
@@ -456,8 +457,16 @@ ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 {
 	if (OidIsValid(relid))
 	{
-		/* Replication origin name for tablesync workers. */
-		snprintf(originname, szoriginname, "pg_%u_%u", suboid, relid);
+		bool		is_null = true;
+
+		/*
+		 * Replication origin name for tablesync workers. First, look into the
+		 * catalog. If originname does not exist, then use the default name.
+		 */
+		GetSubscriptionRelOrigin(suboid, relid,
+								 originname, &is_null);
+		if (is_null)
+			snprintf(originname, szoriginname, "pg_%u_%lld", suboid, (long long) MyLogicalRepWorker->rep_slot_id);
 	}
 	else
 	{
@@ -3576,6 +3585,27 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can move to next table while running
+				 * process_syncing_tables_for_sync. Before we were able to
+				 * reuse tablesync workers, that
+				 * process_syncing_tables_for_sync call would exit the worker
+				 * instead of moving to next table. Now that tablesync workers
+				 * can be reused, we need to take care of memory contexts here
+				 * before moving to sync a table.
+				 */
+				if (MyLogicalRepWorker->ready_to_reuse)
+				{
+					MemoryContextResetAndDeleteChildren(ApplyMessageContext);
+					MemoryContextSwitchTo(TopMemoryContext);
+
+					/* Pop the error context stack */
+					error_context_stack = errcallback.previous;
+					apply_error_context_stack = error_context_stack;
+					return;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -3595,6 +3625,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+			if (MyLogicalRepWorker->ready_to_reuse)
+			{
+				endofstream = true;
+			}
 		}
 
 		/* Cleanup the memory. */
@@ -3697,8 +3731,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * If it's moving to next relation, this is a sync worker. Sync workers
+	 * end the streaming during process_syncing_tables_for_sync. Calling
+	 * endstreaming twice causes "no COPY in progress" errors.
+	 */
+	if (!MyLogicalRepWorker->ready_to_reuse)
+	{
+		/* All done */
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	}
 }
 
 /*
@@ -4282,6 +4324,56 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+ /* stream_build_options
+  * 	Build logical replication streaming options.
+  *
+  * This function sets streaming options including replication slot name
+  * and origin start position. Workers need these options for logical replication.
+  */
+static void
+stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4356,6 +4448,9 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 
 	/* allocate slot name in long-lived context */
 	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+
+	/* Keep the replication slot name used for this sync. */
+	MyLogicalRepWorker->slot_name = *myslotname;
 	pfree(syncslotname);
 }
 
@@ -4393,6 +4488,135 @@ start_apply(XLogRecPtr origin_startpos)
 	PG_END_TRY();
 }
 
+/*
+ * Runs the tablesync worker.
+ * It starts table sync. After successful sync,
+ * builds streaming options and starts streaming.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Set this to false for safety, in case we're already reusing the worker */
+	MyLogicalRepWorker->ready_to_reuse = false;
+
+	start_table_sync(origin_startpos, &slotname);
+
+	/*
+	 * Allocate the origin name in long-lived context for error context
+	 * message.
+	 */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   originname_size);
+	CommitTransactionCommand();
+
+	set_apply_error_context_origin(originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/* Start normal logical streaming replication. */
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+}
+
+/*
+ * Runs the apply worker.
+ * It sets up replication origin, the streaming options
+ * and then starts streaming.
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
+{
+	/* This is the leader apply worker */
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+
+	slotname = MySubscription->slotname;
+
+	/*
+		* This shouldn't happen if the subscription is enabled, but guard
+		* against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+		* crash if slot is NULL.)
+		*/
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											MySubscription->name, &err);
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+		* We don't really use the output identify_system for anything but it
+		* does some initializations on the upstream so let's still call it.
+		*/
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+	{
+		/* Start streaming with two_phase enabled */
+		options->proto.logical.twophase = true;
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
+	}
+	else
+	{
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+	}
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+}
+
 /*
  * Common initialization for leader apply worker and parallel apply worker.
  *
@@ -4464,9 +4688,10 @@ InitializeApplyWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has started",
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", relation \"%s\" with relid %u has started",
 						MySubscription->name,
-						get_rel_name(MyLogicalRepWorker->relid))));
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
 	else
 		ereport(LOG,
 		/* translator: first %s is the name of logical replication worker */
@@ -4485,7 +4710,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	/* Attach to slot */
 	logicalrep_worker_attach(worker_slot);
@@ -4513,156 +4737,46 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
-	 * the subscription relation state.
+	 * the subscription relation state. Do this outside the loop to avoid
+	 * exceeding MAX_SYSCACHE_CALLBACKS
 	 */
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
-
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
-
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-
 	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 * The loop where worker does its job. It loops until the worker is not
+	 * reused.
 	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
-
-	if (!am_tablesync_worker())
-	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
+	do {
+		if (am_tablesync_worker())
 		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
+			/*
+			 * This is a tablesync worker. Start syncing tables before
+			 * starting the apply loop.
+			 */
+			run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 		}
 		else
 		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+			/* This is leader apply worker */
+			run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 		}
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-						MySubscription->name,
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-						"?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
-
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+		/* Run the main loop. */
+		start_apply(origin_startpos);
 
+		if (MyLogicalRepWorker->ready_to_reuse)
+		{
+			StartTransactionCommand();
+			ereport(LOG,
+					(errmsg("logical replication table synchronization worker for subscription \"%s\" has moved to sync table \"%s\" with relid %u.",
+							MySubscription->name, get_rel_name(MyLogicalRepWorker->relid), MyLogicalRepWorker->relid)));
+			CommitTransactionCommand();
+		}
+	} while (MyLogicalRepWorker->ready_to_reuse);
 	proc_exit(0);
 }
 
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index b0f2a1705d..a0ee12e259 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -103,6 +103,9 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
 
 	/* Only publish data originating from the specified origin */
 	text		suborigin BKI_DEFAULT(LOGICALREP_ORIGIN_ANY);
+
+	/* The last used ID to create a replication slot for tablesync */
+	int64		sublastusedid BKI_DEFAULT(0);
 #endif
 } FormData_pg_subscription;
 
@@ -137,6 +140,8 @@ typedef struct Subscription
 	List	   *publications;	/* List of publication names to subscribe to */
 	char	   *origin;			/* Only publish data originating from the
 								 * specified origin */
+	int64		lastusedid;		/* Last used unique ID to create replication
+								 * slots in tablesync */
 } Subscription;
 
 /* Disallow streaming in-progress transactions. */
@@ -157,6 +162,7 @@ typedef struct Subscription
 extern Subscription *GetSubscription(Oid subid, bool missing_ok);
 extern void FreeSubscription(Subscription *sub);
 extern void DisableSubscription(Oid subid);
+extern void UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid);
 
 extern int	CountDBSubscriptions(Oid dbid);
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 60a2bcca23..185164d75e 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -44,6 +44,12 @@ CATALOG(pg_subscription_rel,6102,SubscriptionRelRelationId)
 											 * used for synchronization
 											 * coordination, or NULL if not
 											 * valid */
+	NameData	srrelslotname BKI_FORCE_NULL;	/* name of the replication
+												 * slot for relation in
+												 * subscription */
+	NameData	srreloriginname BKI_FORCE_NULL; /* origin name for relation in
+												 * subscription */
+
 #endif
 } FormData_pg_subscription_rel;
 
@@ -81,10 +87,16 @@ typedef struct SubscriptionRelState
 } SubscriptionRelState;
 
 extern void AddSubscriptionRelState(Oid subid, Oid relid, char state,
-									XLogRecPtr sublsn);
+									XLogRecPtr sublsn, char *relslotname, char *reloriginname);
 extern void UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 									   XLogRecPtr sublsn);
+extern void UpdateSubscriptionRel(Oid subid, Oid relid, char state,
+								  XLogRecPtr sublsn, char *relslotname, char *reloriginname);
+
 extern char GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn);
+extern void GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname);
+extern void GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull);
+
 extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 
 extern bool HasSubscriptionRelations(Oid subid);
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 8872c80cdf..3547daaaec 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -219,8 +219,9 @@ extern bool InvalidateObsoleteReplicationSlots(XLogSegNo oldestSegno);
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
 extern bool ReplicationSlotName(int index, Name name);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, Size szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, int64 slotid, char *syncslotname, Size szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List *GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index dc87a4edd1..353a066414 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -35,6 +35,23 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/*
+	 * Indicates if the sync worker created a replication slot for itself
+	 * in any point of its lifetime.
+	 * False means that the worker has not created a slot yet, and has been
+	 * reusing replication slots created by other workers so far.
+	 */
+	bool		created_slot;
+
+	/*
+	 * Unique identifier for replication slot to be created by tablesnync
+	 * workers, if needed.
+	 */
+	int64		rep_slot_id;
+
+	/* Replication slot name used by the worker. */
+	char	   *slot_name;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
@@ -56,6 +73,12 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/*
+	 * Used to indicate whether sync worker will be reused for another
+	 * relation
+	 */
+	bool		ready_to_reuse;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction or
@@ -231,7 +254,8 @@ extern LogicalRepWorker *logicalrep_worker_find(Oid subid, Oid relid,
 extern List *logicalrep_workers_find(Oid subid, bool only_running);
 extern bool logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname,
 									 Oid userid, Oid relid,
-									 dsm_handle subworker_dsm);
+									 dsm_handle subworker_dsm,
+									 int64 slotid);
 extern void logicalrep_worker_stop(Oid subid, Oid relid);
 extern void logicalrep_pa_worker_stop(int slot_no, uint16 generation);
 extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
@@ -324,4 +348,7 @@ am_parallel_apply_worker(void)
 	return isParallelApplyWorker(MyLogicalRepWorker);
 }
 
+/* Invalid identifier to be used for naming replication slots */
+#define InvalidRepSlotId	0
+
 #endif							/* WORKER_INTERNAL_H */
-- 
2.25.1

#56Melih Mutlu
m.melihmutlu@gmail.com
In reply to: wangw.fnst@fujitsu.com (#50)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Wang,

Thanks for reviewing.
Please see updated patches. [1]/messages/by-id/CAGPVpCQmEE8BygXr=Hi2N2t2kOE=PJwofn9TX0J9J4crjoXarQ@mail.gmail.com

wangw.fnst@fujitsu.com <wangw.fnst@fujitsu.com>, 7 Şub 2023 Sal, 10:28
tarihinde şunu yazdı:

1. In the function ApplyWorkerMain.
I think we need to keep the worker name as "leader apply worker" in the
comment
like the current HEAD.

Done.

I think in this case we also need to pop the error context stack before
returning. Otherwise, I think we might use the wrong callback
(apply error_callback) after we return from this function.

Done.

3. About the function UpdateSubscriptionRelReplicationSlot.
This newly introduced function UpdateSubscriptionRelReplicationSlot does
not
seem to be invoked. Do we need this function?

Removed.

I think if 'need_full_snapshot' is false, it means we will create a snapshot

that can read only catalogs. (see SnapBuild->building_full_snapshot)

Fixed.

```

'use' will use the snapshot for the current transaction executing the
command.
This option must be used in a transaction, and CREATE_REPLICATION_SLOT
must be
the first command run in that transaction.
```

So I think in the function CreateDecodingContext, when "need_full_snapshot"

is
true, we seem to need the following check, just like in the function
CreateInitDecodingContext:

```

if (IsTransactionState() &&
GetTopTransactionIdIfAny() != InvalidTransactionId)
ereport(ERROR,
(errcode(ERRCODE_ACTIVE_SQL_TRANSACTION),
errmsg("cannot create logical replication
slot in transaction that has performed writes")));
```

You're right to "use" the snapshot, it must be the first command in the
transaction. And that check happens here [2]https://github.com/postgres/postgres/blob/master/src/backend/replication/walsender.c#L1108. CreateReplicationSnapshot has
also similar check.
I think the check you're referring to is needed to actually create a
replication slot and it performs whether the snapshot will be "used" or
"exported". This is not the case for CreateReplicationSnapshot.

It seems that we also need to invoke the function

CheckLogicalDecodingRequirements in the new function
CreateReplicationSnapshot,
just like the function CreateReplicationSlot and the function
StartLogicalReplication.

Added this check.

3. The invocation of startup_cb_wrapper in the function

CreateDecodingContext.
I think we should change the third input parameter to true when invoke
function
startup_cb_wrapper for CREATE_REPLICATION_SNAPSHOT. BTW, after applying
patch
v10-0002*, these settings will be inconsistent when sync workers use
"CREATE_REPLICATION_SLOT" and "CREATE_REPLICATION_SNAPSHOT" to take
snapshots.
This input parameter (true) will let us disable streaming and two-phase
transactions in function pgoutput_startup. See the last paragraph of the
commit
message for 4648243 for more details.

I'm not sure if "is_init" should be set to true. CreateDecodingContext only
creates a context for an already existing logical slot and does not
initialize new one.
I think inconsistencies between "CREATE_REPLICATION_SLOT" and
"CREATE_REPLICATION_SNAPSHOT" are expected since one creates a new
replication slot and the other does not.
CreateDecodingContext is also used in other places as well. Not sure how
this change would affect those places. I'll look into this more. Please let
me know if I'm missing something.

[1]: /messages/by-id/CAGPVpCQmEE8BygXr=Hi2N2t2kOE=PJwofn9TX0J9J4crjoXarQ@mail.gmail.com
/messages/by-id/CAGPVpCQmEE8BygXr=Hi2N2t2kOE=PJwofn9TX0J9J4crjoXarQ@mail.gmail.com
[2]: https://github.com/postgres/postgres/blob/master/src/backend/replication/walsender.c#L1108
https://github.com/postgres/postgres/blob/master/src/backend/replication/walsender.c#L1108

Thanks,
--
Melih Mutlu
Microsoft

#57Melanie Plageman
melanieplageman@gmail.com
In reply to: Melih Mutlu (#56)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Wed, Feb 22, 2023 at 8:04 AM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi Wang,

Thanks for reviewing.
Please see updated patches. [1]

This is cool! Thanks for working on this.
I had a chance to review your patchset and I had some thoughts and
questions.

I notice that you've added a new user-facing option to make a snapshot.
I think functionality to independently make a snapshot for use elsewhere
has been discussed in the past for the implementation of different
features (e.g. [1]/messages/by-id/CA+U5nMLRjGtpskUkYSzZOEYZ_8OMc02k+O6FDi4una3mB4rS1w@mail.gmail.com pg_dump but they ended up using replication slots for
this I think?), but I'm not quite sure I understand all the implications
for providing a user-visible create snapshot command. Where can it be
used? When can the snapshot be used? In your patch's case, you know that
you can use the snapshot you are creating, but I just wonder if any
restrictions or caveats need be taken for its general use.

For the worker reuse portion of the code, could it be a separate patch
in the set? It could be independently committable and would be easier to
review (separate from repl slot reuse).

Given table sync worker reuse, I think it is worth considering a more
explicit structure for the table sync worker code now -- i.e. having a
TableSyncWorkerMain() function. Though they still do the
LogicalRepApplyLoop(), much of what else they do is different than the
apply leader.

Apply worker leader does:

ApplyWorkerMain()
walrcv_startstreaming()
LogicalRepApplyLoop()
launch table sync workers
walrcv_endstreaming()
proc_exit()

Table Sync workers master:

ApplyWorkerMain()
start_table_sync()
walrcv_create_slot()
copy_table()
walrcv_startstreaming()
start_apply()
LogicalRepApplyLoop()
walrcv_endstreaming()
proc_exit()

Now table sync workers need to loop back and do start_table_sync() again
for their new table.
You have done this in ApplyWorkerMain(). But I think that this could be
a separate main function since their main loop is effectively totally
different now than an apply worker leader.

Something like:

TableSyncWorkerMain()
TableSyncWorkerLoop()
start_table_sync()
walrcv_startstreaming()
LogicalRepApplyLoop()
walrcv_endstreaming()
wait_for_new_rel_assignment()
proc_exit()

You mainly have this structure, but it is a bit hidden and some of the
shared functions that previously may have made sense for table sync
worker and apply workers to share don't really make sense to share
anymore.

The main thing that table sync workers and apply workers share is the
logic in LogicalRepApplyLoop() (table sync workers use when they do
catchup), so perhaps we should make the other code separate?

Also on the topic of worker reuse, I was wondering if having workers
find their own next table assignment (as you have done in
process_syncing_tables_for_sync()) makes sense.

The way the whole system would work now (with your patch applied), as I
understand it, the apply leader would loop through the subscription rel
states and launch workers up to max_sync_workers_per_subscription for
every candidate table needing sync. The apply leader will continue to do
this, even though none of those workers would exit unless they die
unexpectedly. So, once it reaches max_sync_workers_per_subscription, it
won't launch any more workers.

When one of these sync workers is finished with a table (it is synced
and caught up), it will search through the subscription rel states
itself looking for a candidate table to work on.

It seems it would be common for workers to be looking through the
subscription rel states at the same time, and I don't really see how you
prevent races in who is claiming a relation to work on. Though you take
a shared lock on the LogicalRepWorkerLock, what if in between
logicalrep_worker_find() and updating my MyLogicalRepWorker->relid,
someone else also updates their relid to that relid. I don't think you
can update LogicalRepWorker->relid with only a shared lock.

I wonder if it is not better to have the apply leader, in
process_syncing_tables_for_apply(), first check for an existing worker
for the rel, then check for an available worker without an assignment,
then launch a worker?

Workers could then sleep after finishing their assignment and wait for
the leader to give them a new assignment.

Given an exclusive lock on LogicalRepWorkerLock, it may be okay for
workers to find their own table assignments from the subscriptionrel --
and perhaps this will be much more efficient from a CPU perspective. It
feels just a bit weird to have the code doing that buried in
process_syncing_tables_for_sync(). It seems like it should at least
return out to a main table sync worker loop in which workers loop
through finding a table and assigning it to themselves, syncing the
table, and catching the table up.

- Melanie

[1]: /messages/by-id/CA+U5nMLRjGtpskUkYSzZOEYZ_8OMc02k+O6FDi4una3mB4rS1w@mail.gmail.com

#58Gregory Stark (as CFM)
stark.cfm@gmail.com
In reply to: Melanie Plageman (#57)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Sun, 26 Feb 2023 at 19:11, Melanie Plageman
<melanieplageman@gmail.com> wrote:

This is cool! Thanks for working on this.
I had a chance to review your patchset and I had some thoughts and
questions.

It looks like this patch got a pretty solid review from Melanie
Plageman in February just before the CF started. It was never set to
Waiting on Author but I think that may be the right state for it.

--
Gregory Stark
As Commitfest Manager

#59Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Melanie Plageman (#57)
3 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Melanie,

Thanks for reviewing.

Melanie Plageman <melanieplageman@gmail.com>, 27 Şub 2023 Pzt, 03:10
tarihinde şunu yazdı:

I notice that you've added a new user-facing option to make a snapshot.
I think functionality to independently make a snapshot for use elsewhere
has been discussed in the past for the implementation of different
features (e.g. [1] pg_dump but they ended up using replication slots for
this I think?), but I'm not quite sure I understand all the implications
for providing a user-visible create snapshot command. Where can it be
used? When can the snapshot be used? In your patch's case, you know that
you can use the snapshot you are creating, but I just wonder if any
restrictions or caveats need be taken for its general use.

I can't say a use-case, other than this patch, that needs this user-facing
command. The main reason why I added this command as it is in the patch is
because that's already how other required communication between publisher
and subscriber is done for other operations in logical replication. Even
though it may sound similar to the case in pg_dump discussion, I think the
main difference is that calling CREATE_REPLICATION_SNAPSHOT creates a
snapshot and imports it to wherever it's called (i.e. the same transaction
which invoked CREATE_REPLICATION_SNAPSHOT ), and not used anywhere else.
But I agree that this part of the patch needs more thoughts and reviews.
Honestly, I'm not also sure if this is the ideal way to fix the "snapshot
issue" introduced by reusing the same replication slot.

For the worker reuse portion of the code, could it be a separate patch
in the set? It could be independently committable and would be easier to
review (separate from repl slot reuse).

I did this, please see the patch 0001.

You mainly have this structure, but it is a bit hidden and some of the
shared functions that previously may have made sense for table sync
worker and apply workers to share don't really make sense to share
anymore.

The main thing that table sync workers and apply workers share is the
logic in LogicalRepApplyLoop() (table sync workers use when they do
catchup), so perhaps we should make the other code separate?

You're right that apply and tablesync worker's paths are unnecessarily
intertwined. With the reusing workers/replication slots logic, I guess it
became worse.
I tried to change the structure to something similar to what you explained.
Tablesync workers have different starting point now and it simply runs as
follows:

TableSyncWorkerMain()
loop:
start_table_sync()
walrcv_startstreaming()
LogicalRepApplyLoop()
check if there is a table with INIT state
if there is such table: // reuse case
clean_sync_worker()
else: // exit case
walrcv_endstreaming()
ReplicationSlotDropAtPubNode()
replorigin_drop_by_name
break
proc_exit()

It seems it would be common for workers to be looking through the
subscription rel states at the same time, and I don't really see how you
prevent races in who is claiming a relation to work on. Though you take
a shared lock on the LogicalRepWorkerLock, what if in between
logicalrep_worker_find() and updating my MyLogicalRepWorker->relid,
someone else also updates their relid to that relid. I don't think you
can update LogicalRepWorker->relid with only a shared lock.

I wonder if it is not better to have the apply leader, in
process_syncing_tables_for_apply(), first check for an existing worker
for the rel, then check for an available worker without an assignment,
then launch a worker?

Workers could then sleep after finishing their assignment and wait for
the leader to give them a new assignment.

I'm not sure if we should rely on a single apply worker for the assignment
of several tablesync workers. I suspect that moving the assignment
responsibility to the apply worker may bring some overhead. But I agree
that shared lock on LogicalRepWorkerLock is not good. Changed it to
exclusive lock.

Given an exclusive lock on LogicalRepWorkerLock, it may be okay for
workers to find their own table assignments from the subscriptionrel --
and perhaps this will be much more efficient from a CPU perspective. It
feels just a bit weird to have the code doing that buried in
process_syncing_tables_for_sync(). It seems like it should at least
return out to a main table sync worker loop in which workers loop
through finding a table and assigning it to themselves, syncing the
table, and catching the table up.

Right, it shouldn't be process_syncing_tables_for_sync()'s responsibility.
I moved it into the TableSyncWorkerMain loop.

Also;
I did some benchmarking like I did a couple of times previously [1]/messages/by-id/CAGPVpCQdZ_oj-QFcTOhTrUTs-NCKrrZ=ZNCNPR1qe27rXV-iYw@mail.gmail.com.
Here are the recent numbers:

With empty tables:
+--------+------------+-------------+--------------+
|        |  10 tables | 100 tables  | 1000 tables  |
+--------+------------+-------------+--------------+
| master | 296.689 ms | 2579.154 ms | 41753.043 ms |
+--------+------------+-------------+--------------+
| patch  | 210.580 ms | 1724.230 ms | 36247.061 ms |
+--------+------------+-------------+--------------+
With 10 tables loaded with some data:
+--------+------------+-------------+--------------+
|        |    1 MB    | 10 MB       | 100 MB       |
+--------+------------+-------------+--------------+
| master | 568.072 ms | 2074.557 ms | 16995.399 ms |
+--------+------------+-------------+--------------+
| patch  | 470.700 ms | 1923.386 ms | 16980.686 ms |
+--------+------------+-------------+--------------+

It seems that even though master has improved since the last time I did a
similar experiment, the patch still improves the time spent in table sync
for empty/small tables.
Also there is a decrease in the performance of the patch, compared with the
previous results [1]/messages/by-id/CAGPVpCQdZ_oj-QFcTOhTrUTs-NCKrrZ=ZNCNPR1qe27rXV-iYw@mail.gmail.com. Some portion of it might be caused by switching from
shared locks to exclusive locks. I'll look into that a bit more though.

[1]: /messages/by-id/CAGPVpCQdZ_oj-QFcTOhTrUTs-NCKrrZ=ZNCNPR1qe27rXV-iYw@mail.gmail.com
/messages/by-id/CAGPVpCQdZ_oj-QFcTOhTrUTs-NCKrrZ=ZNCNPR1qe27rXV-iYw@mail.gmail.com

Best,
--
Melih Mutlu
Microsoft

Attachments:

0001-Reuse-Tablesync-Workers.patchapplication/octet-stream; name=0001-Reuse-Tablesync-Workers.patchDownload
From 0c5bb1dacf97219970ea6c7802da5141393bcfa8 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 11 Apr 2023 14:24:22 +0300
Subject: [PATCH 1/3] Reuse Tablesync Workers

This commit allows reusing tablesync workers for syncing more than one
relation sequantially during their lifetime, instead of exiting after
only syncing one relation.

Before this commit, tablesync workers were capable of syncing only one
relation. For each table, a new sync worker was launched and killed
when the worker is done with the current table.

Now, tablesync workers are not only limited with one relation and can
move to another relation in the same subscription. This reduces the
overhead of launching/killing a new background worker for each relation.

A new tablesync worker gets launched only if the number of tablesync
workers for the subscription does not exceed
max_sync_workers_per_subscription. If there is a table needs to be synced,
a tablesync worker picks that up and syncs it.The worker continues to
picking new tables to sync until there is no table left for synchronization
in the subscription.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/postmaster/bgworker.c             |   3 +
 .../replication/logical/applyparallelworker.c |   2 +-
 src/backend/replication/logical/launcher.c    |   5 +-
 src/backend/replication/logical/tablesync.c   |  48 +-
 src/backend/replication/logical/worker.c      | 525 +++++++++++++-----
 src/include/replication/logicalworker.h       |   1 +
 src/include/replication/worker_internal.h     |  10 +-
 7 files changed, 411 insertions(+), 183 deletions(-)

diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 0dd22b2351..5609919edf 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -131,6 +131,9 @@ static const struct
 	},
 	{
 		"ParallelApplyWorkerMain", ParallelApplyWorkerMain
+	},
+	{
+		"TablesyncWorkerMain", TablesyncWorkerMain
 	}
 };
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index ee7a18137f..50aa1386d5 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -940,7 +940,7 @@ ParallelApplyWorkerMain(Datum main_arg)
 	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
 		MyLogicalRepWorker->reply_time = 0;
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index ceea126231..3e89366e7c 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -440,6 +440,7 @@ retry:
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
+	worker->ready_to_reuse = false;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -460,8 +461,10 @@ retry:
 
 	if (is_parallel_apply_worker)
 		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ParallelApplyWorkerMain");
-	else
+	else if (!OidIsValid(relid))
 		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
+	else
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "TablesyncWorkerMain");
 
 	if (OidIsValid(relid))
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 0c71ae9ba7..0a813bc371 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -127,40 +127,6 @@ static bool FetchTableStates(bool *started_tx);
 
 static StringInfo copybuf = NULL;
 
-/*
- * Exit routine for synchronization worker.
- */
-static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
-{
-	/*
-	 * Commit any outstanding transaction. This is the usual case, unless
-	 * there was nothing to do for the table.
-	 */
-	if (IsTransactionState())
-	{
-		CommitTransactionCommand();
-		pgstat_report_stat(true);
-	}
-
-	/* And flush all writes. */
-	XLogFlush(GetXLogWriteRecPtr());
-
-	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
-	CommitTransactionCommand();
-
-	/* Find the leader apply worker and signal it. */
-	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
-
-	/* Stop gracefully */
-	proc_exit(0);
-}
-
 /*
  * Wait until the relation sync state is set in the catalog to the expected
  * one; return true when it happens.
@@ -378,10 +344,16 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 */
 		replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
-	}
-	else
+		/*
+		 * Sync worker is cleaned at this point. It's ready to sync next table,
+		 * if needed.
+		 */
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->ready_to_reuse = true;
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
+	}
+
+	SpinLockRelease(&MyLogicalRepWorker->relmutex);
 }
 
 /*
@@ -1275,7 +1247,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		case SUBREL_STATE_SYNCDONE:
 		case SUBREL_STATE_READY:
 		case SUBREL_STATE_UNKNOWN:
-			finish_sync_worker();	/* doesn't return */
+			sync_worker_exit();	/* doesn't return */
 	}
 
 	/* Calculate the name of the tablesync slot. */
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 879309b316..d5f87b84cb 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -392,6 +392,7 @@ static void stream_open_file(Oid subid, TransactionId xid,
 static void stream_write_change(char action, StringInfo s);
 static void stream_open_and_write_change(TransactionId xid, char action, StringInfo s);
 static void stream_close_file(void);
+static void stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
@@ -3617,6 +3618,27 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can go into process_syncing_tables_for_sync early.
+				 * Before we were able to reuse tablesync workers, that
+				 * process_syncing_tables_for_sync call would exit the worker
+				 * instead of preparing for reuse. Now that tablesync workers
+				 * can be reused and process_syncing_tables_for_sync is not
+				 * responsible for exiting. We need to take care of memory
+				 * contexts here before moving to sync the nex table or exit.
+				 */
+				if (MyLogicalRepWorker->ready_to_reuse)
+				{
+					MemoryContextResetAndDeleteChildren(ApplyMessageContext);
+					MemoryContextSwitchTo(TopMemoryContext);
+
+					/* Pop the error context stack */
+					error_context_stack = errcallback.previous;
+					apply_error_context_stack = error_context_stack;
+					return;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -3636,6 +3658,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+			if (MyLogicalRepWorker->ready_to_reuse)
+			{
+				endofstream = true;
+			}
 		}
 
 		/* Cleanup the memory. */
@@ -3734,12 +3760,15 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 		}
 	}
 
-	/* Pop the error context stack */
-	error_context_stack = errcallback.previous;
-	apply_error_context_stack = error_context_stack;
-
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * If it's still not ready to reuse, this is probably an apply worker.
+	 * End streaming before exiting.
+	 */
+	if (!MyLogicalRepWorker->ready_to_reuse)
+	{
+		/* All done */
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	}
 }
 
 /*
@@ -3865,6 +3894,50 @@ apply_worker_exit(void)
 	proc_exit(0);
 }
 
+/*
+ * Prepares the synchronization worker for reuse or exit.
+ */
+static void
+clean_sync_worker(void)
+{
+	/*
+	 * Commit any outstanding transaction. This is the usual case, unless
+	 * there was nothing to do for the table.
+	 */
+	if (IsTransactionState())
+	{
+		CommitTransactionCommand();
+		pgstat_report_stat(true);
+	}
+
+	/*
+	 * Disconnect from publisher. Otherwise reused sync workers causes
+	 * exceeding max_wal_senders
+	 */
+	walrcv_disconnect(LogRepWorkerWalRcvConn);
+	LogRepWorkerWalRcvConn = NULL;
+
+	/* Find the leader apply worker and signal it. */
+	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
+}
+
+/*
+ * Exit routine for synchronization worker.
+ */
+void
+pg_attribute_noreturn()
+sync_worker_exit(void)
+{
+	clean_sync_worker();
+
+	/* And flush all writes. */
+	XLogFlush(GetXLogWriteRecPtr());
+
+	/* Stop gracefully */
+	proc_exit(0);
+}
+
+
 /*
  * Reread subscription info if needed. Most changes will be exit.
  */
@@ -4324,6 +4397,56 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+ /* stream_build_options
+  * 	Build logical replication streaming options.
+  *
+  * This function sets streaming options including replication slot name
+  * and origin start position. Workers need these options for logical replication.
+  */
+static void
+stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4436,13 +4559,154 @@ start_apply(XLogRecPtr origin_startpos)
 }
 
 /*
- * Common initialization for leader apply worker and parallel apply worker.
+ * Runs the tablesync worker.
+ * It starts table sync. After successful sync,
+ * builds streaming options and starts streaming.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Set this to false for safety, in case we're already reusing the worker */
+	MyLogicalRepWorker->ready_to_reuse = false;
+
+	start_table_sync(origin_startpos, &slotname);
+
+	/*
+	 * Allocate the origin name in long-lived context for error context
+	 * message.
+	 */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   originname_size);
+	CommitTransactionCommand();
+
+	set_apply_error_context_origin(originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/* Start normal logical streaming replication. */
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	/* Start applying changes to catcup. */
+	start_apply(*origin_startpos);
+}
+
+/*
+ * Runs the apply worker.
+ * It sets up replication origin, the streaming options
+ * and then starts streaming.
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
+{
+	/* This is the leader apply worker */
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+	bool		must_use_password;
+
+	slotname = MySubscription->slotname;
+
+	/*
+	 * This shouldn't happen if the subscription is enabled, but guard
+	 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+	 * crash if slot is NULL.)
+	 */
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+	CommitTransactionCommand();
+
+	/* Is the use of a password mandatory? */
+	must_use_password = MySubscription->passwordrequired &&
+		!superuser_arg(MySubscription->owner);
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											must_use_password,
+											MySubscription->name, &err);
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+	 * We don't really use the output identify_system for anything but it
+	 * does some initializations on the upstream so let's still call it.
+	 */
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+	{
+		/* Start streaming with two_phase enabled */
+		options->proto.logical.twophase = true;
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
+	}
+	else
+	{
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+	}
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+
+	/* Run the main loop. */
+	start_apply(*origin_startpos);
+}
+
+/*
+ * Common initialization for logical replication workers; leader apply worker,
+ * parallel apply worker and tablesync worker.
  *
  * Initialize the database connection, in-memory subscription and necessary
  * config options.
  */
 void
-InitializeApplyWorker(void)
+InitializeLogRepWorker(void)
 {
 	MemoryContext oldctx;
 
@@ -4506,9 +4770,10 @@ InitializeApplyWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has started",
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", relation \"%s\" with relid %u has started",
 						MySubscription->name,
-						get_rel_name(MyLogicalRepWorker->relid))));
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
 	else
 		ereport(LOG,
 		/* translator: first %s is the name of logical replication worker */
@@ -4527,7 +4792,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	InitializingApplyWorker = true;
 
@@ -4551,7 +4815,7 @@ ApplyWorkerMain(Datum main_arg)
 	/* Load the libpq-specific functions */
 	load_file("libpqwalreceiver", false);
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
@@ -4559,165 +4823,142 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
+	/*
+	 * Setup callback for syscache so that we know when something changes in
+	 * the subscription relation state. Do this outside the loop to avoid
+	 * exceeding MAX_SYSCACHE_CALLBACKS
+	 */
+	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
+								  invalidate_syncing_table_states,
+								  (Datum) 0);
 
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-		bool		must_use_password;
+	/* This is leader apply worker */
+	run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 
-		myslotname = MySubscription->slotname;
+	proc_exit(0);
+}
 
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
+/* Logical Replication Tablesync worker entry point */
+void
+TablesyncWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *myslotname = NULL;
+	WalRcvStreamOptions options;
+	List	   *rstates;
+	SubscriptionRelState *rstate;
+	ListCell   *lc;
 
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-
-		/* Is the use of a password mandatory? */
-		must_use_password = MySubscription->passwordrequired &&
-			!superuser_arg(MySubscription->owner);
-
-		/* Note that the superuser_arg call can access the DB */
-		CommitTransactionCommand();
+	elog(LOG, "logical replication table synchronization worker has started");
 
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												must_use_password,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
+	/* Attach to slot */
+	logicalrep_worker_attach(worker_slot);
 
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+	/* Setup signal handling */
+	pqsignal(SIGHUP, SignalHandlerForConfigReload);
+	pqsignal(SIGTERM, die);
+	BackgroundWorkerUnblockSignals();
 
-		set_apply_error_context_origin(originname);
-	}
+	/*
+	 * We don't currently need any ResourceOwner in a walreceiver process, but
+	 * if we did, we could call CreateAuxProcessResourceOwner here.
+	 */
+
+	/* Initialise stats to a sanish value */
+	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
+		MyLogicalRepWorker->reply_time = GetCurrentTimestamp();
+
+	/* Load the libpq-specific functions */
+	load_file("libpqwalreceiver", false);
+
+	InitializeLogRepWorker();
+
+	/* Connect to the origin and start the replication. */
+	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
+		 MySubscription->conninfo);
 
 	/*
 	 * Setup callback for syscache so that we know when something changes in
-	 * the subscription relation state.
+	 * the subscription relation state. Do this outside the loop to avoid
+	 * exceeding MAX_SYSCACHE_CALLBACKS
 	 */
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
-
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
-
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-
 	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 * The loop where worker does its job. It loops until there is no relation
+	 * left to sync.
 	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
+	for (;;)
 	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
+		run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
+		if (IsTransactionState())
+			CommitTransactionCommand();
+
+		if (MyLogicalRepWorker->ready_to_reuse)
+		{
+		/* This transaction will be committed by clean_sync_worker. */
+		StartTransactionCommand();
 
-	if (!am_tablesync_worker())
-	{
 		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
+		 * Check if any table whose relation state is still INIT. If a table
+		 * in INIT state is found, the worker will not be finished, it will be
+		 * reused instead.
 		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
+		rstates = GetSubscriptionRelations(MySubscription->oid, true);
+		rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
+
+		foreach(lc, rstates)
 		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+			memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
 
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
+			/*
+			 * Pick the table for the next run if it is not already picked up
+			 * by another worker.
+			 *
+			 * Take exclusive lock to prevent any other sync worker from picking
+			 * the same table.
+			 */
+			LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
+			if (rstate->state != SUBREL_STATE_SYNCDONE &&
+				!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+			{
+				/* Update worker state for the next table */
+				MyLogicalRepWorker->relid = rstate->relid;
+				MyLogicalRepWorker->relstate = rstate->state;
+				MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+				LWLockRelease(LogicalRepWorkerLock);
+				break;
+			}
+			LWLockRelease(LogicalRepWorkerLock);
 		}
+
+		/*
+		 * If a relation with INIT state is assigned, clean up the worker for
+		 * the next iteration.
+		 *
+		 * If there is no more work left for this worker, break the loop to
+		 * exit.
+		 */
+		if ( MyLogicalRepWorker->relstate == SUBREL_STATE_INIT)
+			clean_sync_worker();
 		else
-		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-		}
+			break;
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-						MySubscription->name,
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-						MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-						"?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+		/* If not exited yet, then the worker will sync another table. */
+		StartTransactionCommand();
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\" has moved to sync table \"%s\" with relid %u.",
+						MySubscription->name, get_rel_name(MyLogicalRepWorker->relid), MyLogicalRepWorker->relid)));
+		CommitTransactionCommand();
+		}
 	}
 
-	/* Run the main loop. */
-	start_apply(origin_startpos);
-
-	proc_exit(0);
+	sync_worker_exit();
 }
 
 /*
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 39588da79f..bbd71d0b42 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -18,6 +18,7 @@ extern PGDLLIMPORT volatile sig_atomic_t ParallelApplyMessagePending;
 
 extern void ApplyWorkerMain(Datum main_arg);
 extern void ParallelApplyWorkerMain(Datum main_arg);
+extern void TablesyncWorkerMain(Datum main_arg);
 
 extern bool IsLogicalWorker(void);
 extern bool IsLogicalParallelApplyWorker(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index b57eed052f..412195f143 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -76,6 +76,12 @@ typedef struct LogicalRepWorker
 	/* Indicates whether apply can be performed in parallel. */
 	bool		parallel_apply;
 
+	/*
+	 * Used to indicate whether sync worker is ready for being reused
+	 * to sync another relation.
+	 */
+	bool		ready_to_reuse;
+
 	/* Stats. */
 	XLogRecPtr	last_lsn;
 	TimestampTz last_send_time;
@@ -265,7 +271,7 @@ extern void maybe_reread_subscription(void);
 
 extern void stream_cleanup_files(Oid subid, TransactionId xid);
 
-extern void InitializeApplyWorker(void);
+extern void InitializeLogRepWorker(void);
 
 extern void store_flush_position(XLogRecPtr remote_lsn, XLogRecPtr local_lsn);
 
@@ -273,6 +279,8 @@ extern void store_flush_position(XLogRecPtr remote_lsn, XLogRecPtr local_lsn);
 extern void apply_error_callback(void *arg);
 extern void set_apply_error_context_origin(char *originname);
 
+extern void sync_worker_exit(void);
+
 /* Parallel apply worker setup and interactions */
 extern void pa_allocate_worker(TransactionId xid);
 extern ParallelApplyWorkerInfo *pa_find_worker(TransactionId xid);
-- 
2.25.1

v9-0002-Add-replication-protocol-cmd-to-create-a-snapshot.patchapplication/octet-stream; name=v9-0002-Add-replication-protocol-cmd-to-create-a-snapshot.patchDownload
From 9655559b3ebcc6fc87174764a648c858d2053f22 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 13 Oct 2022 17:05:45 +0300
Subject: [PATCH 2/3] Add replication protocol cmd to create a snapshot

Introduced CREATE_REPLICATION_SNAPSHOT to be able to create and use a
snapshot without creating a new replication slot, but by using an
existing slot.

CREATE_REPLICATION_SNAPSHOT simply does what CREATE_REPLICATION_SLOT does
without creating a new replication slot.

CREATE_REPLICATION_SNAPSHOT command imports the snapshot into the current
transaction and returns consistent_point. The changes earlier than the
consistent_point will be applied by importing the snapshot. All changes
later than the consistent_point will be available to be consumed from
the replication slot.

This is useful for reusing replication slots in logical replication.
Otherwise, tablesync workers cannot start from a consistent point to copy
a relation and then apply changes by consuming from replication slot.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/protocol.sgml                    |  31 ++++++
 .../libpqwalreceiver/libpqwalreceiver.c       |  69 +++++++++++-
 src/backend/replication/logical/logical.c     |  40 ++++++-
 .../replication/logical/logicalfuncs.c        |   1 +
 src/backend/replication/repl_gram.y           |  18 ++-
 src/backend/replication/repl_scanner.l        |   2 +
 src/backend/replication/slotfuncs.c           |   1 +
 src/backend/replication/walsender.c           | 104 +++++++++++++++++-
 src/include/nodes/replnodes.h                 |  11 ++
 src/include/replication/logical.h             |   1 +
 src/include/replication/walreceiver.h         |  13 +++
 src/tools/pgindent/typedefs.list              |   2 +
 12 files changed, 289 insertions(+), 4 deletions(-)

diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index b11d9a6ba3..f7d5acee93 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2595,6 +2595,37 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
      </listitem>
     </varlistentry>
 
+    <varlistentry id="protocol-replication-replication-slot-snapshot">
+     <term><literal>CREATE_REPLICATION_SNAPSHOT</literal> <replaceable class="parameter">slot_name</replaceable> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
+      <indexterm><primary>CREATE_REPLICATION_SNAPSHOT</primary></indexterm>
+     </term>
+     <listitem>
+      <para>
+       Creates a snapshot including all the changes from the replication slot until
+       the point at which the replication slot becomes consistent. Then the snapshot
+       is used in the current transaction. This command is currently only supported
+       for logical replication slots.
+      </para>
+
+      <para>
+       In response to this command, the server will return a one-row result set,
+       containing the following field:
+       <variablelist>
+        <varlistentry>
+         <term><literal>consistent_point</literal> (<type>text</type>)</term>
+         <listitem>
+          <para>
+           The WAL location at which the slot became consistent.  This is the
+           earliest location from which streaming can start on this replication
+           slot.
+          </para>
+         </listitem>
+        </varlistentry>
+       </variablelist>
+      </para>
+     </listitem>
+    </varlistentry>
+
     <varlistentry id="protocol-replication-base-backup" xreflabel="BASE_BACKUP">
      <term><literal>BASE_BACKUP</literal> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
       <indexterm><primary>BASE_BACKUP</primary></indexterm>
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index 052505e46f..33075016a8 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -82,6 +82,8 @@ static WalRcvExecResult *libpqrcv_exec(WalReceiverConn *conn,
 									   const int nRetTypes,
 									   const Oid *retTypes);
 static void libpqrcv_disconnect(WalReceiverConn *conn);
+static void libpqrcv_slot_snapshot(WalReceiverConn *conn, char *slotname,
+								   const WalRcvStreamOptions *options, XLogRecPtr *lsn);
 
 static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_connect = libpqrcv_connect,
@@ -98,7 +100,8 @@ static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_create_slot = libpqrcv_create_slot,
 	.walrcv_get_backend_pid = libpqrcv_get_backend_pid,
 	.walrcv_exec = libpqrcv_exec,
-	.walrcv_disconnect = libpqrcv_disconnect
+	.walrcv_disconnect = libpqrcv_disconnect,
+	.walrcv_slot_snapshot = libpqrcv_slot_snapshot
 };
 
 /* Prototypes for private functions */
@@ -982,6 +985,70 @@ libpqrcv_create_slot(WalReceiverConn *conn, const char *slotname,
 	return snapshot;
 }
 
+/*
+ * TODO
+ */
+static void
+libpqrcv_slot_snapshot(WalReceiverConn *conn,
+					   char *slotname,
+					   const WalRcvStreamOptions *options,
+					   XLogRecPtr *lsn)
+{
+	StringInfoData cmd;
+	PGresult   *res;
+	char	   *pubnames_str;
+	List	   *pubnames;
+	char	   *pubnames_literal;
+
+	initStringInfo(&cmd);
+
+	/* Build the command. */
+	appendStringInfo(&cmd, "CREATE_REPLICATION_SNAPSHOT \"%s\"", slotname);
+	appendStringInfoString(&cmd, " (");
+	appendStringInfo(&cmd, " proto_version '%u'",
+					 options->proto.logical.proto_version);
+
+	/* Add publication names. */
+	pubnames = options->proto.logical.publication_names;
+	pubnames_str = stringlist_to_identifierstr(conn->streamConn, pubnames);
+	if (!pubnames_str)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	pubnames_literal = PQescapeLiteral(conn->streamConn, pubnames_str,
+									   strlen(pubnames_str));
+	if (!pubnames_literal)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	appendStringInfo(&cmd, ", publication_names %s", pubnames_literal);
+	PQfreemem(pubnames_literal);
+	pfree(pubnames_str);
+
+	appendStringInfoString(&cmd, " )");
+
+	/* Execute the command. */
+	res = libpqrcv_PQexec(conn->streamConn, cmd.data);
+	pfree(cmd.data);
+
+	if (PQresultStatus(res) != PGRES_TUPLES_OK)
+	{
+		PQclear(res);
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg("Could not create a snapshot by replication slot \"%s\": %s",
+						slotname, pchomp(PQerrorMessage(conn->streamConn)))));
+	}
+
+	if (lsn)
+		*lsn = DatumGetLSN(DirectFunctionCall1Coll(pg_lsn_in, InvalidOid,
+												   CStringGetDatum(PQgetvalue(res, 0, 0))));
+
+	PQclear(res);
+}
+
 /*
  * Return PID of remote backend process.
  */
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 7e1f677f7a..4623717013 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -476,6 +476,10 @@ CreateInitDecodingContext(const char *plugin,
  * fast_forward
  *		bypass the generation of logical changes.
  *
+ * need_full_snapshot
+ * 		if true, must obtain a snapshot able to read all tables;
+ *  	if false, one that can read only catalogs is acceptable.
+ *
  * xl_routine
  *		XLogReaderRoutine used by underlying xlogreader
  *
@@ -494,6 +498,7 @@ LogicalDecodingContext *
 CreateDecodingContext(XLogRecPtr start_lsn,
 					  List *output_plugin_options,
 					  bool fast_forward,
+					  bool need_full_snapshot,
 					  XLogReaderRoutine *xl_routine,
 					  LogicalOutputPluginWriterPrepareWrite prepare_write,
 					  LogicalOutputPluginWriterWrite do_write,
@@ -502,6 +507,7 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 	LogicalDecodingContext *ctx;
 	ReplicationSlot *slot;
 	MemoryContext old_context;
+	TransactionId xmin_horizon = InvalidTransactionId;
 
 	/* shorter lines... */
 	slot = MyReplicationSlot;
@@ -571,8 +577,40 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 		start_lsn = slot->data.confirmed_flush;
 	}
 
+
+	/*
+	 * We need to determine a safe xmin horizon to start decoding from if we
+	 * want to create a snapshot too. Otherwise we would end up with a
+	 * snapshot that cannot be imported since xmin value from the snapshot may
+	 * be less than the oldest safe xmin. To avoid this call
+	 * GetOldestSafeDecodingTransactionId() to return a safe xmin value, which
+	 * can be used while exporting/importing the snapshot.
+	 *
+	 * So we have to acquire the ProcArrayLock to prevent computation of new
+	 * xmin horizons by other backends, get the safe decoding xid, and inform
+	 * the slot machinery about the new limit. Once that's done the
+	 * ProcArrayLock can be released as the slot machinery now is protecting
+	 * against vacuum.
+	 */
+	if (need_full_snapshot)
+	{
+		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+		xmin_horizon = GetOldestSafeDecodingTransactionId(!need_full_snapshot);
+
+		SpinLockAcquire(&slot->mutex);
+		slot->effective_catalog_xmin = xmin_horizon;
+		slot->data.catalog_xmin = xmin_horizon;
+		slot->effective_xmin = xmin_horizon;
+		SpinLockRelease(&slot->mutex);
+
+		ReplicationSlotsComputeRequiredXmin(true);
+
+		LWLockRelease(ProcArrayLock);
+	}
+
 	ctx = StartupDecodingContext(output_plugin_options,
-								 start_lsn, InvalidTransactionId, false,
+								 start_lsn, xmin_horizon, need_full_snapshot,
 								 fast_forward, xl_routine, prepare_write,
 								 do_write, update_progress);
 
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index 55a24c02c9..85c5cdb633 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -208,6 +208,7 @@ pg_logical_slot_get_changes_guts(FunctionCallInfo fcinfo, bool confirm, bool bin
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									options,
 									false,
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/repl_gram.y b/src/backend/replication/repl_gram.y
index 0c874e33cf..957a5cc653 100644
--- a/src/backend/replication/repl_gram.y
+++ b/src/backend/replication/repl_gram.y
@@ -65,6 +65,7 @@ Node *replication_parse_result;
 %token K_CREATE_REPLICATION_SLOT
 %token K_DROP_REPLICATION_SLOT
 %token K_TIMELINE_HISTORY
+%token K_CREATE_REPLICATION_SNAPSHOT
 %token K_WAIT
 %token K_TIMELINE
 %token K_PHYSICAL
@@ -80,7 +81,7 @@ Node *replication_parse_result;
 %type <node>	command
 %type <node>	base_backup start_replication start_logical_replication
 				create_replication_slot drop_replication_slot identify_system
-				read_replication_slot timeline_history show
+				read_replication_slot timeline_history show create_replication_snapshot
 %type <list>	generic_option_list
 %type <defelt>	generic_option
 %type <uintval>	opt_timeline
@@ -114,6 +115,7 @@ command:
 			| read_replication_slot
 			| timeline_history
 			| show
+			| create_replication_snapshot
 			;
 
 /*
@@ -307,6 +309,19 @@ timeline_history:
 				}
 			;
 
+/*
+ * CREATE_REPLICATION_SNAPSHOT %s options
+ */
+create_replication_snapshot:
+			K_CREATE_REPLICATION_SNAPSHOT var_name plugin_options
+				{
+					CreateReplicationSnapshotCmd *n = makeNode(CreateReplicationSnapshotCmd);
+					n->slotname = $2;
+					n->options = $3;
+					$$ = (Node *) n;
+				}
+			;
+
 opt_physical:
 			K_PHYSICAL
 			| /* EMPTY */
@@ -400,6 +415,7 @@ ident_or_keyword:
 			| K_CREATE_REPLICATION_SLOT	{ $$ = "create_replication_slot"; }
 			| K_DROP_REPLICATION_SLOT		{ $$ = "drop_replication_slot"; }
 			| K_TIMELINE_HISTORY			{ $$ = "timeline_history"; }
+			| K_CREATE_REPLICATION_SNAPSHOT	{ $$ = "create_replication_snapshot"; }
 			| K_WAIT						{ $$ = "wait"; }
 			| K_TIMELINE					{ $$ = "timeline"; }
 			| K_PHYSICAL					{ $$ = "physical"; }
diff --git a/src/backend/replication/repl_scanner.l b/src/backend/replication/repl_scanner.l
index cb467ca46f..5ba2e9c54b 100644
--- a/src/backend/replication/repl_scanner.l
+++ b/src/backend/replication/repl_scanner.l
@@ -126,6 +126,7 @@ START_REPLICATION	{ return K_START_REPLICATION; }
 CREATE_REPLICATION_SLOT		{ return K_CREATE_REPLICATION_SLOT; }
 DROP_REPLICATION_SLOT		{ return K_DROP_REPLICATION_SLOT; }
 TIMELINE_HISTORY	{ return K_TIMELINE_HISTORY; }
+CREATE_REPLICATION_SNAPSHOT	{ return K_CREATE_REPLICATION_SNAPSHOT; }
 PHYSICAL			{ return K_PHYSICAL; }
 RESERVE_WAL			{ return K_RESERVE_WAL; }
 LOGICAL				{ return K_LOGICAL; }
@@ -303,6 +304,7 @@ replication_scanner_is_replication_command(void)
 		case K_DROP_REPLICATION_SLOT:
 		case K_READ_REPLICATION_SLOT:
 		case K_TIMELINE_HISTORY:
+		case K_CREATE_REPLICATION_SNAPSHOT:
 		case K_SHOW:
 			/* Yes; push back the first token so we can parse later. */
 			repl_pushed_back_token = first_token;
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 6035cf4816..c5b2d5b61f 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -486,6 +486,7 @@ pg_logical_replication_slot_advance(XLogRecPtr moveto)
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									NIL,
 									true,	/* fast_forward */
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 45b8b3684f..d335c88eda 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -238,6 +238,7 @@ static void CreateReplicationSlot(CreateReplicationSlotCmd *cmd);
 static void DropReplicationSlot(DropReplicationSlotCmd *cmd);
 static void StartReplication(StartReplicationCmd *cmd);
 static void StartLogicalReplication(StartReplicationCmd *cmd);
+static void CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd);
 static void ProcessStandbyMessage(void);
 static void ProcessStandbyReplyMessage(void);
 static void ProcessStandbyHSFeedbackMessage(void);
@@ -1284,7 +1285,7 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	 * are reported early.
 	 */
 	logical_decoding_ctx =
-		CreateDecodingContext(cmd->startpoint, cmd->options, false,
+		CreateDecodingContext(cmd->startpoint, cmd->options, false, false,
 							  XL_ROUTINE(.page_read = logical_read_xlog_page,
 										 .segment_open = WalSndSegmentOpen,
 										 .segment_close = wal_segment_close),
@@ -1336,6 +1337,98 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	EndCommand(&qc, DestRemote, false);
 }
 
+/*
+ * Create a snapshot from an existing replication slot.
+ */
+static void
+CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd)
+{
+	Snapshot	snap;
+	LogicalDecodingContext *ctx;
+	char		xloc[MAXFNAMELEN];
+	DestReceiver *dest;
+	TupOutputState *tstate;
+	TupleDesc	tupdesc;
+	Datum		values[1];
+	bool		nulls[1] = {0};
+
+	Assert(!MyReplicationSlot);
+
+	CheckLogicalDecodingRequirements();
+
+	if (!IsTransactionBlock())
+		ereport(ERROR,
+				(errmsg("%s must be called inside a transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (XactIsoLevel != XACT_REPEATABLE_READ)
+		ereport(ERROR,
+				(errmsg("%s must be called in REPEATABLE READ isolation mode transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (!XactReadOnly)
+		ereport(ERROR,
+				(errmsg("%s must be called in a read only transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (FirstSnapshotSet)
+		ereport(ERROR,
+				(errmsg("%s must be called before any query",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (IsSubTransaction())
+		ereport(ERROR,
+				(errmsg("%s must not be called in a subtransaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	ReplicationSlotAcquire(cmd->slotname, false);
+
+	ctx = CreateDecodingContext(MyReplicationSlot->data.restart_lsn,
+								cmd->options,
+								false,
+								true,
+								XL_ROUTINE(.page_read = logical_read_xlog_page,
+										   .segment_open = WalSndSegmentOpen,
+										   .segment_close = wal_segment_close),
+								WalSndPrepareWrite, WalSndWriteData,
+								WalSndUpdateProgress);
+
+	/*
+	 * Signal that we don't need the timeout mechanism. We're just creating
+	 * the snapshot with the replication slot and don't yet accept feedback
+	 * messages or send keepalives. As we possibly need to wait for further
+	 * WAL the walsender would otherwise possibly be killed too soon.
+	 */
+	last_reply_timestamp = 0;
+
+	/* build initial snapshot, might take a while */
+	DecodingContextFindStartpoint(ctx);
+
+	snap = SnapBuildInitialSnapshot(ctx->snapshot_builder);
+	RestoreTransactionSnapshot(snap, MyProc);
+
+	/* Don't need the decoding context anymore */
+	FreeDecodingContext(ctx);
+
+	/* Create a tuple to send consistent WAL location */
+	snprintf(xloc, sizeof(xloc), "%X/%X",
+			 LSN_FORMAT_ARGS(MyReplicationSlot->data.confirmed_flush));
+
+	dest = CreateDestReceiver(DestRemoteSimple);
+	tupdesc = CreateTemplateTupleDesc(1);
+	TupleDescInitBuiltinEntry(tupdesc, (AttrNumber) 1, "consistent_point",
+							  TEXTOID, -1, 0);
+	tstate = begin_tup_output_tupdesc(dest, tupdesc, &TTSOpsVirtual);
+
+	/* consistent wal location */
+	values[0] = CStringGetTextDatum(xloc);
+
+	do_tup_output(tstate, values, nulls);
+	end_tup_output(tstate);
+
+	ReplicationSlotRelease();
+}
+
 /*
  * LogicalDecodingContext 'prepare_write' callback.
  *
@@ -1864,6 +1957,15 @@ exec_replication_command(const char *cmd_string)
 			}
 			break;
 
+		case T_CreateReplicationSnapshotCmd:
+			{
+				cmdtag = "CREATE_REPLICATION_SNAPSHOT";
+				set_ps_display(cmdtag);
+				CreateReplicationSnapshot((CreateReplicationSnapshotCmd *) cmd_node);
+				EndReplicationCommand(cmdtag);
+				break;
+			}
+
 		default:
 			elog(ERROR, "unrecognized replication command node tag: %u",
 				 cmd_node->type);
diff --git a/src/include/nodes/replnodes.h b/src/include/nodes/replnodes.h
index 4321ba8f86..154ab74f33 100644
--- a/src/include/nodes/replnodes.h
+++ b/src/include/nodes/replnodes.h
@@ -108,4 +108,15 @@ typedef struct TimeLineHistoryCmd
 	TimeLineID	timeline;
 } TimeLineHistoryCmd;
 
+/* ----------------------
+ *		CREATE_REPLICATION_SNAPSHOT command
+ * ----------------------
+ */
+typedef struct CreateReplicationSnapshotCmd
+{
+	NodeTag		type;
+	char	   *slotname;
+	List	   *options;
+} CreateReplicationSnapshotCmd;
+
 #endif							/* REPLNODES_H */
diff --git a/src/include/replication/logical.h b/src/include/replication/logical.h
index 5f49554ea0..6535786a0e 100644
--- a/src/include/replication/logical.h
+++ b/src/include/replication/logical.h
@@ -125,6 +125,7 @@ extern LogicalDecodingContext *CreateInitDecodingContext(const char *plugin,
 extern LogicalDecodingContext *CreateDecodingContext(XLogRecPtr start_lsn,
 													 List *output_plugin_options,
 													 bool fast_forward,
+													 bool need_full_snapshot,
 													 XLogReaderRoutine *xl_routine,
 													 LogicalOutputPluginWriterPrepareWrite prepare_write,
 													 LogicalOutputPluginWriterWrite do_write,
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 281626fa6f..f01e5f23a7 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -386,6 +386,16 @@ typedef WalRcvExecResult *(*walrcv_exec_fn) (WalReceiverConn *conn,
  */
 typedef void (*walrcv_disconnect_fn) (WalReceiverConn *conn);
 
+/*
+ * walrcv_slot_snapshot_fn
+ *
+ * Create a snapshot by an existing replication slot
+ */
+typedef void (*walrcv_slot_snapshot_fn) (WalReceiverConn *conn,
+										 char *slotname,
+										 const WalRcvStreamOptions *options,
+										 XLogRecPtr *lsn);
+
 typedef struct WalReceiverFunctionsType
 {
 	walrcv_connect_fn walrcv_connect;
@@ -403,6 +413,7 @@ typedef struct WalReceiverFunctionsType
 	walrcv_get_backend_pid_fn walrcv_get_backend_pid;
 	walrcv_exec_fn walrcv_exec;
 	walrcv_disconnect_fn walrcv_disconnect;
+	walrcv_slot_snapshot_fn walrcv_slot_snapshot;
 } WalReceiverFunctionsType;
 
 extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
@@ -437,6 +448,8 @@ extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
 	WalReceiverFunctions->walrcv_exec(conn, exec, nRetTypes, retTypes)
 #define walrcv_disconnect(conn) \
 	WalReceiverFunctions->walrcv_disconnect(conn)
+#define walrcv_slot_snapshot(conn, slotname, options, lsn) \
+	WalReceiverFunctions->walrcv_slot_snapshot(conn, slotname, options, lsn)
 
 static inline void
 walrcv_clear_result(WalRcvExecResult *walres)
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index b4058b88c3..80ace35868 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2343,6 +2343,7 @@ ReplicationSlotInvalidationCause
 ReplicationSlotOnDisk
 ReplicationSlotPersistency
 ReplicationSlotPersistentData
+CreateReplicationSnapshotCmd
 ReplicationState
 ReplicationStateCtl
 ReplicationStateOnDisk
@@ -3818,6 +3819,7 @@ walrcv_receive_fn
 walrcv_send_fn
 walrcv_server_version_fn
 walrcv_startstreaming_fn
+walrcv_slot_snapshot_fn
 wchar2mb_with_len_converter
 wchar_t
 win32_deadchild_waitinfo
-- 
2.25.1

v12-0003-Reuse-Replication-Slot-and-Origin-in-Tablesync.patchapplication/octet-stream; name=v12-0003-Reuse-Replication-Slot-and-Origin-in-Tablesync.patchDownload
From d5dde676d3ff4e1398609ef8937cf1604b70c5c6 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH 3/3] Reuse Replication Slot and Origin in Tablesync

This commit allows reusing replication slots and origins during tablesync.

Earlier, a tablesync worker was creating a new replication slot and origin
each time it syncs a new table. With this patch, replication
slots/origins can be reusable for tablesync.

This reduces the overhead of creating/dropping replication slots and origins
and improves tablesync speed significantly especially for empty or small tables.

If the state of the current table is INIT or DATASYNC, tablesync worker needs a
repliation slot/origin. If the worker has not created slot and origin in
its previous runs, it will create those first. Otherwise the worker reuses
slot and origin created by the same worker in previous iterations earlier.
Tables in FINISHEDCOPY are expected to have a replication slot and origin.
Slot and origin names for such tables are persisted in
pg_subscription_rel catalog. Tablesync worker can fetch them and proceed
with existing slot and origin of FINISHEDCOPY tables and does not need to
create new ones.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/catalogs.sgml                    |  31 ++
 src/backend/catalog/pg_subscription.c         | 247 ++++++++++-
 src/backend/commands/subscriptioncmds.c       | 229 +++++++----
 .../replication/logical/applyparallelworker.c |   3 +-
 src/backend/replication/logical/launcher.c    |   9 +-
 src/backend/replication/logical/tablesync.c   | 382 +++++++++++++-----
 src/backend/replication/logical/worker.c      | 142 ++++---
 src/include/catalog/pg_subscription.h         |   6 +
 src/include/catalog/pg_subscription_rel.h     |  14 +-
 src/include/replication/slot.h                |   3 +-
 src/include/replication/worker_internal.h     |  23 +-
 11 files changed, 852 insertions(+), 237 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 5240840552..0a33897a37 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8040,6 +8040,19 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        origin.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>sublastusedid</structfield> <type>int8</type>
+      </para>
+      <para>
+      The last used ID for tablesync workers. It acts as an unique identifier
+      for replication slots which are created by tablesync workers.
+      The last used ID needs to be persisted to make logical replication safely
+      proceed after any interruption. If sublastusedid is 0, then no table has
+      been synced yet.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
@@ -8124,6 +8137,24 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        otherwise null
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srrelslotname</structfield> <type>name</type>
+      </para>
+      <para>
+       Replication slot name that is used for synchronization of relation
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srreloriginname</structfield> <type>name</type>
+      </para>
+      <para>
+       Origin name that is used for tracking synchronization of relation
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index d07f88ce28..152fdaa310 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -108,6 +108,14 @@ GetSubscription(Oid subid, bool missing_ok)
 								   Anum_pg_subscription_suborigin);
 	sub->origin = TextDatumGetCString(datum);
 
+	/* Get last used id */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
+							tup,
+							Anum_pg_subscription_sublastusedid,
+							&isnull);
+	Assert(!isnull);
+	sub->lastusedid = DatumGetInt64(datum);
+
 	ReleaseSysCache(tup);
 
 	return sub;
@@ -199,6 +207,44 @@ DisableSubscription(Oid subid)
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update the last used replication slot ID for the given subscription.
+ */
+void
+UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid)
+{
+	Relation	rel;
+	bool		nulls[Natts_pg_subscription];
+	bool		replaces[Natts_pg_subscription];
+	Datum		values[Natts_pg_subscription];
+	HeapTuple	tup;
+
+	/* Look up the subscription in the catalog */
+	rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+	tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	/* Form a new tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	replaces[Anum_pg_subscription_sublastusedid - 1] = true;
+	values[Anum_pg_subscription_sublastusedid- 1] = Int64GetDatum(lastusedid);
+
+	/* Update the catalog */
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+	heap_freetuple(tup);
+
+	table_close(rel, NoLock);
+}
+
 /*
  * Convert text array to list of strings.
  *
@@ -228,7 +274,7 @@ textarray_to_stringlist(ArrayType *textarray)
  */
 void
 AddSubscriptionRelState(Oid subid, Oid relid, char state,
-						XLogRecPtr sublsn)
+						XLogRecPtr sublsn, char *relslotname, char *reloriginname)
 {
 	Relation	rel;
 	HeapTuple	tup;
@@ -257,6 +303,16 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
 	else
 		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -269,6 +325,60 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Internal function to modify columns for relation state update
+ */
+static void
+UpdateSubscriptionRelState_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char state,
+									XLogRecPtr sublsn)
+{
+	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
+	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+
+	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (sublsn != InvalidXLogRecPtr)
+		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
+	else
+		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication slot update
+ */
+static void
+UpdateSubscriptionRelReplicationSlot_internal(Datum *values,
+											bool *nulls,
+											bool *replaces,
+											char *relslotname)
+{
+	replaces[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication origin update
+ */
+static void
+UpdateSubscriptionRelOrigin_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char *reloriginname)
+{
+	replaces[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+}
+
 /*
  * Update the state of a subscription table.
  */
@@ -299,14 +409,56 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	memset(nulls, false, sizeof(nulls));
 	memset(replaces, false, sizeof(replaces));
 
-	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
-	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
 
-	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
-	if (sublsn != InvalidXLogRecPtr)
-		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
-	else
-		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update replication slot name, origin name and state of
+ * a subscription table in one transaction.
+ */
+void
+UpdateSubscriptionRel(Oid subid,
+					  Oid relid,
+					  char state,
+					  XLogRecPtr sublsn,
+					  char *relslotname,
+					  char *reloriginname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
+	UpdateSubscriptionRelOrigin_internal(values, nulls, replaces, reloriginname);
 
 	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
 							replaces);
@@ -318,6 +470,85 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Get origin name of subscription table.
+ *
+ * reloriginname's value has the replication origin name if the origin exists.
+ */
+void
+GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*originname;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srreloriginname, isnull);
+	if (!*isnull)
+	{
+		originname = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(reloriginname, originname, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
+/*
+ * Get replication slot name of subscription table.
+ *
+ * slotname's value has the replication slot name if the subscription has any.
+ */
+void
+GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*relrepslot;
+	bool		isnull;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srrelslotname, &isnull);
+	if (!isnull)
+	{
+		relrepslot = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(slotname, relrepslot, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
 /*
  * Get state of subscription table.
  *
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 8b3de032ee..7941f6281e 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -708,6 +708,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 		publicationListToArray(publications);
 	values[Anum_pg_subscription_suborigin - 1] =
 		CStringGetTextDatum(opts.origin);
+	values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(0);
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -771,7 +772,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 										 rv->schemaname, rv->relname);
 
 				AddSubscriptionRelState(subid, relid, table_state,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 			}
 
 			/*
@@ -862,6 +863,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
 	bool		must_use_password;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -941,7 +944,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				AddSubscriptionRelState(sub->oid, relid,
 										copy_data ? SUBREL_STATE_INIT : SUBREL_STATE_READY,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 				ereport(DEBUG1,
 						(errmsg_internal("table \"%s.%s\" added to subscription \"%s\"",
 										 rv->schemaname, rv->relname, sub->name)));
@@ -965,6 +968,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				char		state;
 				XLogRecPtr	statelsn;
+				char		slotname[NAMEDATALEN] = {0};
 
 				/*
 				 * Lock pg_subscription_rel with AccessExclusiveLock to
@@ -991,13 +995,36 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/*
+				 * Find the logical replication sync worker. If exists, store
+				 * the slot number for dropping associated replication slots
+				 * later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot_name);
+				}
+				else
+				{
+					/*
+					 * Sync of this relation might be failed in an earlier
+					 * attempt, but the replication slot might still exist.
+					 */
+					GetSubscriptionRelReplicationSlot(sub->oid, relid, slotname);
+					if (strlen(slotname) > 0)
+						sub_remove_slots = lappend(sub_remove_slots, slotname);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
 				 * tablesync origin.
 				 */
-				if (state != SUBREL_STATE_READY)
+				if (state != SUBREL_STATE_READY &&
+					state != SUBREL_STATE_SYNCDONE)
 				{
 					char		originname[NAMEDATALEN];
 
@@ -1025,31 +1052,24 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
-		 * the database operations we won't be able to rollback dropped slots.
+		 * Drop the replication slots associated with tablesync workers for
+		 * removed tables. This has to be at the end because otherwise if
+		 * there is an error while doing the database operations we won't be
+		 * able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			memcpy(syncslotname, lfirst(lc), sizeof(NAMEDATALEN));
+
+			/*
+			 * There is no certainty, maybe the slot does not exist yet. Also,
+			 * if we fail after removing some of the slots, next time, it will
+			 * again try to drop already dropped slots and fail. For these
+			 * reasons, we allow missing_ok = true for the drop.
+			 */
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1472,6 +1492,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	char	   *subname;
 	char	   *conninfo;
 	char	   *slotname;
+	int64		lastusedid;
 	List	   *subworkers;
 	ListCell   *lc;
 	char		originname[NAMEDATALEN];
@@ -1544,6 +1565,14 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	else
 		slotname = NULL;
 
+	/* Get the last used identifier by the subscription */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID, tup,
+							Anum_pg_subscription_sublastusedid, &isnull);
+	if (!isnull)
+		lastusedid = DatumGetInt64(datum);
+	else
+		lastusedid = 0;
+
 	/*
 	 * Since dropping a replication slot is not transactional, the replication
 	 * slot stays dropped even if the transaction rolls back.  So we cannot
@@ -1593,6 +1622,8 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	}
 	list_free(subworkers);
 
+	rstates = GetSubscriptionRelations(subid, true);
+
 	/*
 	 * Remove the no-longer-useful entry in the launcher's table of apply
 	 * worker start times.
@@ -1604,36 +1635,26 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	ApplyLauncherForgetWorkerStartTime(subid);
 
 	/*
-	 * Cleanup of tablesync replication origins.
-	 *
-	 * Any READY-state relations would already have dealt with clean-ups.
+	 * Cleanup of tablesync replication origins associated with the
+	 * subscription, if exists. Try to drop origins by creating all origin
+	 * names created for this subscription.
 	 *
 	 * Note that the state can't change because we have already stopped both
 	 * the apply and tablesync workers and they can't restart because of
 	 * exclusive lock on the subscription.
+	 *
+	 * XXX: This can be handled better instead of looping through all possible
 	 */
-	rstates = GetSubscriptionRelations(subid, true);
-	foreach(lc, rstates)
+	for (int64 i = 1; i <= lastusedid; i++)
 	{
-		SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-		Oid			relid = rstate->relid;
-
-		/* Only cleanup resources of tablesync workers */
-		if (!OidIsValid(relid))
-			continue;
+		char		originname_to_drop[NAMEDATALEN] = {0};
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * It is possible that the origin is not yet created for tablesync
-		 * worker so passing missing_ok = true. This can happen for the states
-		 * before SUBREL_STATE_FINISHEDCOPY.
-		 */
-		ReplicationOriginNameForLogicalRep(subid, relid, originname,
-										   sizeof(originname));
-		replorigin_drop_by_name(originname, true, false);
+		snprintf(originname_to_drop, sizeof(originname_to_drop), "pg_%u_%lld", subid, (long long) i);
+		/* missing_ok = true, since the origin might be already dropped. */
+		replorigin_drop_by_name(originname_to_drop, true, false);
 	}
 
+
 	/* Clean up dependencies */
 	deleteSharedDependencyRecordsFor(SubscriptionRelationId, subid, 0);
 
@@ -1686,39 +1707,17 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
+		List	   *slots = NULL;
 
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
 
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char	   *syncslotname = (char *) lfirst(lc);
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1741,6 +1740,71 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * Get the replication slot names associated with the subscription.
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok)
+{
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+					 " FROM pg_replication_slots"
+					 " WHERE slot_name LIKE 'pg_%i_sync_%%';",
+					 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+					errmsg("could not receive list of slots associated with the subscription %u, error: %s",
+					subid, res->err));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();
+		return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -2153,6 +2217,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -2163,18 +2228,20 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
-			char		syncslotname[NAMEDATALEN] = {0};
-
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
-				 syncslotname);
+				 worker->slot_name);
 		}
 	}
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 50aa1386d5..ffa27d7111 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -440,7 +440,8 @@ pa_launch_parallel_worker(void)
 										MySubscription->name,
 										MyLogicalRepWorker->userid,
 										InvalidOid,
-										dsm_segment_handle(winfo->dsm_seg));
+										dsm_segment_handle(winfo->dsm_seg),
+										InvalidRepSlotId);
 
 	if (launched)
 	{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 3e89366e7c..e040249d93 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -304,7 +304,7 @@ logicalrep_workers_find(Oid subid, bool only_running)
  */
 bool
 logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
-						 Oid relid, dsm_handle subworker_dsm)
+						 Oid relid, dsm_handle subworker_dsm, int64 slotid)
 {
 	BackgroundWorker bgw;
 	BackgroundWorkerHandle *bgw_handle;
@@ -430,6 +430,9 @@ retry:
 	worker->launch_time = now;
 	worker->in_use = true;
 	worker->generation++;
+	worker->created_slot = false;
+	worker->rep_slot_id = slotid;
+	worker->slot_name = (char *) palloc(NAMEDATALEN);
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -437,6 +440,7 @@ retry:
 	worker->relid = relid;
 	worker->relstate = SUBREL_STATE_UNKNOWN;
 	worker->relstate_lsn = InvalidXLogRecPtr;
+	worker->ready_to_reuse = false;
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
@@ -1161,7 +1165,8 @@ ApplyLauncherMain(Datum main_arg)
 				ApplyLauncherSetWorkerStartTime(sub->oid, now);
 				logicalrep_worker_launch(sub->dbid, sub->oid, sub->name,
 										 sub->owner, InvalidOid,
-										 DSM_HANDLE_INVALID);
+										 DSM_HANDLE_INVALID,
+										 InvalidRepSlotId);
 			}
 			else
 			{
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 0a813bc371..3d51e732bd 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -275,40 +275,29 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 								   MyLogicalRepWorker->relid,
 								   MyLogicalRepWorker->relstate,
 								   MyLogicalRepWorker->relstate_lsn);
+		CommitTransactionCommand();
 
 		/*
-		 * End streaming so that LogRepWorkerWalRcvConn can be used to drop
-		 * the slot.
-		 */
-		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
-
-		/*
-		 * Cleanup the tablesync slot.
+		 * Cleanup the tablesync slot. If the slot name used by this worker is
+		 * different from the default slot name for the worker, this means the
+		 * current table had started to being synchronized by another worker
+		 * and replication slot. And this worker is reusing a replication slot
+		 * from a previous attempt. We do not need that replication slot
+		 * anymore.
 		 *
 		 * This has to be done after updating the state because otherwise if
 		 * there is an error while doing the database operations we won't be
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
+										MyLogicalRepWorker->rep_slot_id,
 										syncslotname,
 										sizeof(syncslotname));
 
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
-		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
-
-		CommitTransactionCommand();
-		pgstat_report_stat(false);
-
-		/*
-		 * Start a new transaction to clean up the tablesync origin tracking.
-		 * This transaction will be ended within the finish_sync_worker().
-		 * Now, even, if we fail to remove this here, the apply worker will
-		 * ensure to clean it up afterward.
+		 * We are safe to drop the replication tracking origin after this
+		 * point. Now, even, if we fail to remove this here, the apply worker
+		 * will ensure to clean it up afterward.
 		 *
 		 * We need to do this after the table state is set to SYNCDONE.
 		 * Otherwise, if an error occurs while performing the database
@@ -317,32 +306,78 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 * have been cleared before restart. So, the restarted worker will use
 		 * invalid replication progress state resulting in replay of
 		 * transactions that have already been applied.
+		 *
+		 * Firstly reset the origin session to remove the ownership of the
+		 * slot. This is needed to allow the origin to be dropped or reused
+		 * later.
 		 */
+		replorigin_session_reset();
+		replorigin_session_origin = InvalidRepOriginId;
+		replorigin_session_origin_lsn = InvalidXLogRecPtr;
+		replorigin_session_origin_timestamp = 0;
+
 		StartTransactionCommand();
+		if (MyLogicalRepWorker->slot_name && strcmp(syncslotname, MyLogicalRepWorker->slot_name) != 0)
+		{
+			/*
+			 * End streaming so that LogRepWorkerWalRcvConn can be used to
+			 * drop the slot.
+			 */
+			walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
 
-		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
+			ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+											   MyLogicalRepWorker->relid,
+											   originname,
+											   sizeof(originname));
+
+			/*
+			 * Drop replication origin
+			 *
+			 * There is a chance that the user is concurrently performing refresh
+			 * for the subscription where we remove the table state and its origin
+			 * or the apply worker would have removed this origin. So passing
+			 * missing_ok = true.
+			 */
+			replorigin_drop_by_name(originname, true, false);
+		}
 
 		/*
-		 * Resetting the origin session removes the ownership of the slot.
-		 * This is needed to allow the origin to be dropped.
+		 * We are safe to remove persisted replication slot and origin data,
+		 * since it's already in SYNCDONE state. They will not be needed
+		 * anymore.
 		 */
-		replorigin_session_reset();
-		replorigin_session_origin = InvalidRepOriginId;
-		replorigin_session_origin_lsn = InvalidXLogRecPtr;
-		replorigin_session_origin_timestamp = 0;
+		UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+							  MyLogicalRepWorker->relid,
+							  MyLogicalRepWorker->relstate,
+							  MyLogicalRepWorker->relstate_lsn,
+							  NULL,
+							  NULL);
+		ereport(DEBUG2,
+			(errmsg("process_syncing_tables_for_sync: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					"NULL", "NULL", MyLogicalRepWorker->relstate,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", relation \"%s\" with relid %u has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
+
+		CommitTransactionCommand();
+		pgstat_report_stat(false);
+
+		StartTransactionCommand();
 
 		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * There is a chance that the user is concurrently performing refresh
-		 * for the subscription where we remove the table state and its origin
-		 * or the apply worker would have removed this origin. So passing
-		 * missing_ok = true.
+		 * This should return the default origin name for the worker. Even if
+		 * the worker used a different origin for this table, it should be
+		 * dropped and removed from the catalog so far.
 		 */
-		replorigin_drop_by_name(originname, true, false);
+		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+										   MyLogicalRepWorker->relid,
+										   originname,
+										   sizeof(originname));
 
 		/*
 		 * Sync worker is cleaned at this point. It's ready to sync next table,
@@ -352,8 +387,8 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		MyLogicalRepWorker->ready_to_reuse = true;
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 	}
-
-	SpinLockRelease(&MyLogicalRepWorker->relmutex);
+	else
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 }
 
 /*
@@ -437,6 +472,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 			if (current_lsn >= rstate->lsn)
 			{
 				char		originname[NAMEDATALEN];
+				bool		is_origin_null = true;
 
 				rstate->state = SUBREL_STATE_READY;
 				rstate->lsn = current_lsn;
@@ -457,18 +493,31 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				 * error while dropping we won't restart it to drop the
 				 * origin. So passing missing_ok = true.
 				 */
-				ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-												   rstate->relid,
-												   originname,
-												   sizeof(originname));
-				replorigin_drop_by_name(originname, true, false);
+				GetSubscriptionRelOrigin(MyLogicalRepWorker->subid,
+										 rstate->relid, originname,
+										 &is_origin_null);
+
+				if (!is_origin_null)
+				{
+					replorigin_drop_by_name(originname, true, false);
+				}
 
 				/*
 				 * Update the state to READY only after the origin cleanup.
 				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-										   rstate->relid, rstate->state,
-										   rstate->lsn);
+				UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+									  rstate->relid,
+									  rstate->state,
+									  rstate->lsn,
+									  NULL,
+									  NULL);
+				ereport(DEBUG2,
+					(errmsg("process_syncing_tables_for_apply: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+							"NULL", "NULL", rstate->state,
+							rstate->relid, MyLogicalRepWorker->subid)));
+
+				CommitTransactionCommand();
+				started_tx = false;
 			}
 		}
 		else
@@ -557,12 +606,25 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						TimestampDifferenceExceeds(hentry->last_start_time, now,
 												   wal_retrieve_retry_interval))
 					{
+						if (IsTransactionState())
+							CommitTransactionCommand();
+						StartTransactionCommand();
+						started_tx = true;
+
+						MySubscription->lastusedid++;
+						UpdateSubscriptionLastSlotId(MyLogicalRepWorker->subid,
+													 MySubscription->lastusedid);
+						ereport(DEBUG2,
+								(errmsg("process_syncing_tables_for_apply: incremented lastusedid to %lld for subscription %u",
+										(long long) MySubscription->lastusedid, MySubscription->oid)));
+
 						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
 												 MySubscription->oid,
 												 MySubscription->name,
 												 MyLogicalRepWorker->userid,
 												 rstate->relid,
-												 DSM_HANDLE_INVALID);
+												 DSM_HANDLE_INVALID,
+												 MySubscription->lastusedid);
 						hentry->last_start_time = now;
 					}
 				}
@@ -1185,8 +1247,8 @@ copy_table(Relation rel)
  * The name must not exceed NAMEDATALEN - 1 because of remote node constraints
  * on slot name length. We append system_identifier to avoid slot_name
  * collision with subscriptions in other clusters. With the current scheme
- * pg_%u_sync_%u_UINT64_FORMAT (3 + 10 + 6 + 10 + 20 + '\0'), the maximum
- * length of slot_name will be 50.
+ * pg_%u_sync_%lu_UINT64_FORMAT (3 + 10 + 6 + 20 + 20 + '\0'), the maximum
+ * length of slot_name will be 45.
  *
  * The returned slot name is stored in the supplied buffer (syncslotname) with
  * the given size.
@@ -1197,11 +1259,11 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
+ReplicationSlotNameForTablesync(Oid suboid, int64 slotid,
 								char *syncslotname, Size szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%lld_" UINT64_FORMAT, suboid,
+			(long long) slotid, GetSystemIdentifier());
 }
 
 /*
@@ -1225,6 +1287,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	char		originname[NAMEDATALEN];
 	RepOriginId originid;
 	bool		must_use_password;
+	char	   *prev_slotname;
 
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
@@ -1253,7 +1316,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
+									MyLogicalRepWorker->rep_slot_id,
 									slotname,
 									NAMEDATALEN);
 
@@ -1279,12 +1342,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC ||
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY);
 
+	/*
+	 * See if tablesync of the current relation has been started with another
+	 * replication slot.
+	 *
+	 * Read previous slot name from the catalog, if exists.
+	 */
+	prev_slotname = (char *) palloc(NAMEDATALEN);
+	StartTransactionCommand();
+	GetSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+									  MyLogicalRepWorker->relid,
+									  prev_slotname);
+
 	/* Assign the origin tracking record name. */
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   sizeof(originname));
 
+	CommitTransactionCommand();
+
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC)
 	{
 		/*
@@ -1298,10 +1375,53 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		 * breakdown then it wouldn't have succeeded so trying it next time
 		 * seems like a better bet.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, slotname, true);
+		if (strlen(prev_slotname) > 0)
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);
+
+			StartTransactionCommand();
+			/* Replication origin might still exist. Try to drop */
+			replorigin_drop_by_name(originname, true, false);
+
+			/*
+			 * Remove replication slot and origin name from the relation's
+			 * catalog record
+			 */
+			UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+								  MyLogicalRepWorker->relid,
+								  MyLogicalRepWorker->relstate,
+								  MyLogicalRepWorker->relstate_lsn,
+								  NULL,
+								  NULL);
+			CommitTransactionCommand();
+			ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+						"NULL", "NULL", MyLogicalRepWorker->relstate,
+						MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+		}
 	}
 	else if (MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY)
 	{
+		/*
+		 * At this point, the table that is currently being synchronized
+		 * should have its replication slot name filled in the catalog. The
+		 * tablesync process was started with another sync worker and
+		 * replication slot. We need to continue using the same replication
+		 * slot in this worker too.
+		 */
+		if (strlen(prev_slotname) == 0)
+		{
+			elog(ERROR, "Replication slot could not be found for subscription %u, relation %u",
+				 MyLogicalRepWorker->subid,
+				 MyLogicalRepWorker->relid);
+		}
+
+		/*
+		 * Proceed with the correct replication slot. Use previously created
+		 * replication slot to sync this table.
+		 */
+		memcpy(slotname, prev_slotname, NAMEDATALEN);
+
 		/*
 		 * The COPY phase was previously done, but tablesync then crashed
 		 * before it was able to finish normally.
@@ -1321,7 +1441,9 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 		goto copy_table_done;
 	}
+	pfree(prev_slotname);
 
+	/* Preparing for table copy operation */
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 	MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
 	MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
@@ -1329,11 +1451,31 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   MyLogicalRepWorker->relstate,
-							   MyLogicalRepWorker->relstate_lsn);
+
+	/*
+	 * Refresh the originname in case of having non-existing origin
+	 * from previous failed sync attempts.
+	 * If that's the case, it should be removed from the catalog so far.
+	 * Then, we can continue by reusing the origin created by the current
+	 * worker instead of .
+	 */
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									MyLogicalRepWorker->relid,
+									originname,
+									sizeof(originname));
+
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  MyLogicalRepWorker->relstate,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					slotname, originname, MyLogicalRepWorker->relstate,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+
 	pgstat_report_stat(true);
 
 	StartTransactionCommand();
@@ -1371,6 +1513,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						GetUserNameFromId(GetUserId(), true),
 						RelationGetRelationName(rel))));
 
+
 	/*
 	 * Start a transaction in the remote node in REPEATABLE READ mode.  This
 	 * ensures that both the replication slot we create (see below) and the
@@ -1386,48 +1529,95 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						res->err)));
 	walrcv_clear_result(res);
 
+	originid = replorigin_by_name(originname, true);
+
 	/*
 	 * Create a new permanent logical decoding slot. This slot will be used
 	 * for the catchup phase after COPY is done, so tell it to use the
 	 * snapshot to make the final data consistent.
+	 *
+	 * Replication slot will only be created if either this is the first run
+	 * of the worker or we're not using a previous replication slot.
 	 */
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-
-	/*
-	 * Setup replication origin tracking. The purpose of doing this before the
-	 * copy is to avoid doing the copy again due to any error in setting up
-	 * origin tracking.
-	 */
-	originid = replorigin_by_name(originname, true);
-	if (!OidIsValid(originid))
+	if (!MyLogicalRepWorker->created_slot)
 	{
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						   slotname, false /* permanent */ , false /* two_phase */ ,
+						   CRS_USE_SNAPSHOT, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: created replication slot %s for subscription %u",
+						slotname, MyLogicalRepWorker->subid)));
+
 		/*
-		 * Origin tracking does not exist, so create it now.
-		 *
-		 * Then advance to the LSN got from walrcv_create_slot. This is WAL
-		 * logged for the purpose of recovery. Locks are to prevent the
-		 * replication origin from vanishing while advancing.
+		 * Remember that we created the slot so that we will not try to create
+		 * it again.
 		 */
-		originid = replorigin_create(originname);
-
-		LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
-		replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
-						   true /* go backward */ , true /* WAL log */ );
-		UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->created_slot = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
+		/*
+		 * Setup replication origin tracking. The purpose of doing this before
+		 * the copy is to avoid doing the copy again due to any error in
+		 * setting up origin tracking.
+		 */
+		if (!OidIsValid(originid))
+		{
+			/*
+			 * Origin tracking does not exist, so create it now.
+			 */
+			originid = replorigin_create(originname);
+		}
+		else
+		{
+			/*
+			 * At this point, there shouldn't be any existing replication
+			 * origin with the same name.
+			 */
+			ereport(ERROR,
+					(errcode(ERRCODE_DUPLICATE_OBJECT),
+					 errmsg("replication origin \"%s\" already exists",
+							originname)));
+		}
 	}
 	else
 	{
-		ereport(ERROR,
-				(errcode(ERRCODE_DUPLICATE_OBJECT),
-				 errmsg("replication origin \"%s\" already exists",
-						originname)));
+		/*
+		 * Do not create a new replication slot, reuse the existing one
+		 * instead. Use a new snapshot for the replication slot to ensure that
+		 * tablesync and apply proceses are consistent with each other.
+		 */
+		WalRcvStreamOptions options;
+		int			server_version;
+
+		server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+		options.proto.logical.proto_version =
+			server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+			server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+			LOGICALREP_PROTO_VERSION_NUM;
+		options.proto.logical.publication_names = MySubscription->publications;
+
+		walrcv_slot_snapshot(LogRepWorkerWalRcvConn, slotname, &options, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: reusing replication slot %s for relation %u in subscription %u",
+						slotname, MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
 	}
 
+	/*
+	 * Advance to the LSN got from walrcv_create_slot or walrcv_slot_snapshot.
+	 * This is WAL logged for the purpose of recovery. Locks are to prevent
+	 * the replication origin from vanishing while advancing.
+	 *
+	 * Then setup replication origin tracking.
+	 */
+	LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+	replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
+					   true /* go backward */ , true /* WAL log */ );
+	UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+
 	/* Now do the initial data copy */
 	PushActiveSnapshot(GetTransactionSnapshot());
 	copy_table(rel);
@@ -1450,12 +1640,18 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	 * Update the persisted state to indicate the COPY phase is done; make it
 	 * visible to others.
 	 */
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   SUBREL_STATE_FINISHEDCOPY,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  SUBREL_STATE_FINISHEDCOPY,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					originname, slotname, SUBREL_STATE_FINISHEDCOPY,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
 
 copy_table_done:
 
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index d5f87b84cb..69748d4656 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -464,8 +464,16 @@ ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 {
 	if (OidIsValid(relid))
 	{
-		/* Replication origin name for tablesync workers. */
-		snprintf(originname, szoriginname, "pg_%u_%u", suboid, relid);
+		bool		is_null = true;
+
+		/*
+		 * Replication origin name for tablesync workers. First, look into the
+		 * catalog. If originname does not exist, then use the default name.
+		 */
+		GetSubscriptionRelOrigin(suboid, relid,
+								 originname, &is_null);
+		if (is_null)
+			snprintf(originname, szoriginname, "pg_%u_%lld", suboid, (long long) MyLogicalRepWorker->rep_slot_id);
 	}
 	else
 	{
@@ -3760,6 +3768,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 		}
 	}
 
+	/* Pop the error context stack */
+	error_context_stack = errcallback.previous;
+	apply_error_context_stack = error_context_stack;
+
 	/*
 	 * If it's still not ready to reuse, this is probably an apply worker.
 	 * End streaming before exiting.
@@ -4521,6 +4533,9 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 
 	/* allocate slot name in long-lived context */
 	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+
+	/* Keep the replication slot name used for this sync. */
+	MyLogicalRepWorker->slot_name = *myslotname;
 	pfree(syncslotname);
 }
 
@@ -4637,11 +4652,13 @@ run_apply_worker(WalRcvStreamOptions *options,
 	replorigin_session_setup(originid, 0);
 	replorigin_session_origin = originid;
 	*origin_startpos = replorigin_session_get_progress(false);
-	CommitTransactionCommand();
 
 	/* Is the use of a password mandatory? */
-	must_use_password = MySubscription->passwordrequired &&
-		!superuser_arg(MySubscription->owner);
+		must_use_password = MySubscription->passwordrequired &&
+			!superuser_arg(MySubscription->owner);
+
+	CommitTransactionCommand();
+
 	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
 											must_use_password,
 											MySubscription->name, &err);
@@ -4901,60 +4918,87 @@ TablesyncWorkerMain(Datum main_arg)
 
 		if (MyLogicalRepWorker->ready_to_reuse)
 		{
-		/* This transaction will be committed by clean_sync_worker. */
-		StartTransactionCommand();
+			/* This transaction will be committed by clean_sync_worker. */
+			StartTransactionCommand();
 
-		/*
-		 * Check if any table whose relation state is still INIT. If a table
-		 * in INIT state is found, the worker will not be finished, it will be
-		 * reused instead.
-		 */
-		rstates = GetSubscriptionRelations(MySubscription->oid, true);
-		rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
+			/*
+			 * Check if any table whose relation state is still INIT. If a table
+			 * in INIT state is found, the worker will not be finished, it will be
+			 * reused instead.
+			 */
+			rstates = GetSubscriptionRelations(MySubscription->oid, true);
+			rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
 
-		foreach(lc, rstates)
-		{
-			memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
+			foreach(lc, rstates)
+			{
+				memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
+
+				/*
+				 * Pick the table for the next run if it is not already picked up
+				 * by another worker.
+				 *
+				 * Take exclusive lock to prevent any other sync worker from picking
+				 * the same table.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
+				if (rstate->state != SUBREL_STATE_SYNCDONE &&
+					!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+				{
+					/* Update worker state for the next table */
+					MyLogicalRepWorker->relid = rstate->relid;
+					MyLogicalRepWorker->relstate = rstate->state;
+					MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+					LWLockRelease(LogicalRepWorkerLock);
+					break;
+				}
+				LWLockRelease(LogicalRepWorkerLock);
+			}
 
 			/*
-			 * Pick the table for the next run if it is not already picked up
-			 * by another worker.
+			 * If a relation with INIT state is assigned, clean up the worker for
+			 * the next iteration.
 			 *
-			 * Take exclusive lock to prevent any other sync worker from picking
-			 * the same table.
+			 * If there is no more work left for this worker, drop repllication
+			 * slot and origin, then break the loop to exit.
 			 */
-			LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
-			if (rstate->state != SUBREL_STATE_SYNCDONE &&
-				!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+			if (MyLogicalRepWorker->relstate == SUBREL_STATE_INIT)
 			{
-				/* Update worker state for the next table */
-				MyLogicalRepWorker->relid = rstate->relid;
-				MyLogicalRepWorker->relstate = rstate->state;
-				MyLogicalRepWorker->relstate_lsn = rstate->lsn;
-				LWLockRelease(LogicalRepWorkerLock);
-				break;
+				clean_sync_worker();
 			}
-			LWLockRelease(LogicalRepWorkerLock);
-		}
+			else
+			{
+				TimeLineID	tli;
 
-		/*
-		 * If a relation with INIT state is assigned, clean up the worker for
-		 * the next iteration.
-		 *
-		 * If there is no more work left for this worker, break the loop to
-		 * exit.
-		 */
-		if ( MyLogicalRepWorker->relstate == SUBREL_STATE_INIT)
-			clean_sync_worker();
-		else
-			break;
+				/*
+				 * It is important to give an error if we are unable to drop the
+				 * slot, otherwise, it won't be dropped till the corresponding
+				 * subscription is dropped. So passing missing_ok = false.
+				 */
+				if (MyLogicalRepWorker->created_slot)
+				{
+					walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+					ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
+				}
 
-		/* If not exited yet, then the worker will sync another table. */
-		StartTransactionCommand();
-		ereport(LOG,
-				(errmsg("logical replication table synchronization worker for subscription \"%s\" has moved to sync table \"%s\" with relid %u.",
-						MySubscription->name, get_rel_name(MyLogicalRepWorker->relid), MyLogicalRepWorker->relid)));
-		CommitTransactionCommand();
+				/*
+				 * Drop replication origin before exiting.
+				 *
+				 * There is a chance that the user is concurrently performing refresh
+				 * for the subscription where we remove the table state and its origin
+				 * or the apply worker would have removed this origin. So passing
+				 * missing_ok = true.
+				 */
+				replorigin_drop_by_name(originname, true, false);
+
+				break;
+			}
+
+			/* If not exited yet, then the worker will sync another table. */
+			StartTransactionCommand();
+			ereport(LOG,
+					(errmsg("logical replication table synchronization worker for subscription \"%s\" has moved to sync table \"%s\" with relid %u.",
+							MySubscription->name, get_rel_name(MyLogicalRepWorker->relid), MyLogicalRepWorker->relid)));
+			CommitTransactionCommand();
 		}
 	}
 
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 91d729d62d..493055ea98 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -108,6 +108,9 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
 
 	/* Only publish data originating from the specified origin */
 	text		suborigin BKI_DEFAULT(LOGICALREP_ORIGIN_ANY);
+
+	/* The last used ID to create a replication slot for tablesync */
+	int64		sublastusedid BKI_DEFAULT(0);
 #endif
 } FormData_pg_subscription;
 
@@ -144,6 +147,8 @@ typedef struct Subscription
 	List	   *publications;	/* List of publication names to subscribe to */
 	char	   *origin;			/* Only publish data originating from the
 								 * specified origin */
+	int64		lastusedid;		/* Last used unique ID to create replication
+								 * slots in tablesync */
 } Subscription;
 
 /* Disallow streaming in-progress transactions. */
@@ -164,6 +169,7 @@ typedef struct Subscription
 extern Subscription *GetSubscription(Oid subid, bool missing_ok);
 extern void FreeSubscription(Subscription *sub);
 extern void DisableSubscription(Oid subid);
+extern void UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid);
 
 extern int	CountDBSubscriptions(Oid dbid);
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 60a2bcca23..185164d75e 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -44,6 +44,12 @@ CATALOG(pg_subscription_rel,6102,SubscriptionRelRelationId)
 											 * used for synchronization
 											 * coordination, or NULL if not
 											 * valid */
+	NameData	srrelslotname BKI_FORCE_NULL;	/* name of the replication
+												 * slot for relation in
+												 * subscription */
+	NameData	srreloriginname BKI_FORCE_NULL; /* origin name for relation in
+												 * subscription */
+
 #endif
 } FormData_pg_subscription_rel;
 
@@ -81,10 +87,16 @@ typedef struct SubscriptionRelState
 } SubscriptionRelState;
 
 extern void AddSubscriptionRelState(Oid subid, Oid relid, char state,
-									XLogRecPtr sublsn);
+									XLogRecPtr sublsn, char *relslotname, char *reloriginname);
 extern void UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 									   XLogRecPtr sublsn);
+extern void UpdateSubscriptionRel(Oid subid, Oid relid, char state,
+								  XLogRecPtr sublsn, char *relslotname, char *reloriginname);
+
 extern char GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn);
+extern void GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname);
+extern void GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull);
+
 extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 
 extern bool HasSubscriptionRelations(Oid subid);
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index a8a89dc784..31b2c41893 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -237,8 +237,9 @@ extern bool InvalidateObsoleteReplicationSlots(ReplicationSlotInvalidationCause
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
 extern bool ReplicationSlotName(int index, Name name);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, Size szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, int64 slotid, char *syncslotname, Size szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List *GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 412195f143..d6adb2fc39 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -35,6 +35,23 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/*
+	 * Indicates if the sync worker created a replication slot for itself
+	 * in any point of its lifetime.
+	 * False means that the worker has not created a slot yet, and has been
+	 * reusing replication slots created by other workers so far.
+	 */
+	bool		created_slot;
+
+	/*
+	 * Unique identifier for replication slot to be created by tablesnync
+	 * workers, if needed.
+	 */
+	int64		rep_slot_id;
+
+	/* Replication slot name used by the worker. */
+	char	   *slot_name;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
@@ -239,7 +256,8 @@ extern LogicalRepWorker *logicalrep_worker_find(Oid subid, Oid relid,
 extern List *logicalrep_workers_find(Oid subid, bool only_running);
 extern bool logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname,
 									 Oid userid, Oid relid,
-									 dsm_handle subworker_dsm);
+									 dsm_handle subworker_dsm,
+									 int64 slotid);
 extern void logicalrep_worker_stop(Oid subid, Oid relid);
 extern void logicalrep_pa_worker_stop(int slot_no, uint16 generation);
 extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
@@ -334,4 +352,7 @@ am_parallel_apply_worker(void)
 	return isParallelApplyWorker(MyLogicalRepWorker);
 }
 
+/* Invalid identifier to be used for naming replication slots */
+#define InvalidRepSlotId	0
+
 #endif							/* WORKER_INTERNAL_H */
-- 
2.25.1

#60Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#59)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi, and thanks for the patch! It is an interesting idea.

I have not yet fully read this thread, so below are only my first
impressions after looking at patch 0001. Sorry if some of these were
already discussed earlier.

TBH the patch "reuse-workers" logic seemed more complicated than I had
imagined it might be.

1.
IIUC with patch 0001, each/every tablesync worker (a.k.a. TSW) when it
finishes dealing with one table then goes looking to find if there is
some relation that it can process next. So now every TSW has a loop
where it will fight with every other available TSW over who will get
to process the next relation.

Somehow this seems all backwards to me. Isn't it strange for the TSW
to be the one deciding what relation it would deal with next?

IMO it seems more natural to simply return the finished TSW to some
kind of "pool" of available workers and the main Apply process can
just grab a TSW from that pool instead of launching a brand new one in
the existing function process_syncing_tables_for_apply(). Or, maybe
those "available" workers can be returned to a pool maintained within
the launcher.c code, which logicalrep_worker_launch() can draw from
instead of launching a whole new process?

(I need to read the earlier posts to see if these options were already
discussed and rejected)

~~

2.
AFAIK the thing that identifies a tablesync worker is the fact that
only TSW will have a 'relid'.

But it feels very awkward to me to have a TSW marked as "available"
and yet that LogicalRepWorker must still have some OLD relid field
value lurking (otherwise it will forget that it is a "tablesync"
worker!).

IMO perhaps it is time now to introduce some enum 'type' to the
LogicalRepWorker. Then an "in_use" type=TSW would have a valid 'relid'
whereas an "available" type=TSW would have relid == InvalidOid.

~~

3.
Maybe I am mistaken, but it seems the benchmark results posted are
only using quite a small/default values for
"max_sync_workers_per_subscription", so I wondered how those results
are affected by increasing that GUC. I think having only very few
workers would cause more sequential processing, so conveniently the
effect of the patch avoiding re-launch might be seen in the best
possible light. OTOH, using more TSW in the first place might reduce
the overall tablesync time because the subscriber can do more work in
parallel.

So I'm not quite sure what the goal is here. E.g. if the user doesn't
care much about how long tablesync phase takes then there is maybe no
need for this patch at all. OTOH, I thought if a user does care about
the subscription startup time, won't those users be opting for a much
larger "max_sync_workers_per_subscription" in the first place?
Therefore shouldn't the benchmarking be using a larger number too?

======

Here are a few other random things noticed while looking at patch 0001:

1. Commit message

1a. typo /sequantially/sequentially/

1b. Saying "killed" and "killing" seemed a bit extreme and implies
somebody else is killing the process. But I think mostly tablesync is
just ending by a normal proc exit, so maybe reword this a bit.

~~~

2. It seemed odd that some -- clearly tablesync specific -- functions
are in the worker.c instead of in tablesync.c.

2a. e.g. clean_sync_worker

2b. e.g. sync_worker_exit

~~~

3. process_syncing_tables_for_sync

+ /*
+ * Sync worker is cleaned at this point. It's ready to sync next table,
+ * if needed.
+ */
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->ready_to_reuse = true;
  SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ }
+
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);

Isn't there a double release of that mutex happening there?

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#61Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Peter Smith (#60)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi,

Peter Smith <smithpb2250@gmail.com>, 24 May 2023 Çar, 05:59 tarihinde şunu
yazdı:

Hi, and thanks for the patch! It is an interesting idea.

I have not yet fully read this thread, so below are only my first
impressions after looking at patch 0001. Sorry if some of these were
already discussed earlier.

TBH the patch "reuse-workers" logic seemed more complicated than I had
imagined it might be.

If you mean patch 0001 by the patch "reuse-workers", most of the complexity
comes with some refactoring to split apply worker and tablesync worker
paths. [1]/messages/by-id/CAAKRu_YKGyF+svRQqe1th-mG9xLdzneWgh9H1z1DtypBkawkkw@mail.gmail.com
If you mean the whole patch set, then I believe it's because reusing
replication slots also requires having a proper snapshot each time the
worker moves to a new table. [2]/messages/by-id/CAGPVpCRWEVhXa7ovrhuSQofx4to7o22oU9iKtrOgAOtz_=Y6vg@mail.gmail.com

1.
IIUC with patch 0001, each/every tablesync worker (a.k.a. TSW) when it
finishes dealing with one table then goes looking to find if there is
some relation that it can process next. So now every TSW has a loop
where it will fight with every other available TSW over who will get
to process the next relation.

Somehow this seems all backwards to me. Isn't it strange for the TSW
to be the one deciding what relation it would deal with next?

IMO it seems more natural to simply return the finished TSW to some
kind of "pool" of available workers and the main Apply process can
just grab a TSW from that pool instead of launching a brand new one in
the existing function process_syncing_tables_for_apply(). Or, maybe
those "available" workers can be returned to a pool maintained within
the launcher.c code, which logicalrep_worker_launch() can draw from
instead of launching a whole new process?

(I need to read the earlier posts to see if these options were already
discussed and rejected)

I think ([3]/messages/by-id/CAGPVpCRzD-ZZEc9ienhyrVpCzd9AJ7fxE--OFFJBnBg3E0438w@mail.gmail.com) relying on a single apply worker for the assignment of
several tablesync workers might bring some overhead, it's possible that
some tablesync workers wait in idle until the apply worker assigns them
something. OTOH yes, the current approach makes tablesync workers race for
a new table to sync.
TBF both ways might be worth discussing/investigating more, before deciding
which way to go.

2.
AFAIK the thing that identifies a tablesync worker is the fact that
only TSW will have a 'relid'.

But it feels very awkward to me to have a TSW marked as "available"
and yet that LogicalRepWorker must still have some OLD relid field
value lurking (otherwise it will forget that it is a "tablesync"
worker!).

IMO perhaps it is time now to introduce some enum 'type' to the
LogicalRepWorker. Then an "in_use" type=TSW would have a valid 'relid'
whereas an "available" type=TSW would have relid == InvalidOid.

Hmm, relid will be immediately updated when the worker moves to a new
table. And the time between finishing sync of a table and finding a new
table to sync should be minimal. I'm not sure how having an old relid for
such a small amount of time can do any harm.

3.
Maybe I am mistaken, but it seems the benchmark results posted are
only using quite a small/default values for
"max_sync_workers_per_subscription", so I wondered how those results
are affected by increasing that GUC. I think having only very few
workers would cause more sequential processing, so conveniently the
effect of the patch avoiding re-launch might be seen in the best
possible light. OTOH, using more TSW in the first place might reduce
the overall tablesync time because the subscriber can do more work in
parallel.

So I'm not quite sure what the goal is here. E.g. if the user doesn't

care much about how long tablesync phase takes then there is maybe no

need for this patch at all. OTOH, I thought if a user does care about
the subscription startup time, won't those users be opting for a much
larger "max_sync_workers_per_subscription" in the first place?
Therefore shouldn't the benchmarking be using a larger number too?

Regardless of how many tablesync workers there are, reusing workers will
speed things up if a worker has a chance to sync more than one table.
Increasing the number of tablesync workers, of course, improves the
tablesync performance. But if it doesn't make 100% parallel ( meaning that
# of sync workers != # of tables to sync), then reusing workers can bring
an additional improvement.

Here are some benchmarks similar to earlier, but with 100 tables and
different number of workers:

+--------+-------------+-------------+-------------+------------+
|        | 2 workers   | 4 workers   | 6 workers   | 8 workers  |
+--------+-------------+-------------+-------------+------------+
| master | 2579.154 ms | 1383.153 ms | 1001.559 ms | 911.758 ms |
+--------+-------------+-------------+-------------+------------+
| patch  | 1724.230 ms | 853.894 ms  | 601.176 ms  | 496.395 ms |
+--------+-------------+-------------+-------------+------------+

So yes, increasing the number of workers makes it faster. But reusing
workers can still improve more.

[1]: /messages/by-id/CAAKRu_YKGyF+svRQqe1th-mG9xLdzneWgh9H1z1DtypBkawkkw@mail.gmail.com
/messages/by-id/CAAKRu_YKGyF+svRQqe1th-mG9xLdzneWgh9H1z1DtypBkawkkw@mail.gmail.com
[2]: /messages/by-id/CAGPVpCRWEVhXa7ovrhuSQofx4to7o22oU9iKtrOgAOtz_=Y6vg@mail.gmail.com
/messages/by-id/CAGPVpCRWEVhXa7ovrhuSQofx4to7o22oU9iKtrOgAOtz_=Y6vg@mail.gmail.com
[3]: /messages/by-id/CAGPVpCRzD-ZZEc9ienhyrVpCzd9AJ7fxE--OFFJBnBg3E0438w@mail.gmail.com
/messages/by-id/CAGPVpCRzD-ZZEc9ienhyrVpCzd9AJ7fxE--OFFJBnBg3E0438w@mail.gmail.com

Best,
--
Melih Mutlu
Microsoft

#62Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#61)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Thu, May 25, 2023 at 6:59 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi,

Peter Smith <smithpb2250@gmail.com>, 24 May 2023 Çar, 05:59 tarihinde şunu yazdı:

Hi, and thanks for the patch! It is an interesting idea.

I have not yet fully read this thread, so below are only my first
impressions after looking at patch 0001. Sorry if some of these were
already discussed earlier.

TBH the patch "reuse-workers" logic seemed more complicated than I had
imagined it might be.

If you mean patch 0001 by the patch "reuse-workers", most of the complexity comes with some refactoring to split apply worker and tablesync worker paths. [1]
If you mean the whole patch set, then I believe it's because reusing replication slots also requires having a proper snapshot each time the worker moves to a new table. [2]

Yes, I was mostly referring to the same as point 1 below about patch
0001. I guess I just found the concept of mixing A) launching TSW (via
apply worker) with B) reassigning TSW to another relation (by the TSW
battling with its peers) to be a bit difficult to understand. I
thought most of the refactoring seemed to arise from choosing to do it
that way.

1.
IIUC with patch 0001, each/every tablesync worker (a.k.a. TSW) when it
finishes dealing with one table then goes looking to find if there is
some relation that it can process next. So now every TSW has a loop
where it will fight with every other available TSW over who will get
to process the next relation.

Somehow this seems all backwards to me. Isn't it strange for the TSW
to be the one deciding what relation it would deal with next?

IMO it seems more natural to simply return the finished TSW to some
kind of "pool" of available workers and the main Apply process can
just grab a TSW from that pool instead of launching a brand new one in
the existing function process_syncing_tables_for_apply(). Or, maybe
those "available" workers can be returned to a pool maintained within
the launcher.c code, which logicalrep_worker_launch() can draw from
instead of launching a whole new process?

(I need to read the earlier posts to see if these options were already
discussed and rejected)

I think ([3]) relying on a single apply worker for the assignment of several tablesync workers might bring some overhead, it's possible that some tablesync workers wait in idle until the apply worker assigns them something. OTOH yes, the current approach makes tablesync workers race for a new table to sync.

Yes, it might be slower than the 'patched' code because "available"
workers might be momentarily idle while they wait to be re-assigned to
the next relation. We would need to try it to find out.

TBF both ways might be worth discussing/investigating more, before deciding which way to go.

+1. I think it would be nice to see POC of both ways for benchmark
comparison because IMO performance is not the only consideration --
unless there is an obvious winner, then they need to be judged also by
the complexity of the logic, the amount of code that needed to be
refactored, etc.

2.
AFAIK the thing that identifies a tablesync worker is the fact that
only TSW will have a 'relid'.

But it feels very awkward to me to have a TSW marked as "available"
and yet that LogicalRepWorker must still have some OLD relid field
value lurking (otherwise it will forget that it is a "tablesync"
worker!).

IMO perhaps it is time now to introduce some enum 'type' to the
LogicalRepWorker. Then an "in_use" type=TSW would have a valid 'relid'
whereas an "available" type=TSW would have relid == InvalidOid.

Hmm, relid will be immediately updated when the worker moves to a new table. And the time between finishing sync of a table and finding a new table to sync should be minimal. I'm not sure how having an old relid for such a small amount of time can do any harm.

There is no "harm", but it just didn't feel right to make the
LogicalRepWorker to transition through some meaningless state
("available" for re-use but still assigned some relid), just because
it was easy to do it that way. I think it is more natural for the
'relid' to be valid only when it is valid for the worker and to be
InvalidOid when it is not valid. --- Maybe this gripe would become
more apparent if the implementation use the "free-list" idea because
then you would have a lot of bogus relids assigned to the workers of
that list for longer than just a moment.

3.
Maybe I am mistaken, but it seems the benchmark results posted are
only using quite a small/default values for
"max_sync_workers_per_subscription", so I wondered how those results
are affected by increasing that GUC. I think having only very few
workers would cause more sequential processing, so conveniently the
effect of the patch avoiding re-launch might be seen in the best
possible light. OTOH, using more TSW in the first place might reduce
the overall tablesync time because the subscriber can do more work in
parallel.

So I'm not quite sure what the goal is here. E.g. if the user doesn't

care much about how long tablesync phase takes then there is maybe no
need for this patch at all. OTOH, I thought if a user does care about
the subscription startup time, won't those users be opting for a much
larger "max_sync_workers_per_subscription" in the first place?
Therefore shouldn't the benchmarking be using a larger number too?

Regardless of how many tablesync workers there are, reusing workers will speed things up if a worker has a chance to sync more than one table. Increasing the number of tablesync workers, of course, improves the tablesync performance. But if it doesn't make 100% parallel ( meaning that # of sync workers != # of tables to sync), then reusing workers can bring an additional improvement.

Here are some benchmarks similar to earlier, but with 100 tables and different number of workers:

+--------+-------------+-------------+-------------+------------+
|        | 2 workers   | 4 workers   | 6 workers   | 8 workers  |
+--------+-------------+-------------+-------------+------------+
| master | 2579.154 ms | 1383.153 ms | 1001.559 ms | 911.758 ms |
+--------+-------------+-------------+-------------+------------+
| patch  | 1724.230 ms | 853.894 ms  | 601.176 ms  | 496.395 ms |
+--------+-------------+-------------+-------------+------------+

So yes, increasing the number of workers makes it faster. But reusing workers can still improve more.

Thanks for the benchmark results! There is no denying they seem pretty
good numbers.

But it is difficult to get an overall picture of the behaviour. Mostly
when benchmarks were posted you hold one variable fixed and show only
one other varying. It always leaves me wondering -- what about not
empty tables, or what about different numbers of tables etc. Is it
possible to make some script to gather a bigger set of results so we
can see everything at once? Perhaps then it will become clear there is
some "sweet spot" where the patch is really good but beyond that it
degrades (actually, who knows what it might show).

For example:

=== empty tables

workers:2 workers:4 workers:8 workers:16
tables:10 tables:10 tables:10 tables:10
data:0 data:0 data:0 data:0
master/patch master/patch master/patch master/patch

workers:2 workers:4 workers:8 workers:16
tables:100 tables:100 tables:100 tables:100
data:0 data:0 data:0 data:0
master/patch master/patch master/patch master/patch

workers:2 workers:4 workers:8 workers:16
tables:1000 tables:1000 tables:1000 tables:1000
data:0 data:0 data:0 data:0
master/patch master/patch master/patch master/patch

=== 1M data

workers:2 workers:4 workers:8 workers:16
tables:10 tables:10 tables:10 tables:10
data:1M data:1M data:1M data:1M
master/patch master/patch master/patch master/patch

workers:2 workers:4 workers:8 workers:16
tables:100 tables:100 tables:100 tables:100
data:1M data:1M data:1M data:1M
master/patch master/patch master/patch master/patch

workers:2 workers:4 workers:8 workers:16
tables:1000 tables:1000 tables:1000 tables:1000
data:1M data:1M data:1M data:1M
master/patch master/patch master/patch master/patch

=== 10M data

workers:2 workers:4 workers:8 workers:16
tables:10 tables:10 tables:10 tables:10
data:10M data:10M data:10M data:10M
master/patch master/patch master/patch master/patch

workers:2 workers:4 workers:8 workers:16
tables:100 tables:100 tables:100 tables:100
data:10M data:10M data:10M data:10M
master/patch master/patch master/patch master/patch

workers:2 workers:4 workers:8 workers:16
tables:1000 tables:1000 tables:1000 tables:1000
data:10M data:10M data:10M data:10M
master/patch master/patch master/patch master/patch

== 100M data

workers:2 workers:4 workers:8 workers:16
tables:10 tables:10 tables:10 tables:10
data:100M data:100M data:100M data:100M
master/patch master/patch master/patch master/patch

workers:2 workers:4 workers:8 workers:16
tables:100 tables:100 tables:100 tables:100
data:100M data:100M data:100M data:100M
master/patch master/patch master/patch master/patch

workers:2 workers:4 workers:8 workers:16
tables:1000 tables:1000 tables:1000 tables:1000
data:100M data:100M data:100M data:100M
master/patch master/patch master/patch master/patch

------
Kind Regards,
Peter Smith
Fujitsu Australia

#63Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Peter Smith (#60)
3 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi,

I rebased the patch and addressed the following reviews.

Peter Smith <smithpb2250@gmail.com>, 24 May 2023 Çar, 05:59 tarihinde
şunu yazdı:

Here are a few other random things noticed while looking at patch 0001:

1. Commit message

1a. typo /sequantially/sequentially/

1b. Saying "killed" and "killing" seemed a bit extreme and implies
somebody else is killing the process. But I think mostly tablesync is
just ending by a normal proc exit, so maybe reword this a bit.

Fixed the type and reworded a bit.

2. It seemed odd that some -- clearly tablesync specific -- functions
are in the worker.c instead of in tablesync.c.

2a. e.g. clean_sync_worker

2b. e.g. sync_worker_exit

Honestly, the distinction between worker.c and tablesync.c is not that
clear to me. Both seem like they're doing some work for tablesync and
apply.
But yes, you're right. Those functions fit better to tablesync.c. Moved them.

3. process_syncing_tables_for_sync

+ /*
+ * Sync worker is cleaned at this point. It's ready to sync next table,
+ * if needed.
+ */
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->ready_to_reuse = true;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ }
+
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);

Isn't there a double release of that mutex happening there?

Fixed.

Thanks,
--
Melih Mutlu
Microsoft

Attachments:

v2-0001-Reuse-Tablesync-Workers.patchapplication/octet-stream; name=v2-0001-Reuse-Tablesync-Workers.patchDownload
From 32fc88978366bf1dd3144ed714575daf8600ccaa Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 11 Apr 2023 14:24:22 +0300
Subject: [PATCH 1/3] Reuse Tablesync Workers

This commit allows reusing tablesync workers for syncing more than one
relation sequentially during their lifetime, instead of exiting after
only syncing one relation.

Before this commit, tablesync workers were capable of syncing only one
relation. For each table, a new sync worker was launched and the worker
would exit when the worker is done with the current table.

Now, tablesync workers are not only limited with one relation and can
move to another relation in the same subscription. This reduces the
overhead of launching a new background worker and exiting from that
worker for each relation.

A new tablesync worker gets launched only if the number of tablesync
workers for the subscription does not exceed
max_sync_workers_per_subscription. If there is a table needs to be synced,
a tablesync worker picks that up and syncs it.The worker continues to
picking new tables to sync until there is no table left for synchronization
in the subscription.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/postmaster/bgworker.c             |   3 +
 .../replication/logical/applyparallelworker.c |   2 +-
 src/backend/replication/logical/launcher.c    |   5 +-
 src/backend/replication/logical/tablesync.c   |  45 +-
 src/backend/replication/logical/worker.c      | 481 ++++++++++++------
 src/include/replication/logicalworker.h       |   1 +
 src/include/replication/worker_internal.h     |  11 +-
 7 files changed, 388 insertions(+), 160 deletions(-)

diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 0dd22b2351..5609919edf 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -131,6 +131,9 @@ static const struct
 	},
 	{
 		"ParallelApplyWorkerMain", ParallelApplyWorkerMain
+	},
+	{
+		"TablesyncWorkerMain", TablesyncWorkerMain
 	}
 };
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 82c1ddcdcb..f16e2377bf 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -942,7 +942,7 @@ ParallelApplyWorkerMain(Datum main_arg)
 	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
 		MyLogicalRepWorker->reply_time = 0;
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 87b5593d2d..86ee1f25c2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -440,6 +440,7 @@ retry:
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
+	worker->ready_to_reuse = false;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -460,8 +461,10 @@ retry:
 
 	if (is_parallel_apply_worker)
 		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ParallelApplyWorkerMain");
-	else
+	else if (!OidIsValid(relid))
 		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
+	else
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "TablesyncWorkerMain");
 
 	if (OidIsValid(relid))
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index c56d42dcd2..524be7088a 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -128,11 +128,10 @@ static bool FetchTableStates(bool *started_tx);
 static StringInfo copybuf = NULL;
 
 /*
- * Exit routine for synchronization worker.
+ * Prepares the synchronization worker for reuse or exit.
  */
-static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+void
+clean_sync_worker(void)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -144,18 +143,28 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
-	/* And flush all writes. */
-	XLogFlush(GetXLogWriteRecPtr());
-
-	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
-	CommitTransactionCommand();
+	/*
+	 * Disconnect from publisher. Otherwise reused sync workers causes
+	 * exceeding max_wal_senders
+	 */
+	walrcv_disconnect(LogRepWorkerWalRcvConn);
+	LogRepWorkerWalRcvConn = NULL;
 
 	/* Find the leader apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
+}
+
+/*
+ * Exit routine for synchronization worker.
+ */
+void
+pg_attribute_noreturn()
+sync_worker_exit(void)
+{
+	clean_sync_worker();
+
+	/* And flush all writes. */
+	XLogFlush(GetXLogWriteRecPtr());
 
 	/* Stop gracefully */
 	proc_exit(0);
@@ -378,7 +387,13 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 */
 		replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
+		/*
+		 * Sync worker is cleaned at this point. It's ready to sync next table,
+		 * if needed.
+		 */
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->ready_to_reuse = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -1275,7 +1290,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		case SUBREL_STATE_SYNCDONE:
 		case SUBREL_STATE_READY:
 		case SUBREL_STATE_UNKNOWN:
-			finish_sync_worker();	/* doesn't return */
+			sync_worker_exit();	/* doesn't return */
 	}
 
 	/* Calculate the name of the tablesync slot. */
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 78926f8647..8ca6078db4 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -392,6 +392,7 @@ static void stream_open_file(Oid subid, TransactionId xid,
 static void stream_write_change(char action, StringInfo s);
 static void stream_open_and_write_change(TransactionId xid, char action, StringInfo s);
 static void stream_close_file(void);
+static void stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
@@ -3617,6 +3618,27 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can go into process_syncing_tables_for_sync early.
+				 * Before we were able to reuse tablesync workers, that
+				 * process_syncing_tables_for_sync call would exit the worker
+				 * instead of preparing for reuse. Now that tablesync workers
+				 * can be reused and process_syncing_tables_for_sync is not
+				 * responsible for exiting. We need to take care of memory
+				 * contexts here before moving to sync the nex table or exit.
+				 */
+				if (MyLogicalRepWorker->ready_to_reuse)
+				{
+					MemoryContextResetAndDeleteChildren(ApplyMessageContext);
+					MemoryContextSwitchTo(TopMemoryContext);
+
+					/* Pop the error context stack */
+					error_context_stack = errcallback.previous;
+					apply_error_context_stack = error_context_stack;
+					return;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -3636,6 +3658,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+			if (MyLogicalRepWorker->ready_to_reuse)
+			{
+				endofstream = true;
+			}
 		}
 
 		/* Cleanup the memory. */
@@ -3734,12 +3760,15 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 		}
 	}
 
-	/* Pop the error context stack */
-	error_context_stack = errcallback.previous;
-	apply_error_context_stack = error_context_stack;
-
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * If it's still not ready to reuse, this is probably an apply worker.
+	 * End streaming before exiting.
+	 */
+	if (!MyLogicalRepWorker->ready_to_reuse)
+	{
+		/* All done */
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	}
 }
 
 /*
@@ -4324,6 +4353,56 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+ /* stream_build_options
+  * 	Build logical replication streaming options.
+  *
+  * This function sets streaming options including replication slot name
+  * and origin start position. Workers need these options for logical replication.
+  */
+static void
+stream_build_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4436,13 +4515,154 @@ start_apply(XLogRecPtr origin_startpos)
 }
 
 /*
- * Common initialization for leader apply worker and parallel apply worker.
+ * Runs the tablesync worker.
+ * It starts table sync. After successful sync,
+ * builds streaming options and starts streaming.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Set this to false for safety, in case we're already reusing the worker */
+	MyLogicalRepWorker->ready_to_reuse = false;
+
+	start_table_sync(origin_startpos, &slotname);
+
+	/*
+	 * Allocate the origin name in long-lived context for error context
+	 * message.
+	 */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   originname_size);
+	CommitTransactionCommand();
+
+	set_apply_error_context_origin(originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/* Start normal logical streaming replication. */
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	/* Start applying changes to catcup. */
+	start_apply(*origin_startpos);
+}
+
+/*
+ * Runs the apply worker.
+ * It sets up replication origin, the streaming options
+ * and then starts streaming.
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
+{
+	/* This is the leader apply worker */
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+	bool		must_use_password;
+
+	slotname = MySubscription->slotname;
+
+	/*
+	 * This shouldn't happen if the subscription is enabled, but guard
+	 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+	 * crash if slot is NULL.)
+	 */
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+	CommitTransactionCommand();
+
+	/* Is the use of a password mandatory? */
+	must_use_password = MySubscription->passwordrequired &&
+		!superuser_arg(MySubscription->owner);
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											must_use_password,
+											MySubscription->name, &err);
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+	 * We don't really use the output identify_system for anything but it
+	 * does some initializations on the upstream so let's still call it.
+	 */
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	stream_build_options(options, slotname, origin_startpos);
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+	{
+		/* Start streaming with two_phase enabled */
+		options->proto.logical.twophase = true;
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
+	}
+	else
+	{
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+	}
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+
+	/* Run the main loop. */
+	start_apply(*origin_startpos);
+}
+
+/*
+ * Common initialization for logical replication workers; leader apply worker,
+ * parallel apply worker and tablesync worker.
  *
  * Initialize the database connection, in-memory subscription and necessary
  * config options.
  */
 void
-InitializeApplyWorker(void)
+InitializeLogRepWorker(void)
 {
 	MemoryContext oldctx;
 
@@ -4506,9 +4726,10 @@ InitializeApplyWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has started",
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", relation \"%s\" with relid %u has started",
 						MySubscription->name,
-						get_rel_name(MyLogicalRepWorker->relid))));
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
 	else
 		ereport(LOG,
 		/* translator: first %s is the name of logical replication worker */
@@ -4527,7 +4748,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	InitializingApplyWorker = true;
 
@@ -4551,7 +4771,7 @@ ApplyWorkerMain(Datum main_arg)
 	/* Load the libpq-specific functions */
 	load_file("libpqwalreceiver", false);
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
@@ -4559,165 +4779,142 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
+	/*
+	 * Setup callback for syscache so that we know when something changes in
+	 * the subscription relation state. Do this outside the loop to avoid
+	 * exceeding MAX_SYSCACHE_CALLBACKS
+	 */
+	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
+								  invalidate_syncing_table_states,
+								  (Datum) 0);
 
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-		bool		must_use_password;
+	/* This is leader apply worker */
+	run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 
-		myslotname = MySubscription->slotname;
+	proc_exit(0);
+}
 
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
+/* Logical Replication Tablesync worker entry point */
+void
+TablesyncWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *myslotname = NULL;
+	WalRcvStreamOptions options;
+	List	   *rstates;
+	SubscriptionRelState *rstate;
+	ListCell   *lc;
 
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-
-		/* Is the use of a password mandatory? */
-		must_use_password = MySubscription->passwordrequired &&
-			!superuser_arg(MySubscription->owner);
-
-		/* Note that the superuser_arg call can access the DB */
-		CommitTransactionCommand();
+	elog(LOG, "logical replication table synchronization worker has started");
 
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												must_use_password,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
+	/* Attach to slot */
+	logicalrep_worker_attach(worker_slot);
 
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+	/* Setup signal handling */
+	pqsignal(SIGHUP, SignalHandlerForConfigReload);
+	pqsignal(SIGTERM, die);
+	BackgroundWorkerUnblockSignals();
 
-		set_apply_error_context_origin(originname);
-	}
+	/*
+	 * We don't currently need any ResourceOwner in a walreceiver process, but
+	 * if we did, we could call CreateAuxProcessResourceOwner here.
+	 */
+
+	/* Initialise stats to a sanish value */
+	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
+		MyLogicalRepWorker->reply_time = GetCurrentTimestamp();
+
+	/* Load the libpq-specific functions */
+	load_file("libpqwalreceiver", false);
+
+	InitializeLogRepWorker();
+
+	/* Connect to the origin and start the replication. */
+	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
+		 MySubscription->conninfo);
 
 	/*
 	 * Setup callback for syscache so that we know when something changes in
-	 * the subscription relation state.
+	 * the subscription relation state. Do this outside the loop to avoid
+	 * exceeding MAX_SYSCACHE_CALLBACKS
 	 */
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
-
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
-
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-
 	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 * The loop where worker does its job. It loops until there is no relation
+	 * left to sync.
 	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
+	for (;;)
 	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
+		run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
+		if (IsTransactionState())
+			CommitTransactionCommand();
+
+		if (MyLogicalRepWorker->ready_to_reuse)
+		{
+		/* This transaction will be committed by clean_sync_worker. */
+		StartTransactionCommand();
 
-	if (!am_tablesync_worker())
-	{
 		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
+		 * Check if any table whose relation state is still INIT. If a table
+		 * in INIT state is found, the worker will not be finished, it will be
+		 * reused instead.
 		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
+		rstates = GetSubscriptionRelations(MySubscription->oid, true);
+		rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
+
+		foreach(lc, rstates)
 		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+			memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
 
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
+			/*
+			 * Pick the table for the next run if it is not already picked up
+			 * by another worker.
+			 *
+			 * Take exclusive lock to prevent any other sync worker from picking
+			 * the same table.
+			 */
+			LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
+			if (rstate->state != SUBREL_STATE_SYNCDONE &&
+				!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+			{
+				/* Update worker state for the next table */
+				MyLogicalRepWorker->relid = rstate->relid;
+				MyLogicalRepWorker->relstate = rstate->state;
+				MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+				LWLockRelease(LogicalRepWorkerLock);
+				break;
+			}
+			LWLockRelease(LogicalRepWorkerLock);
 		}
+
+		/*
+		 * If a relation with INIT state is assigned, clean up the worker for
+		 * the next iteration.
+		 *
+		 * If there is no more work left for this worker, break the loop to
+		 * exit.
+		 */
+		if ( MyLogicalRepWorker->relstate == SUBREL_STATE_INIT)
+			clean_sync_worker();
 		else
-		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-		}
+			break;
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-								 MySubscription->name,
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-								 "?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+		/* If not exited yet, then the worker will sync another table. */
+		StartTransactionCommand();
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\" has moved to sync table \"%s\" with relid %u.",
+						MySubscription->name, get_rel_name(MyLogicalRepWorker->relid), MyLogicalRepWorker->relid)));
+		CommitTransactionCommand();
+		}
 	}
 
-	/* Run the main loop. */
-	start_apply(origin_startpos);
-
-	proc_exit(0);
+	sync_worker_exit();
 }
 
 /*
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 39588da79f..bbd71d0b42 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -18,6 +18,7 @@ extern PGDLLIMPORT volatile sig_atomic_t ParallelApplyMessagePending;
 
 extern void ApplyWorkerMain(Datum main_arg);
 extern void ParallelApplyWorkerMain(Datum main_arg);
+extern void TablesyncWorkerMain(Datum main_arg);
 
 extern bool IsLogicalWorker(void);
 extern bool IsLogicalParallelApplyWorker(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 343e781896..242737e42b 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -76,6 +76,12 @@ typedef struct LogicalRepWorker
 	/* Indicates whether apply can be performed in parallel. */
 	bool		parallel_apply;
 
+	/*
+	 * Used to indicate whether sync worker is ready for being reused
+	 * to sync another relation.
+	 */
+	bool		ready_to_reuse;
+
 	/* Stats. */
 	XLogRecPtr	last_lsn;
 	TimestampTz last_send_time;
@@ -265,7 +271,7 @@ extern void maybe_reread_subscription(void);
 
 extern void stream_cleanup_files(Oid subid, TransactionId xid);
 
-extern void InitializeApplyWorker(void);
+extern void InitializeLogRepWorker(void);
 
 extern void store_flush_position(XLogRecPtr remote_lsn, XLogRecPtr local_lsn);
 
@@ -273,6 +279,9 @@ extern void store_flush_position(XLogRecPtr remote_lsn, XLogRecPtr local_lsn);
 extern void apply_error_callback(void *arg);
 extern void set_apply_error_context_origin(char *originname);
 
+extern void sync_worker_exit(void);
+extern void clean_sync_worker(void);
+
 /* Parallel apply worker setup and interactions */
 extern void pa_allocate_worker(TransactionId xid);
 extern ParallelApplyWorkerInfo *pa_find_worker(TransactionId xid);
-- 
2.25.1

v10-0002-Add-replication-protocol-cmd-to-create-a-snapshot.patchapplication/octet-stream; name=v10-0002-Add-replication-protocol-cmd-to-create-a-snapshot.patchDownload
From 41b575dcf8b3e3272d7dfaf76f6c2b56e4b70019 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 13 Oct 2022 17:05:45 +0300
Subject: [PATCH 2/3] Add replication protocol cmd to create a snapshot

Introduced CREATE_REPLICATION_SNAPSHOT to be able to create and use a
snapshot without creating a new replication slot, but by using an
existing slot.

CREATE_REPLICATION_SNAPSHOT simply does what CREATE_REPLICATION_SLOT does
without creating a new replication slot.

CREATE_REPLICATION_SNAPSHOT command imports the snapshot into the current
transaction and returns consistent_point. The changes earlier than the
consistent_point will be applied by importing the snapshot. All changes
later than the consistent_point will be available to be consumed from
the replication slot.

This is useful for reusing replication slots in logical replication.
Otherwise, tablesync workers cannot start from a consistent point to copy
a relation and then apply changes by consuming from replication slot.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/protocol.sgml                    |  31 ++++++
 .../libpqwalreceiver/libpqwalreceiver.c       |  69 +++++++++++-
 src/backend/replication/logical/logical.c     |  40 ++++++-
 .../replication/logical/logicalfuncs.c        |   1 +
 src/backend/replication/repl_gram.y           |  18 ++-
 src/backend/replication/repl_scanner.l        |   2 +
 src/backend/replication/slotfuncs.c           |   1 +
 src/backend/replication/walsender.c           | 104 +++++++++++++++++-
 src/include/nodes/replnodes.h                 |  11 ++
 src/include/replication/logical.h             |   1 +
 src/include/replication/walreceiver.h         |  13 +++
 src/tools/pgindent/typedefs.list              |   2 +
 12 files changed, 289 insertions(+), 4 deletions(-)

diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index b11d9a6ba3..f7d5acee93 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2595,6 +2595,37 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
      </listitem>
     </varlistentry>
 
+    <varlistentry id="protocol-replication-replication-slot-snapshot">
+     <term><literal>CREATE_REPLICATION_SNAPSHOT</literal> <replaceable class="parameter">slot_name</replaceable> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
+      <indexterm><primary>CREATE_REPLICATION_SNAPSHOT</primary></indexterm>
+     </term>
+     <listitem>
+      <para>
+       Creates a snapshot including all the changes from the replication slot until
+       the point at which the replication slot becomes consistent. Then the snapshot
+       is used in the current transaction. This command is currently only supported
+       for logical replication slots.
+      </para>
+
+      <para>
+       In response to this command, the server will return a one-row result set,
+       containing the following field:
+       <variablelist>
+        <varlistentry>
+         <term><literal>consistent_point</literal> (<type>text</type>)</term>
+         <listitem>
+          <para>
+           The WAL location at which the slot became consistent.  This is the
+           earliest location from which streaming can start on this replication
+           slot.
+          </para>
+         </listitem>
+        </varlistentry>
+       </variablelist>
+      </para>
+     </listitem>
+    </varlistentry>
+
     <varlistentry id="protocol-replication-base-backup" xreflabel="BASE_BACKUP">
      <term><literal>BASE_BACKUP</literal> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
       <indexterm><primary>BASE_BACKUP</primary></indexterm>
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index dc9c5c82d9..cf88a3b7d4 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -82,6 +82,8 @@ static WalRcvExecResult *libpqrcv_exec(WalReceiverConn *conn,
 									   const int nRetTypes,
 									   const Oid *retTypes);
 static void libpqrcv_disconnect(WalReceiverConn *conn);
+static void libpqrcv_slot_snapshot(WalReceiverConn *conn, char *slotname,
+								   const WalRcvStreamOptions *options, XLogRecPtr *lsn);
 
 static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_connect = libpqrcv_connect,
@@ -98,7 +100,8 @@ static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_create_slot = libpqrcv_create_slot,
 	.walrcv_get_backend_pid = libpqrcv_get_backend_pid,
 	.walrcv_exec = libpqrcv_exec,
-	.walrcv_disconnect = libpqrcv_disconnect
+	.walrcv_disconnect = libpqrcv_disconnect,
+	.walrcv_slot_snapshot = libpqrcv_slot_snapshot
 };
 
 /* Prototypes for private functions */
@@ -982,6 +985,70 @@ libpqrcv_create_slot(WalReceiverConn *conn, const char *slotname,
 	return snapshot;
 }
 
+/*
+ * TODO
+ */
+static void
+libpqrcv_slot_snapshot(WalReceiverConn *conn,
+					   char *slotname,
+					   const WalRcvStreamOptions *options,
+					   XLogRecPtr *lsn)
+{
+	StringInfoData cmd;
+	PGresult   *res;
+	char	   *pubnames_str;
+	List	   *pubnames;
+	char	   *pubnames_literal;
+
+	initStringInfo(&cmd);
+
+	/* Build the command. */
+	appendStringInfo(&cmd, "CREATE_REPLICATION_SNAPSHOT \"%s\"", slotname);
+	appendStringInfoString(&cmd, " (");
+	appendStringInfo(&cmd, " proto_version '%u'",
+					 options->proto.logical.proto_version);
+
+	/* Add publication names. */
+	pubnames = options->proto.logical.publication_names;
+	pubnames_str = stringlist_to_identifierstr(conn->streamConn, pubnames);
+	if (!pubnames_str)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	pubnames_literal = PQescapeLiteral(conn->streamConn, pubnames_str,
+									   strlen(pubnames_str));
+	if (!pubnames_literal)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	appendStringInfo(&cmd, ", publication_names %s", pubnames_literal);
+	PQfreemem(pubnames_literal);
+	pfree(pubnames_str);
+
+	appendStringInfoString(&cmd, " )");
+
+	/* Execute the command. */
+	res = libpqrcv_PQexec(conn->streamConn, cmd.data);
+	pfree(cmd.data);
+
+	if (PQresultStatus(res) != PGRES_TUPLES_OK)
+	{
+		PQclear(res);
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg("Could not create a snapshot by replication slot \"%s\": %s",
+						slotname, pchomp(PQerrorMessage(conn->streamConn)))));
+	}
+
+	if (lsn)
+		*lsn = DatumGetLSN(DirectFunctionCall1Coll(pg_lsn_in, InvalidOid,
+												   CStringGetDatum(PQgetvalue(res, 0, 0))));
+
+	PQclear(res);
+}
+
 /*
  * Return PID of remote backend process.
  */
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 41243d0187..3c11f8e1dd 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -476,6 +476,10 @@ CreateInitDecodingContext(const char *plugin,
  * fast_forward
  *		bypass the generation of logical changes.
  *
+ * need_full_snapshot
+ * 		if true, must obtain a snapshot able to read all tables;
+ *  	if false, one that can read only catalogs is acceptable.
+ *
  * xl_routine
  *		XLogReaderRoutine used by underlying xlogreader
  *
@@ -494,6 +498,7 @@ LogicalDecodingContext *
 CreateDecodingContext(XLogRecPtr start_lsn,
 					  List *output_plugin_options,
 					  bool fast_forward,
+					  bool need_full_snapshot,
 					  XLogReaderRoutine *xl_routine,
 					  LogicalOutputPluginWriterPrepareWrite prepare_write,
 					  LogicalOutputPluginWriterWrite do_write,
@@ -502,6 +507,7 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 	LogicalDecodingContext *ctx;
 	ReplicationSlot *slot;
 	MemoryContext old_context;
+	TransactionId xmin_horizon = InvalidTransactionId;
 
 	/* shorter lines... */
 	slot = MyReplicationSlot;
@@ -571,8 +577,40 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 		start_lsn = slot->data.confirmed_flush;
 	}
 
+
+	/*
+	 * We need to determine a safe xmin horizon to start decoding from if we
+	 * want to create a snapshot too. Otherwise we would end up with a
+	 * snapshot that cannot be imported since xmin value from the snapshot may
+	 * be less than the oldest safe xmin. To avoid this call
+	 * GetOldestSafeDecodingTransactionId() to return a safe xmin value, which
+	 * can be used while exporting/importing the snapshot.
+	 *
+	 * So we have to acquire the ProcArrayLock to prevent computation of new
+	 * xmin horizons by other backends, get the safe decoding xid, and inform
+	 * the slot machinery about the new limit. Once that's done the
+	 * ProcArrayLock can be released as the slot machinery now is protecting
+	 * against vacuum.
+	 */
+	if (need_full_snapshot)
+	{
+		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+		xmin_horizon = GetOldestSafeDecodingTransactionId(!need_full_snapshot);
+
+		SpinLockAcquire(&slot->mutex);
+		slot->effective_catalog_xmin = xmin_horizon;
+		slot->data.catalog_xmin = xmin_horizon;
+		slot->effective_xmin = xmin_horizon;
+		SpinLockRelease(&slot->mutex);
+
+		ReplicationSlotsComputeRequiredXmin(true);
+
+		LWLockRelease(ProcArrayLock);
+	}
+
 	ctx = StartupDecodingContext(output_plugin_options,
-								 start_lsn, InvalidTransactionId, false,
+								 start_lsn, xmin_horizon, need_full_snapshot,
 								 fast_forward, xl_routine, prepare_write,
 								 do_write, update_progress);
 
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index 55a24c02c9..85c5cdb633 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -208,6 +208,7 @@ pg_logical_slot_get_changes_guts(FunctionCallInfo fcinfo, bool confirm, bool bin
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									options,
 									false,
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/repl_gram.y b/src/backend/replication/repl_gram.y
index 0c874e33cf..957a5cc653 100644
--- a/src/backend/replication/repl_gram.y
+++ b/src/backend/replication/repl_gram.y
@@ -65,6 +65,7 @@ Node *replication_parse_result;
 %token K_CREATE_REPLICATION_SLOT
 %token K_DROP_REPLICATION_SLOT
 %token K_TIMELINE_HISTORY
+%token K_CREATE_REPLICATION_SNAPSHOT
 %token K_WAIT
 %token K_TIMELINE
 %token K_PHYSICAL
@@ -80,7 +81,7 @@ Node *replication_parse_result;
 %type <node>	command
 %type <node>	base_backup start_replication start_logical_replication
 				create_replication_slot drop_replication_slot identify_system
-				read_replication_slot timeline_history show
+				read_replication_slot timeline_history show create_replication_snapshot
 %type <list>	generic_option_list
 %type <defelt>	generic_option
 %type <uintval>	opt_timeline
@@ -114,6 +115,7 @@ command:
 			| read_replication_slot
 			| timeline_history
 			| show
+			| create_replication_snapshot
 			;
 
 /*
@@ -307,6 +309,19 @@ timeline_history:
 				}
 			;
 
+/*
+ * CREATE_REPLICATION_SNAPSHOT %s options
+ */
+create_replication_snapshot:
+			K_CREATE_REPLICATION_SNAPSHOT var_name plugin_options
+				{
+					CreateReplicationSnapshotCmd *n = makeNode(CreateReplicationSnapshotCmd);
+					n->slotname = $2;
+					n->options = $3;
+					$$ = (Node *) n;
+				}
+			;
+
 opt_physical:
 			K_PHYSICAL
 			| /* EMPTY */
@@ -400,6 +415,7 @@ ident_or_keyword:
 			| K_CREATE_REPLICATION_SLOT	{ $$ = "create_replication_slot"; }
 			| K_DROP_REPLICATION_SLOT		{ $$ = "drop_replication_slot"; }
 			| K_TIMELINE_HISTORY			{ $$ = "timeline_history"; }
+			| K_CREATE_REPLICATION_SNAPSHOT	{ $$ = "create_replication_snapshot"; }
 			| K_WAIT						{ $$ = "wait"; }
 			| K_TIMELINE					{ $$ = "timeline"; }
 			| K_PHYSICAL					{ $$ = "physical"; }
diff --git a/src/backend/replication/repl_scanner.l b/src/backend/replication/repl_scanner.l
index cb467ca46f..5ba2e9c54b 100644
--- a/src/backend/replication/repl_scanner.l
+++ b/src/backend/replication/repl_scanner.l
@@ -126,6 +126,7 @@ START_REPLICATION	{ return K_START_REPLICATION; }
 CREATE_REPLICATION_SLOT		{ return K_CREATE_REPLICATION_SLOT; }
 DROP_REPLICATION_SLOT		{ return K_DROP_REPLICATION_SLOT; }
 TIMELINE_HISTORY	{ return K_TIMELINE_HISTORY; }
+CREATE_REPLICATION_SNAPSHOT	{ return K_CREATE_REPLICATION_SNAPSHOT; }
 PHYSICAL			{ return K_PHYSICAL; }
 RESERVE_WAL			{ return K_RESERVE_WAL; }
 LOGICAL				{ return K_LOGICAL; }
@@ -303,6 +304,7 @@ replication_scanner_is_replication_command(void)
 		case K_DROP_REPLICATION_SLOT:
 		case K_READ_REPLICATION_SLOT:
 		case K_TIMELINE_HISTORY:
+		case K_CREATE_REPLICATION_SNAPSHOT:
 		case K_SHOW:
 			/* Yes; push back the first token so we can parse later. */
 			repl_pushed_back_token = first_token;
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 6035cf4816..c5b2d5b61f 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -486,6 +486,7 @@ pg_logical_replication_slot_advance(XLogRecPtr moveto)
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									NIL,
 									true,	/* fast_forward */
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index d3a136b6f5..87392dea1f 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -238,6 +238,7 @@ static void CreateReplicationSlot(CreateReplicationSlotCmd *cmd);
 static void DropReplicationSlot(DropReplicationSlotCmd *cmd);
 static void StartReplication(StartReplicationCmd *cmd);
 static void StartLogicalReplication(StartReplicationCmd *cmd);
+static void CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd);
 static void ProcessStandbyMessage(void);
 static void ProcessStandbyReplyMessage(void);
 static void ProcessStandbyHSFeedbackMessage(void);
@@ -1284,7 +1285,7 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	 * are reported early.
 	 */
 	logical_decoding_ctx =
-		CreateDecodingContext(cmd->startpoint, cmd->options, false,
+		CreateDecodingContext(cmd->startpoint, cmd->options, false, false,
 							  XL_ROUTINE(.page_read = logical_read_xlog_page,
 										 .segment_open = WalSndSegmentOpen,
 										 .segment_close = wal_segment_close),
@@ -1336,6 +1337,98 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	EndCommand(&qc, DestRemote, false);
 }
 
+/*
+ * Create a snapshot from an existing replication slot.
+ */
+static void
+CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd)
+{
+	Snapshot	snap;
+	LogicalDecodingContext *ctx;
+	char		xloc[MAXFNAMELEN];
+	DestReceiver *dest;
+	TupOutputState *tstate;
+	TupleDesc	tupdesc;
+	Datum		values[1];
+	bool		nulls[1] = {0};
+
+	Assert(!MyReplicationSlot);
+
+	CheckLogicalDecodingRequirements();
+
+	if (!IsTransactionBlock())
+		ereport(ERROR,
+				(errmsg("%s must be called inside a transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (XactIsoLevel != XACT_REPEATABLE_READ)
+		ereport(ERROR,
+				(errmsg("%s must be called in REPEATABLE READ isolation mode transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (!XactReadOnly)
+		ereport(ERROR,
+				(errmsg("%s must be called in a read only transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (FirstSnapshotSet)
+		ereport(ERROR,
+				(errmsg("%s must be called before any query",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (IsSubTransaction())
+		ereport(ERROR,
+				(errmsg("%s must not be called in a subtransaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	ReplicationSlotAcquire(cmd->slotname, false);
+
+	ctx = CreateDecodingContext(MyReplicationSlot->data.restart_lsn,
+								cmd->options,
+								false,
+								true,
+								XL_ROUTINE(.page_read = logical_read_xlog_page,
+										   .segment_open = WalSndSegmentOpen,
+										   .segment_close = wal_segment_close),
+								WalSndPrepareWrite, WalSndWriteData,
+								WalSndUpdateProgress);
+
+	/*
+	 * Signal that we don't need the timeout mechanism. We're just creating
+	 * the snapshot with the replication slot and don't yet accept feedback
+	 * messages or send keepalives. As we possibly need to wait for further
+	 * WAL the walsender would otherwise possibly be killed too soon.
+	 */
+	last_reply_timestamp = 0;
+
+	/* build initial snapshot, might take a while */
+	DecodingContextFindStartpoint(ctx);
+
+	snap = SnapBuildInitialSnapshot(ctx->snapshot_builder);
+	RestoreTransactionSnapshot(snap, MyProc);
+
+	/* Don't need the decoding context anymore */
+	FreeDecodingContext(ctx);
+
+	/* Create a tuple to send consistent WAL location */
+	snprintf(xloc, sizeof(xloc), "%X/%X",
+			 LSN_FORMAT_ARGS(MyReplicationSlot->data.confirmed_flush));
+
+	dest = CreateDestReceiver(DestRemoteSimple);
+	tupdesc = CreateTemplateTupleDesc(1);
+	TupleDescInitBuiltinEntry(tupdesc, (AttrNumber) 1, "consistent_point",
+							  TEXTOID, -1, 0);
+	tstate = begin_tup_output_tupdesc(dest, tupdesc, &TTSOpsVirtual);
+
+	/* consistent wal location */
+	values[0] = CStringGetTextDatum(xloc);
+
+	do_tup_output(tstate, values, nulls);
+	end_tup_output(tstate);
+
+	ReplicationSlotRelease();
+}
+
 /*
  * LogicalDecodingContext 'prepare_write' callback.
  *
@@ -1864,6 +1957,15 @@ exec_replication_command(const char *cmd_string)
 			}
 			break;
 
+		case T_CreateReplicationSnapshotCmd:
+			{
+				cmdtag = "CREATE_REPLICATION_SNAPSHOT";
+				set_ps_display(cmdtag);
+				CreateReplicationSnapshot((CreateReplicationSnapshotCmd *) cmd_node);
+				EndReplicationCommand(cmdtag);
+				break;
+			}
+
 		default:
 			elog(ERROR, "unrecognized replication command node tag: %u",
 				 cmd_node->type);
diff --git a/src/include/nodes/replnodes.h b/src/include/nodes/replnodes.h
index 4321ba8f86..154ab74f33 100644
--- a/src/include/nodes/replnodes.h
+++ b/src/include/nodes/replnodes.h
@@ -108,4 +108,15 @@ typedef struct TimeLineHistoryCmd
 	TimeLineID	timeline;
 } TimeLineHistoryCmd;
 
+/* ----------------------
+ *		CREATE_REPLICATION_SNAPSHOT command
+ * ----------------------
+ */
+typedef struct CreateReplicationSnapshotCmd
+{
+	NodeTag		type;
+	char	   *slotname;
+	List	   *options;
+} CreateReplicationSnapshotCmd;
+
 #endif							/* REPLNODES_H */
diff --git a/src/include/replication/logical.h b/src/include/replication/logical.h
index 5f49554ea0..6535786a0e 100644
--- a/src/include/replication/logical.h
+++ b/src/include/replication/logical.h
@@ -125,6 +125,7 @@ extern LogicalDecodingContext *CreateInitDecodingContext(const char *plugin,
 extern LogicalDecodingContext *CreateDecodingContext(XLogRecPtr start_lsn,
 													 List *output_plugin_options,
 													 bool fast_forward,
+													 bool need_full_snapshot,
 													 XLogReaderRoutine *xl_routine,
 													 LogicalOutputPluginWriterPrepareWrite prepare_write,
 													 LogicalOutputPluginWriterWrite do_write,
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 281626fa6f..f01e5f23a7 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -386,6 +386,16 @@ typedef WalRcvExecResult *(*walrcv_exec_fn) (WalReceiverConn *conn,
  */
 typedef void (*walrcv_disconnect_fn) (WalReceiverConn *conn);
 
+/*
+ * walrcv_slot_snapshot_fn
+ *
+ * Create a snapshot by an existing replication slot
+ */
+typedef void (*walrcv_slot_snapshot_fn) (WalReceiverConn *conn,
+										 char *slotname,
+										 const WalRcvStreamOptions *options,
+										 XLogRecPtr *lsn);
+
 typedef struct WalReceiverFunctionsType
 {
 	walrcv_connect_fn walrcv_connect;
@@ -403,6 +413,7 @@ typedef struct WalReceiverFunctionsType
 	walrcv_get_backend_pid_fn walrcv_get_backend_pid;
 	walrcv_exec_fn walrcv_exec;
 	walrcv_disconnect_fn walrcv_disconnect;
+	walrcv_slot_snapshot_fn walrcv_slot_snapshot;
 } WalReceiverFunctionsType;
 
 extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
@@ -437,6 +448,8 @@ extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
 	WalReceiverFunctions->walrcv_exec(conn, exec, nRetTypes, retTypes)
 #define walrcv_disconnect(conn) \
 	WalReceiverFunctions->walrcv_disconnect(conn)
+#define walrcv_slot_snapshot(conn, slotname, options, lsn) \
+	WalReceiverFunctions->walrcv_slot_snapshot(conn, slotname, options, lsn)
 
 static inline void
 walrcv_clear_result(WalRcvExecResult *walres)
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 260854747b..985a4ab7fd 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2348,6 +2348,7 @@ ReplicationSlotInvalidationCause
 ReplicationSlotOnDisk
 ReplicationSlotPersistency
 ReplicationSlotPersistentData
+CreateReplicationSnapshotCmd
 ReplicationState
 ReplicationStateCtl
 ReplicationStateOnDisk
@@ -3853,6 +3854,7 @@ walrcv_receive_fn
 walrcv_send_fn
 walrcv_server_version_fn
 walrcv_startstreaming_fn
+walrcv_slot_snapshot_fn
 wchar2mb_with_len_converter
 wchar_t
 win32_deadchild_waitinfo
-- 
2.25.1

v13-0003-Reuse-Replication-Slot-and-Origin-in-Tablesync.patchapplication/octet-stream; name=v13-0003-Reuse-Replication-Slot-and-Origin-in-Tablesync.patchDownload
From f85b562995feaec25d0755d8b55d473c8cff8276 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH 3/3] Reuse Replication Slot and Origin in Tablesync

This commit allows reusing replication slots and origins during tablesync.

Earlier, a tablesync worker was creating a new replication slot and origin
each time it syncs a new table. With this patch, replication
slots/origins can be reusable for tablesync.

This reduces the overhead of creating/dropping replication slots and origins
and improves tablesync speed significantly especially for empty or small tables.

If the state of the current table is INIT or DATASYNC, tablesync worker needs a
repliation slot/origin. If the worker has not created slot and origin in
its previous runs, it will create those first. Otherwise the worker reuses
slot and origin created by the same worker in previous iterations earlier.
Tables in FINISHEDCOPY are expected to have a replication slot and origin.
Slot and origin names for such tables are persisted in
pg_subscription_rel catalog. Tablesync worker can fetch them and proceed
with existing slot and origin of FINISHEDCOPY tables and does not need to
create new ones.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/catalogs.sgml                    |  31 ++
 src/backend/catalog/pg_subscription.c         | 247 +++++++++++-
 src/backend/commands/subscriptioncmds.c       | 229 +++++++----
 .../replication/logical/applyparallelworker.c |   3 +-
 src/backend/replication/logical/launcher.c    |   9 +-
 src/backend/replication/logical/tablesync.c   | 378 +++++++++++++-----
 src/backend/replication/logical/worker.c      | 142 ++++---
 src/include/catalog/pg_subscription.h         |   6 +
 src/include/catalog/pg_subscription_rel.h     |  14 +-
 src/include/replication/slot.h                |   3 +-
 src/include/replication/worker_internal.h     |  23 +-
 11 files changed, 850 insertions(+), 235 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index ed32ca0349..0183ecd21e 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8030,6 +8030,19 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        origin.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>sublastusedid</structfield> <type>int8</type>
+      </para>
+      <para>
+      The last used ID for tablesync workers. It acts as an unique identifier
+      for replication slots which are created by tablesync workers.
+      The last used ID needs to be persisted to make logical replication safely
+      proceed after any interruption. If sublastusedid is 0, then no table has
+      been synced yet.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
@@ -8114,6 +8127,24 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        otherwise null
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srrelslotname</structfield> <type>name</type>
+      </para>
+      <para>
+       Replication slot name that is used for synchronization of relation
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srreloriginname</structfield> <type>name</type>
+      </para>
+      <para>
+       Origin name that is used for tracking synchronization of relation
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index d07f88ce28..152fdaa310 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -108,6 +108,14 @@ GetSubscription(Oid subid, bool missing_ok)
 								   Anum_pg_subscription_suborigin);
 	sub->origin = TextDatumGetCString(datum);
 
+	/* Get last used id */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
+							tup,
+							Anum_pg_subscription_sublastusedid,
+							&isnull);
+	Assert(!isnull);
+	sub->lastusedid = DatumGetInt64(datum);
+
 	ReleaseSysCache(tup);
 
 	return sub;
@@ -199,6 +207,44 @@ DisableSubscription(Oid subid)
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update the last used replication slot ID for the given subscription.
+ */
+void
+UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid)
+{
+	Relation	rel;
+	bool		nulls[Natts_pg_subscription];
+	bool		replaces[Natts_pg_subscription];
+	Datum		values[Natts_pg_subscription];
+	HeapTuple	tup;
+
+	/* Look up the subscription in the catalog */
+	rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+	tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	/* Form a new tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	replaces[Anum_pg_subscription_sublastusedid - 1] = true;
+	values[Anum_pg_subscription_sublastusedid- 1] = Int64GetDatum(lastusedid);
+
+	/* Update the catalog */
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+	heap_freetuple(tup);
+
+	table_close(rel, NoLock);
+}
+
 /*
  * Convert text array to list of strings.
  *
@@ -228,7 +274,7 @@ textarray_to_stringlist(ArrayType *textarray)
  */
 void
 AddSubscriptionRelState(Oid subid, Oid relid, char state,
-						XLogRecPtr sublsn)
+						XLogRecPtr sublsn, char *relslotname, char *reloriginname)
 {
 	Relation	rel;
 	HeapTuple	tup;
@@ -257,6 +303,16 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
 	else
 		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -269,6 +325,60 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Internal function to modify columns for relation state update
+ */
+static void
+UpdateSubscriptionRelState_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char state,
+									XLogRecPtr sublsn)
+{
+	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
+	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+
+	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (sublsn != InvalidXLogRecPtr)
+		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
+	else
+		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication slot update
+ */
+static void
+UpdateSubscriptionRelReplicationSlot_internal(Datum *values,
+											bool *nulls,
+											bool *replaces,
+											char *relslotname)
+{
+	replaces[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication origin update
+ */
+static void
+UpdateSubscriptionRelOrigin_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char *reloriginname)
+{
+	replaces[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+}
+
 /*
  * Update the state of a subscription table.
  */
@@ -299,14 +409,56 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	memset(nulls, false, sizeof(nulls));
 	memset(replaces, false, sizeof(replaces));
 
-	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
-	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
 
-	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
-	if (sublsn != InvalidXLogRecPtr)
-		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
-	else
-		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update replication slot name, origin name and state of
+ * a subscription table in one transaction.
+ */
+void
+UpdateSubscriptionRel(Oid subid,
+					  Oid relid,
+					  char state,
+					  XLogRecPtr sublsn,
+					  char *relslotname,
+					  char *reloriginname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
+	UpdateSubscriptionRelOrigin_internal(values, nulls, replaces, reloriginname);
 
 	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
 							replaces);
@@ -318,6 +470,85 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Get origin name of subscription table.
+ *
+ * reloriginname's value has the replication origin name if the origin exists.
+ */
+void
+GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*originname;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srreloriginname, isnull);
+	if (!*isnull)
+	{
+		originname = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(reloriginname, originname, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
+/*
+ * Get replication slot name of subscription table.
+ *
+ * slotname's value has the replication slot name if the subscription has any.
+ */
+void
+GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*relrepslot;
+	bool		isnull;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srrelslotname, &isnull);
+	if (!isnull)
+	{
+		relrepslot = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(slotname, relrepslot, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
 /*
  * Get state of subscription table.
  *
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 1c88c2bccb..7cec23a806 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -710,6 +710,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 		publicationListToArray(publications);
 	values[Anum_pg_subscription_suborigin - 1] =
 		CStringGetTextDatum(opts.origin);
+	values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(0);
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -773,7 +774,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 										 rv->schemaname, rv->relname);
 
 				AddSubscriptionRelState(subid, relid, table_state,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 			}
 
 			/*
@@ -864,6 +865,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
 	bool		must_use_password;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -943,7 +946,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				AddSubscriptionRelState(sub->oid, relid,
 										copy_data ? SUBREL_STATE_INIT : SUBREL_STATE_READY,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 				ereport(DEBUG1,
 						(errmsg_internal("table \"%s.%s\" added to subscription \"%s\"",
 										 rv->schemaname, rv->relname, sub->name)));
@@ -967,6 +970,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				char		state;
 				XLogRecPtr	statelsn;
+				char		slotname[NAMEDATALEN] = {0};
 
 				/*
 				 * Lock pg_subscription_rel with AccessExclusiveLock to
@@ -993,13 +997,36 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/*
+				 * Find the logical replication sync worker. If exists, store
+				 * the slot number for dropping associated replication slots
+				 * later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot_name);
+				}
+				else
+				{
+					/*
+					 * Sync of this relation might be failed in an earlier
+					 * attempt, but the replication slot might still exist.
+					 */
+					GetSubscriptionRelReplicationSlot(sub->oid, relid, slotname);
+					if (strlen(slotname) > 0)
+						sub_remove_slots = lappend(sub_remove_slots, slotname);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
 				 * tablesync origin.
 				 */
-				if (state != SUBREL_STATE_READY)
+				if (state != SUBREL_STATE_READY &&
+					state != SUBREL_STATE_SYNCDONE)
 				{
 					char		originname[NAMEDATALEN];
 
@@ -1027,31 +1054,24 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
-		 * the database operations we won't be able to rollback dropped slots.
+		 * Drop the replication slots associated with tablesync workers for
+		 * removed tables. This has to be at the end because otherwise if
+		 * there is an error while doing the database operations we won't be
+		 * able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			memcpy(syncslotname, lfirst(lc), sizeof(NAMEDATALEN));
+
+			/*
+			 * There is no certainty, maybe the slot does not exist yet. Also,
+			 * if we fail after removing some of the slots, next time, it will
+			 * again try to drop already dropped slots and fail. For these
+			 * reasons, we allow missing_ok = true for the drop.
+			 */
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1474,6 +1494,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	char	   *subname;
 	char	   *conninfo;
 	char	   *slotname;
+	int64		lastusedid;
 	List	   *subworkers;
 	ListCell   *lc;
 	char		originname[NAMEDATALEN];
@@ -1546,6 +1567,14 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	else
 		slotname = NULL;
 
+	/* Get the last used identifier by the subscription */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID, tup,
+							Anum_pg_subscription_sublastusedid, &isnull);
+	if (!isnull)
+		lastusedid = DatumGetInt64(datum);
+	else
+		lastusedid = 0;
+
 	/*
 	 * Since dropping a replication slot is not transactional, the replication
 	 * slot stays dropped even if the transaction rolls back.  So we cannot
@@ -1595,6 +1624,8 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	}
 	list_free(subworkers);
 
+	rstates = GetSubscriptionRelations(subid, true);
+
 	/*
 	 * Remove the no-longer-useful entry in the launcher's table of apply
 	 * worker start times.
@@ -1606,36 +1637,26 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	ApplyLauncherForgetWorkerStartTime(subid);
 
 	/*
-	 * Cleanup of tablesync replication origins.
-	 *
-	 * Any READY-state relations would already have dealt with clean-ups.
+	 * Cleanup of tablesync replication origins associated with the
+	 * subscription, if exists. Try to drop origins by creating all origin
+	 * names created for this subscription.
 	 *
 	 * Note that the state can't change because we have already stopped both
 	 * the apply and tablesync workers and they can't restart because of
 	 * exclusive lock on the subscription.
+	 *
+	 * XXX: This can be handled better instead of looping through all possible
 	 */
-	rstates = GetSubscriptionRelations(subid, true);
-	foreach(lc, rstates)
+	for (int64 i = 1; i <= lastusedid; i++)
 	{
-		SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-		Oid			relid = rstate->relid;
-
-		/* Only cleanup resources of tablesync workers */
-		if (!OidIsValid(relid))
-			continue;
+		char		originname_to_drop[NAMEDATALEN] = {0};
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * It is possible that the origin is not yet created for tablesync
-		 * worker so passing missing_ok = true. This can happen for the states
-		 * before SUBREL_STATE_FINISHEDCOPY.
-		 */
-		ReplicationOriginNameForLogicalRep(subid, relid, originname,
-										   sizeof(originname));
-		replorigin_drop_by_name(originname, true, false);
+		snprintf(originname_to_drop, sizeof(originname_to_drop), "pg_%u_%lld", subid, (long long) i);
+		/* missing_ok = true, since the origin might be already dropped. */
+		replorigin_drop_by_name(originname_to_drop, true, false);
 	}
 
+
 	/* Clean up dependencies */
 	deleteSharedDependencyRecordsFor(SubscriptionRelationId, subid, 0);
 
@@ -1688,39 +1709,17 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
+		List	   *slots = NULL;
 
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
 
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char	   *syncslotname = (char *) lfirst(lc);
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1743,6 +1742,71 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * Get the replication slot names associated with the subscription.
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok)
+{
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+					 " FROM pg_replication_slots"
+					 " WHERE slot_name LIKE 'pg_%i_sync_%%';",
+					 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+					errmsg("could not receive list of slots associated with the subscription %u, error: %s",
+					subid, res->err));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();
+		return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -2155,6 +2219,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -2165,18 +2230,20 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
-			char		syncslotname[NAMEDATALEN] = {0};
-
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
-				 syncslotname);
+				 worker->slot_name);
 		}
 	}
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index f16e2377bf..a57f178087 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -440,7 +440,8 @@ pa_launch_parallel_worker(void)
 										MySubscription->name,
 										MyLogicalRepWorker->userid,
 										InvalidOid,
-										dsm_segment_handle(winfo->dsm_seg));
+										dsm_segment_handle(winfo->dsm_seg),
+										InvalidRepSlotId);
 
 	if (launched)
 	{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 86ee1f25c2..9ab40fa4b8 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -304,7 +304,7 @@ logicalrep_workers_find(Oid subid, bool only_running)
  */
 bool
 logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
-						 Oid relid, dsm_handle subworker_dsm)
+						 Oid relid, dsm_handle subworker_dsm, int64 slotid)
 {
 	BackgroundWorker bgw;
 	BackgroundWorkerHandle *bgw_handle;
@@ -430,6 +430,9 @@ retry:
 	worker->launch_time = now;
 	worker->in_use = true;
 	worker->generation++;
+	worker->created_slot = false;
+	worker->rep_slot_id = slotid;
+	worker->slot_name = (char *) palloc(NAMEDATALEN);
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -437,6 +440,7 @@ retry:
 	worker->relid = relid;
 	worker->relstate = SUBREL_STATE_UNKNOWN;
 	worker->relstate_lsn = InvalidXLogRecPtr;
+	worker->ready_to_reuse = false;
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
@@ -1179,7 +1183,8 @@ ApplyLauncherMain(Datum main_arg)
 				ApplyLauncherSetWorkerStartTime(sub->oid, now);
 				logicalrep_worker_launch(sub->dbid, sub->oid, sub->name,
 										 sub->owner, InvalidOid,
-										 DSM_HANDLE_INVALID);
+										 DSM_HANDLE_INVALID,
+										 InvalidRepSlotId);
 			}
 			else
 			{
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 524be7088a..2d50b9a8a6 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -318,40 +318,29 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 								   MyLogicalRepWorker->relid,
 								   MyLogicalRepWorker->relstate,
 								   MyLogicalRepWorker->relstate_lsn);
+		CommitTransactionCommand();
 
 		/*
-		 * End streaming so that LogRepWorkerWalRcvConn can be used to drop
-		 * the slot.
-		 */
-		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
-
-		/*
-		 * Cleanup the tablesync slot.
+		 * Cleanup the tablesync slot. If the slot name used by this worker is
+		 * different from the default slot name for the worker, this means the
+		 * current table had started to being synchronized by another worker
+		 * and replication slot. And this worker is reusing a replication slot
+		 * from a previous attempt. We do not need that replication slot
+		 * anymore.
 		 *
 		 * This has to be done after updating the state because otherwise if
 		 * there is an error while doing the database operations we won't be
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
+										MyLogicalRepWorker->rep_slot_id,
 										syncslotname,
 										sizeof(syncslotname));
 
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
-		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
-
-		CommitTransactionCommand();
-		pgstat_report_stat(false);
-
-		/*
-		 * Start a new transaction to clean up the tablesync origin tracking.
-		 * This transaction will be ended within the finish_sync_worker().
-		 * Now, even, if we fail to remove this here, the apply worker will
-		 * ensure to clean it up afterward.
+		 * We are safe to drop the replication tracking origin after this
+		 * point. Now, even, if we fail to remove this here, the apply worker
+		 * will ensure to clean it up afterward.
 		 *
 		 * We need to do this after the table state is set to SYNCDONE.
 		 * Otherwise, if an error occurs while performing the database
@@ -360,32 +349,78 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 * have been cleared before restart. So, the restarted worker will use
 		 * invalid replication progress state resulting in replay of
 		 * transactions that have already been applied.
+		 *
+		 * Firstly reset the origin session to remove the ownership of the
+		 * slot. This is needed to allow the origin to be dropped or reused
+		 * later.
 		 */
+		replorigin_session_reset();
+		replorigin_session_origin = InvalidRepOriginId;
+		replorigin_session_origin_lsn = InvalidXLogRecPtr;
+		replorigin_session_origin_timestamp = 0;
+
 		StartTransactionCommand();
+		if (MyLogicalRepWorker->slot_name && strcmp(syncslotname, MyLogicalRepWorker->slot_name) != 0)
+		{
+			/*
+			 * End streaming so that LogRepWorkerWalRcvConn can be used to
+			 * drop the slot.
+			 */
+			walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
 
-		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
+			ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+											   MyLogicalRepWorker->relid,
+											   originname,
+											   sizeof(originname));
+
+			/*
+			 * Drop replication origin
+			 *
+			 * There is a chance that the user is concurrently performing refresh
+			 * for the subscription where we remove the table state and its origin
+			 * or the apply worker would have removed this origin. So passing
+			 * missing_ok = true.
+			 */
+			replorigin_drop_by_name(originname, true, false);
+		}
 
 		/*
-		 * Resetting the origin session removes the ownership of the slot.
-		 * This is needed to allow the origin to be dropped.
+		 * We are safe to remove persisted replication slot and origin data,
+		 * since it's already in SYNCDONE state. They will not be needed
+		 * anymore.
 		 */
-		replorigin_session_reset();
-		replorigin_session_origin = InvalidRepOriginId;
-		replorigin_session_origin_lsn = InvalidXLogRecPtr;
-		replorigin_session_origin_timestamp = 0;
+		UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+							  MyLogicalRepWorker->relid,
+							  MyLogicalRepWorker->relstate,
+							  MyLogicalRepWorker->relstate_lsn,
+							  NULL,
+							  NULL);
+		ereport(DEBUG2,
+			(errmsg("process_syncing_tables_for_sync: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					"NULL", "NULL", MyLogicalRepWorker->relstate,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", relation \"%s\" with relid %u has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
+
+		CommitTransactionCommand();
+		pgstat_report_stat(false);
+
+		StartTransactionCommand();
 
 		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * There is a chance that the user is concurrently performing refresh
-		 * for the subscription where we remove the table state and its origin
-		 * or the apply worker would have removed this origin. So passing
-		 * missing_ok = true.
+		 * This should return the default origin name for the worker. Even if
+		 * the worker used a different origin for this table, it should be
+		 * dropped and removed from the catalog so far.
 		 */
-		replorigin_drop_by_name(originname, true, false);
+		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+										   MyLogicalRepWorker->relid,
+										   originname,
+										   sizeof(originname));
 
 		/*
 		 * Sync worker is cleaned at this point. It's ready to sync next table,
@@ -480,6 +515,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 			if (current_lsn >= rstate->lsn)
 			{
 				char		originname[NAMEDATALEN];
+				bool		is_origin_null = true;
 
 				rstate->state = SUBREL_STATE_READY;
 				rstate->lsn = current_lsn;
@@ -500,18 +536,31 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				 * error while dropping we won't restart it to drop the
 				 * origin. So passing missing_ok = true.
 				 */
-				ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-												   rstate->relid,
-												   originname,
-												   sizeof(originname));
-				replorigin_drop_by_name(originname, true, false);
+				GetSubscriptionRelOrigin(MyLogicalRepWorker->subid,
+										 rstate->relid, originname,
+										 &is_origin_null);
+
+				if (!is_origin_null)
+				{
+					replorigin_drop_by_name(originname, true, false);
+				}
 
 				/*
 				 * Update the state to READY only after the origin cleanup.
 				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-										   rstate->relid, rstate->state,
-										   rstate->lsn);
+				UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+									  rstate->relid,
+									  rstate->state,
+									  rstate->lsn,
+									  NULL,
+									  NULL);
+				ereport(DEBUG2,
+					(errmsg("process_syncing_tables_for_apply: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+							"NULL", "NULL", rstate->state,
+							rstate->relid, MyLogicalRepWorker->subid)));
+
+				CommitTransactionCommand();
+				started_tx = false;
 			}
 		}
 		else
@@ -600,12 +649,25 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						TimestampDifferenceExceeds(hentry->last_start_time, now,
 												   wal_retrieve_retry_interval))
 					{
+						if (IsTransactionState())
+							CommitTransactionCommand();
+						StartTransactionCommand();
+						started_tx = true;
+
+						MySubscription->lastusedid++;
+						UpdateSubscriptionLastSlotId(MyLogicalRepWorker->subid,
+													 MySubscription->lastusedid);
+						ereport(DEBUG2,
+								(errmsg("process_syncing_tables_for_apply: incremented lastusedid to %lld for subscription %u",
+										(long long) MySubscription->lastusedid, MySubscription->oid)));
+
 						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
 												 MySubscription->oid,
 												 MySubscription->name,
 												 MyLogicalRepWorker->userid,
 												 rstate->relid,
-												 DSM_HANDLE_INVALID);
+												 DSM_HANDLE_INVALID,
+												 MySubscription->lastusedid);
 						hentry->last_start_time = now;
 					}
 				}
@@ -1228,8 +1290,8 @@ copy_table(Relation rel)
  * The name must not exceed NAMEDATALEN - 1 because of remote node constraints
  * on slot name length. We append system_identifier to avoid slot_name
  * collision with subscriptions in other clusters. With the current scheme
- * pg_%u_sync_%u_UINT64_FORMAT (3 + 10 + 6 + 10 + 20 + '\0'), the maximum
- * length of slot_name will be 50.
+ * pg_%u_sync_%lu_UINT64_FORMAT (3 + 10 + 6 + 20 + 20 + '\0'), the maximum
+ * length of slot_name will be 45.
  *
  * The returned slot name is stored in the supplied buffer (syncslotname) with
  * the given size.
@@ -1240,11 +1302,11 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
+ReplicationSlotNameForTablesync(Oid suboid, int64 slotid,
 								char *syncslotname, Size szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%lld_" UINT64_FORMAT, suboid,
+			(long long) slotid, GetSystemIdentifier());
 }
 
 /*
@@ -1268,6 +1330,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	char		originname[NAMEDATALEN];
 	RepOriginId originid;
 	bool		must_use_password;
+	char	   *prev_slotname;
 
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
@@ -1296,7 +1359,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
+									MyLogicalRepWorker->rep_slot_id,
 									slotname,
 									NAMEDATALEN);
 
@@ -1322,12 +1385,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC ||
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY);
 
+	/*
+	 * See if tablesync of the current relation has been started with another
+	 * replication slot.
+	 *
+	 * Read previous slot name from the catalog, if exists.
+	 */
+	prev_slotname = (char *) palloc(NAMEDATALEN);
+	StartTransactionCommand();
+	GetSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+									  MyLogicalRepWorker->relid,
+									  prev_slotname);
+
 	/* Assign the origin tracking record name. */
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   sizeof(originname));
 
+	CommitTransactionCommand();
+
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC)
 	{
 		/*
@@ -1341,10 +1418,53 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		 * breakdown then it wouldn't have succeeded so trying it next time
 		 * seems like a better bet.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, slotname, true);
+		if (strlen(prev_slotname) > 0)
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);
+
+			StartTransactionCommand();
+			/* Replication origin might still exist. Try to drop */
+			replorigin_drop_by_name(originname, true, false);
+
+			/*
+			 * Remove replication slot and origin name from the relation's
+			 * catalog record
+			 */
+			UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+								  MyLogicalRepWorker->relid,
+								  MyLogicalRepWorker->relstate,
+								  MyLogicalRepWorker->relstate_lsn,
+								  NULL,
+								  NULL);
+			CommitTransactionCommand();
+			ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+						"NULL", "NULL", MyLogicalRepWorker->relstate,
+						MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+		}
 	}
 	else if (MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY)
 	{
+		/*
+		 * At this point, the table that is currently being synchronized
+		 * should have its replication slot name filled in the catalog. The
+		 * tablesync process was started with another sync worker and
+		 * replication slot. We need to continue using the same replication
+		 * slot in this worker too.
+		 */
+		if (strlen(prev_slotname) == 0)
+		{
+			elog(ERROR, "Replication slot could not be found for subscription %u, relation %u",
+				 MyLogicalRepWorker->subid,
+				 MyLogicalRepWorker->relid);
+		}
+
+		/*
+		 * Proceed with the correct replication slot. Use previously created
+		 * replication slot to sync this table.
+		 */
+		memcpy(slotname, prev_slotname, NAMEDATALEN);
+
 		/*
 		 * The COPY phase was previously done, but tablesync then crashed
 		 * before it was able to finish normally.
@@ -1364,7 +1484,9 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 		goto copy_table_done;
 	}
+	pfree(prev_slotname);
 
+	/* Preparing for table copy operation */
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 	MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
 	MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
@@ -1372,11 +1494,31 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   MyLogicalRepWorker->relstate,
-							   MyLogicalRepWorker->relstate_lsn);
+
+	/*
+	 * Refresh the originname in case of having non-existing origin
+	 * from previous failed sync attempts.
+	 * If that's the case, it should be removed from the catalog so far.
+	 * Then, we can continue by reusing the origin created by the current
+	 * worker instead of .
+	 */
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									MyLogicalRepWorker->relid,
+									originname,
+									sizeof(originname));
+
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  MyLogicalRepWorker->relstate,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					slotname, originname, MyLogicalRepWorker->relstate,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+
 	pgstat_report_stat(true);
 
 	StartTransactionCommand();
@@ -1414,6 +1556,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						GetUserNameFromId(GetUserId(), true),
 						RelationGetRelationName(rel))));
 
+
 	/*
 	 * Start a transaction in the remote node in REPEATABLE READ mode.  This
 	 * ensures that both the replication slot we create (see below) and the
@@ -1429,48 +1572,95 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						res->err)));
 	walrcv_clear_result(res);
 
+	originid = replorigin_by_name(originname, true);
+
 	/*
 	 * Create a new permanent logical decoding slot. This slot will be used
 	 * for the catchup phase after COPY is done, so tell it to use the
 	 * snapshot to make the final data consistent.
+	 *
+	 * Replication slot will only be created if either this is the first run
+	 * of the worker or we're not using a previous replication slot.
 	 */
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-
-	/*
-	 * Setup replication origin tracking. The purpose of doing this before the
-	 * copy is to avoid doing the copy again due to any error in setting up
-	 * origin tracking.
-	 */
-	originid = replorigin_by_name(originname, true);
-	if (!OidIsValid(originid))
+	if (!MyLogicalRepWorker->created_slot)
 	{
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						   slotname, false /* permanent */ , false /* two_phase */ ,
+						   CRS_USE_SNAPSHOT, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: created replication slot %s for subscription %u",
+						slotname, MyLogicalRepWorker->subid)));
+
 		/*
-		 * Origin tracking does not exist, so create it now.
-		 *
-		 * Then advance to the LSN got from walrcv_create_slot. This is WAL
-		 * logged for the purpose of recovery. Locks are to prevent the
-		 * replication origin from vanishing while advancing.
+		 * Remember that we created the slot so that we will not try to create
+		 * it again.
 		 */
-		originid = replorigin_create(originname);
-
-		LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
-		replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
-						   true /* go backward */ , true /* WAL log */ );
-		UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->created_slot = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
+		/*
+		 * Setup replication origin tracking. The purpose of doing this before
+		 * the copy is to avoid doing the copy again due to any error in
+		 * setting up origin tracking.
+		 */
+		if (!OidIsValid(originid))
+		{
+			/*
+			 * Origin tracking does not exist, so create it now.
+			 */
+			originid = replorigin_create(originname);
+		}
+		else
+		{
+			/*
+			 * At this point, there shouldn't be any existing replication
+			 * origin with the same name.
+			 */
+			ereport(ERROR,
+					(errcode(ERRCODE_DUPLICATE_OBJECT),
+					 errmsg("replication origin \"%s\" already exists",
+							originname)));
+		}
 	}
 	else
 	{
-		ereport(ERROR,
-				(errcode(ERRCODE_DUPLICATE_OBJECT),
-				 errmsg("replication origin \"%s\" already exists",
-						originname)));
+		/*
+		 * Do not create a new replication slot, reuse the existing one
+		 * instead. Use a new snapshot for the replication slot to ensure that
+		 * tablesync and apply proceses are consistent with each other.
+		 */
+		WalRcvStreamOptions options;
+		int			server_version;
+
+		server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+		options.proto.logical.proto_version =
+			server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+			server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+			LOGICALREP_PROTO_VERSION_NUM;
+		options.proto.logical.publication_names = MySubscription->publications;
+
+		walrcv_slot_snapshot(LogRepWorkerWalRcvConn, slotname, &options, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: reusing replication slot %s for relation %u in subscription %u",
+						slotname, MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
 	}
 
+	/*
+	 * Advance to the LSN got from walrcv_create_slot or walrcv_slot_snapshot.
+	 * This is WAL logged for the purpose of recovery. Locks are to prevent
+	 * the replication origin from vanishing while advancing.
+	 *
+	 * Then setup replication origin tracking.
+	 */
+	LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+	replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
+					   true /* go backward */ , true /* WAL log */ );
+	UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+
 	/* Now do the initial data copy */
 	PushActiveSnapshot(GetTransactionSnapshot());
 	copy_table(rel);
@@ -1493,12 +1683,18 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	 * Update the persisted state to indicate the COPY phase is done; make it
 	 * visible to others.
 	 */
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   SUBREL_STATE_FINISHEDCOPY,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  SUBREL_STATE_FINISHEDCOPY,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					originname, slotname, SUBREL_STATE_FINISHEDCOPY,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
 
 copy_table_done:
 
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 8ca6078db4..f7199c434d 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -464,8 +464,16 @@ ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 {
 	if (OidIsValid(relid))
 	{
-		/* Replication origin name for tablesync workers. */
-		snprintf(originname, szoriginname, "pg_%u_%u", suboid, relid);
+		bool		is_null = true;
+
+		/*
+		 * Replication origin name for tablesync workers. First, look into the
+		 * catalog. If originname does not exist, then use the default name.
+		 */
+		GetSubscriptionRelOrigin(suboid, relid,
+								 originname, &is_null);
+		if (is_null)
+			snprintf(originname, szoriginname, "pg_%u_%lld", suboid, (long long) MyLogicalRepWorker->rep_slot_id);
 	}
 	else
 	{
@@ -3760,6 +3768,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 		}
 	}
 
+	/* Pop the error context stack */
+	error_context_stack = errcallback.previous;
+	apply_error_context_stack = error_context_stack;
+
 	/*
 	 * If it's still not ready to reuse, this is probably an apply worker.
 	 * End streaming before exiting.
@@ -4477,6 +4489,9 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 
 	/* allocate slot name in long-lived context */
 	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+
+	/* Keep the replication slot name used for this sync. */
+	MyLogicalRepWorker->slot_name = *myslotname;
 	pfree(syncslotname);
 }
 
@@ -4593,11 +4608,13 @@ run_apply_worker(WalRcvStreamOptions *options,
 	replorigin_session_setup(originid, 0);
 	replorigin_session_origin = originid;
 	*origin_startpos = replorigin_session_get_progress(false);
-	CommitTransactionCommand();
 
 	/* Is the use of a password mandatory? */
-	must_use_password = MySubscription->passwordrequired &&
-		!superuser_arg(MySubscription->owner);
+		must_use_password = MySubscription->passwordrequired &&
+			!superuser_arg(MySubscription->owner);
+
+	CommitTransactionCommand();
+
 	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
 											must_use_password,
 											MySubscription->name, &err);
@@ -4857,60 +4874,87 @@ TablesyncWorkerMain(Datum main_arg)
 
 		if (MyLogicalRepWorker->ready_to_reuse)
 		{
-		/* This transaction will be committed by clean_sync_worker. */
-		StartTransactionCommand();
+			/* This transaction will be committed by clean_sync_worker. */
+			StartTransactionCommand();
 
-		/*
-		 * Check if any table whose relation state is still INIT. If a table
-		 * in INIT state is found, the worker will not be finished, it will be
-		 * reused instead.
-		 */
-		rstates = GetSubscriptionRelations(MySubscription->oid, true);
-		rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
+			/*
+			 * Check if any table whose relation state is still INIT. If a table
+			 * in INIT state is found, the worker will not be finished, it will be
+			 * reused instead.
+			 */
+			rstates = GetSubscriptionRelations(MySubscription->oid, true);
+			rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
 
-		foreach(lc, rstates)
-		{
-			memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
+			foreach(lc, rstates)
+			{
+				memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
+
+				/*
+				 * Pick the table for the next run if it is not already picked up
+				 * by another worker.
+				 *
+				 * Take exclusive lock to prevent any other sync worker from picking
+				 * the same table.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
+				if (rstate->state != SUBREL_STATE_SYNCDONE &&
+					!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+				{
+					/* Update worker state for the next table */
+					MyLogicalRepWorker->relid = rstate->relid;
+					MyLogicalRepWorker->relstate = rstate->state;
+					MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+					LWLockRelease(LogicalRepWorkerLock);
+					break;
+				}
+				LWLockRelease(LogicalRepWorkerLock);
+			}
 
 			/*
-			 * Pick the table for the next run if it is not already picked up
-			 * by another worker.
+			 * If a relation with INIT state is assigned, clean up the worker for
+			 * the next iteration.
 			 *
-			 * Take exclusive lock to prevent any other sync worker from picking
-			 * the same table.
+			 * If there is no more work left for this worker, drop repllication
+			 * slot and origin, then break the loop to exit.
 			 */
-			LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
-			if (rstate->state != SUBREL_STATE_SYNCDONE &&
-				!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+			if (MyLogicalRepWorker->relstate == SUBREL_STATE_INIT)
 			{
-				/* Update worker state for the next table */
-				MyLogicalRepWorker->relid = rstate->relid;
-				MyLogicalRepWorker->relstate = rstate->state;
-				MyLogicalRepWorker->relstate_lsn = rstate->lsn;
-				LWLockRelease(LogicalRepWorkerLock);
-				break;
+				clean_sync_worker();
 			}
-			LWLockRelease(LogicalRepWorkerLock);
-		}
+			else
+			{
+				TimeLineID	tli;
 
-		/*
-		 * If a relation with INIT state is assigned, clean up the worker for
-		 * the next iteration.
-		 *
-		 * If there is no more work left for this worker, break the loop to
-		 * exit.
-		 */
-		if ( MyLogicalRepWorker->relstate == SUBREL_STATE_INIT)
-			clean_sync_worker();
-		else
-			break;
+				/*
+				 * It is important to give an error if we are unable to drop the
+				 * slot, otherwise, it won't be dropped till the corresponding
+				 * subscription is dropped. So passing missing_ok = false.
+				 */
+				if (MyLogicalRepWorker->created_slot)
+				{
+					walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+					ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
+				}
 
-		/* If not exited yet, then the worker will sync another table. */
-		StartTransactionCommand();
-		ereport(LOG,
-				(errmsg("logical replication table synchronization worker for subscription \"%s\" has moved to sync table \"%s\" with relid %u.",
-						MySubscription->name, get_rel_name(MyLogicalRepWorker->relid), MyLogicalRepWorker->relid)));
-		CommitTransactionCommand();
+				/*
+				 * Drop replication origin before exiting.
+				 *
+				 * There is a chance that the user is concurrently performing refresh
+				 * for the subscription where we remove the table state and its origin
+				 * or the apply worker would have removed this origin. So passing
+				 * missing_ok = true.
+				 */
+				replorigin_drop_by_name(originname, true, false);
+
+				break;
+			}
+
+			/* If not exited yet, then the worker will sync another table. */
+			StartTransactionCommand();
+			ereport(LOG,
+					(errmsg("logical replication table synchronization worker for subscription \"%s\" has moved to sync table \"%s\" with relid %u.",
+							MySubscription->name, get_rel_name(MyLogicalRepWorker->relid), MyLogicalRepWorker->relid)));
+			CommitTransactionCommand();
 		}
 	}
 
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 1d40eebc78..7e13f59847 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -108,6 +108,9 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
 
 	/* Only publish data originating from the specified origin */
 	text		suborigin BKI_DEFAULT(LOGICALREP_ORIGIN_ANY);
+
+	/* The last used ID to create a replication slot for tablesync */
+	int64		sublastusedid BKI_DEFAULT(0);
 #endif
 } FormData_pg_subscription;
 
@@ -144,6 +147,8 @@ typedef struct Subscription
 	List	   *publications;	/* List of publication names to subscribe to */
 	char	   *origin;			/* Only publish data originating from the
 								 * specified origin */
+	int64		lastusedid;		/* Last used unique ID to create replication
+								 * slots in tablesync */
 } Subscription;
 
 /* Disallow streaming in-progress transactions. */
@@ -164,6 +169,7 @@ typedef struct Subscription
 extern Subscription *GetSubscription(Oid subid, bool missing_ok);
 extern void FreeSubscription(Subscription *sub);
 extern void DisableSubscription(Oid subid);
+extern void UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid);
 
 extern int	CountDBSubscriptions(Oid dbid);
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 60a2bcca23..185164d75e 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -44,6 +44,12 @@ CATALOG(pg_subscription_rel,6102,SubscriptionRelRelationId)
 											 * used for synchronization
 											 * coordination, or NULL if not
 											 * valid */
+	NameData	srrelslotname BKI_FORCE_NULL;	/* name of the replication
+												 * slot for relation in
+												 * subscription */
+	NameData	srreloriginname BKI_FORCE_NULL; /* origin name for relation in
+												 * subscription */
+
 #endif
 } FormData_pg_subscription_rel;
 
@@ -81,10 +87,16 @@ typedef struct SubscriptionRelState
 } SubscriptionRelState;
 
 extern void AddSubscriptionRelState(Oid subid, Oid relid, char state,
-									XLogRecPtr sublsn);
+									XLogRecPtr sublsn, char *relslotname, char *reloriginname);
 extern void UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 									   XLogRecPtr sublsn);
+extern void UpdateSubscriptionRel(Oid subid, Oid relid, char state,
+								  XLogRecPtr sublsn, char *relslotname, char *reloriginname);
+
 extern char GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn);
+extern void GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname);
+extern void GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull);
+
 extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 
 extern bool HasSubscriptionRelations(Oid subid);
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index a8a89dc784..31b2c41893 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -237,8 +237,9 @@ extern bool InvalidateObsoleteReplicationSlots(ReplicationSlotInvalidationCause
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
 extern bool ReplicationSlotName(int index, Name name);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, Size szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, int64 slotid, char *syncslotname, Size szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List *GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 242737e42b..6bbdf743e0 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -35,6 +35,23 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/*
+	 * Indicates if the sync worker created a replication slot for itself
+	 * in any point of its lifetime.
+	 * False means that the worker has not created a slot yet, and has been
+	 * reusing replication slots created by other workers so far.
+	 */
+	bool		created_slot;
+
+	/*
+	 * Unique identifier for replication slot to be created by tablesnync
+	 * workers, if needed.
+	 */
+	int64		rep_slot_id;
+
+	/* Replication slot name used by the worker. */
+	char	   *slot_name;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
@@ -239,7 +256,8 @@ extern LogicalRepWorker *logicalrep_worker_find(Oid subid, Oid relid,
 extern List *logicalrep_workers_find(Oid subid, bool only_running);
 extern bool logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname,
 									 Oid userid, Oid relid,
-									 dsm_handle subworker_dsm);
+									 dsm_handle subworker_dsm,
+									 int64 slotid);
 extern void logicalrep_worker_stop(Oid subid, Oid relid);
 extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
 extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
@@ -335,4 +353,7 @@ am_parallel_apply_worker(void)
 	return isParallelApplyWorker(MyLogicalRepWorker);
 }
 
+/* Invalid identifier to be used for naming replication slots */
+#define InvalidRepSlotId	0
+
 #endif							/* WORKER_INTERNAL_H */
-- 
2.25.1

#64Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Peter Smith (#62)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Peter,

Peter Smith <smithpb2250@gmail.com>, 26 May 2023 Cum, 10:30 tarihinde
şunu yazdı:

On Thu, May 25, 2023 at 6:59 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:
Yes, I was mostly referring to the same as point 1 below about patch
0001. I guess I just found the concept of mixing A) launching TSW (via
apply worker) with B) reassigning TSW to another relation (by the TSW
battling with its peers) to be a bit difficult to understand. I
thought most of the refactoring seemed to arise from choosing to do it
that way.

No, the refactoring is not related to the way of assigning a new
table. In fact, the patch did not include such refactoring a couple
versions earlier [1]/messages/by-id/CAGPVpCQmEE8BygXr=Hi2N2t2kOE=PJwofn9TX0J9J4crjoXarQ@mail.gmail.com and was still assigning tables the same way. It
was suggested here [2]/messages/by-id/CAAKRu_YKGyF+svRQqe1th-mG9xLdzneWgh9H1z1DtypBkawkkw@mail.gmail.com. Then, I made the patch 0001 which includes
some refactoring and only reuses the worker and nothing else. Also I
find it more understandable this way, maybe it's a bit subjective.

I feel that logical replication related files are getting more and
more complex and hard to understand with each change. IMHO, even
without reusing anything, those need some refactoring anyway. But for
this patch, refactoring some places made it simpler to reuse workers
and/or replication slots, regardless of how tables are assigned to
TSW's.

+1. I think it would be nice to see POC of both ways for benchmark
comparison because IMO performance is not the only consideration --
unless there is an obvious winner, then they need to be judged also by
the complexity of the logic, the amount of code that needed to be
refactored, etc.

Will try to do that. But, like I mentioned above, I don't think that
such a change would reduce the complexity or number of lines changed.

But it is difficult to get an overall picture of the behaviour. Mostly
when benchmarks were posted you hold one variable fixed and show only
one other varying. It always leaves me wondering -- what about not
empty tables, or what about different numbers of tables etc. Is it
possible to make some script to gather a bigger set of results so we
can see everything at once? Perhaps then it will become clear there is
some "sweet spot" where the patch is really good but beyond that it
degrades (actually, who knows what it might show).

I actually shared the benchmarks with different numbers of tables and
sizes. But those were all with 2 workers. I guess you want a similar
benchmark with different numbers of workers.
Will work on this and share soon.

[1]: /messages/by-id/CAGPVpCQmEE8BygXr=Hi2N2t2kOE=PJwofn9TX0J9J4crjoXarQ@mail.gmail.com
[2]: /messages/by-id/CAAKRu_YKGyF+svRQqe1th-mG9xLdzneWgh9H1z1DtypBkawkkw@mail.gmail.com

Thanks,
--
Melih Mutlu
Microsoft

#65Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#64)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Thu, Jun 1, 2023 at 7:22 AM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi Peter,

Peter Smith <smithpb2250@gmail.com>, 26 May 2023 Cum, 10:30 tarihinde
şunu yazdı:

On Thu, May 25, 2023 at 6:59 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:
Yes, I was mostly referring to the same as point 1 below about patch
0001. I guess I just found the concept of mixing A) launching TSW (via
apply worker) with B) reassigning TSW to another relation (by the TSW
battling with its peers) to be a bit difficult to understand. I
thought most of the refactoring seemed to arise from choosing to do it
that way.

No, the refactoring is not related to the way of assigning a new
table. In fact, the patch did not include such refactoring a couple
versions earlier [1] and was still assigning tables the same way. It
was suggested here [2]. Then, I made the patch 0001 which includes
some refactoring and only reuses the worker and nothing else. Also I
find it more understandable this way, maybe it's a bit subjective.

I feel that logical replication related files are getting more and
more complex and hard to understand with each change. IMHO, even
without reusing anything, those need some refactoring anyway. But for
this patch, refactoring some places made it simpler to reuse workers
and/or replication slots, regardless of how tables are assigned to
TSW's.

If refactoring is wanted anyway (regardless of the chosen "reuse"
logic), then will it be better to split off a separate 0001 patch just
to get that part out of the way first?

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#66Yu Shi (Fujitsu)
shiy.fnst@fujitsu.com
In reply to: Melih Mutlu (#63)
RE: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Thu, Jun 1, 2023 6:54 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi,

I rebased the patch and addressed the following reviews.

Thanks for updating the patch. Here are some comments on 0001 patch.

1.
- ereport(LOG,
- (errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
- MySubscription->name,
- get_rel_name(MyLogicalRepWorker->relid))));

Could we move this to somewhere else instead of removing it?

2.
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+	CommitTransactionCommand();
+
+	/* Is the use of a password mandatory? */
+	must_use_password = MySubscription->passwordrequired &&
+		!superuser_arg(MySubscription->owner);
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											must_use_password,
+											MySubscription->name, &err);

It seems that there is a problem when refactoring.
See commit e7e7da2f8d5.

3.
+	/* Set this to false for safety, in case we're already reusing the worker */
+	MyLogicalRepWorker->ready_to_reuse = false;
+

I am not sure do we need to lock when setting it.

4.
+	/*
+	 * Allocate the origin name in long-lived context for error context
+	 * message.
+	 */
+	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   originname_size);
+	CommitTransactionCommand();

Do we need the call to StartTransactionCommand() and CommitTransactionCommand()
here? Besides, the comment here is the same as the comment atop
set_apply_error_context_origin(), do we need it?

5.
I saw a segmentation fault when debugging.

It happened when calling sync_worker_exit() called (see the code below in
LogicalRepSyncTableStart()). In the case that this is not the first table the
worker synchronizes, clean_sync_worker() has been called before (in
TablesyncWorkerMain()), and LogRepWorkerWalRcvConn has been set to NULL. Then, a
segmentation fault happened because LogRepWorkerWalRcvConn is a null pointer.

switch (relstate)
{
case SUBREL_STATE_SYNCDONE:
case SUBREL_STATE_READY:
case SUBREL_STATE_UNKNOWN:
sync_worker_exit(); /* doesn't return */
}

Here is the backtrace.

#0 0x00007fc8a8ce4c95 in libpqrcv_disconnect (conn=0x0) at libpqwalreceiver.c:757
#1 0x000000000092b8c0 in clean_sync_worker () at tablesync.c:150
#2 0x000000000092b8ed in sync_worker_exit () at tablesync.c:164
#3 0x000000000092d8f6 in LogicalRepSyncTableStart (origin_startpos=0x7ffd50f30f08) at tablesync.c:1293
#4 0x0000000000934f76 in start_table_sync (origin_startpos=0x7ffd50f30f08, myslotname=0x7ffd50f30e80) at worker.c:4457
#5 0x000000000093513b in run_tablesync_worker (options=0x7ffd50f30ec0, slotname=0x0, originname=0x7ffd50f30f10 "pg_16394_16395",
originname_size=64, origin_startpos=0x7ffd50f30f08) at worker.c:4532
#6 0x0000000000935a3a in TablesyncWorkerMain (main_arg=1) at worker.c:4853
#7 0x00000000008e97f6 in StartBackgroundWorker () at bgworker.c:864
#8 0x00000000008f350b in do_start_bgworker (rw=0x12fc1a0) at postmaster.c:5762
#9 0x00000000008f38b7 in maybe_start_bgworkers () at postmaster.c:5986
#10 0x00000000008f2975 in process_pm_pmsignal () at postmaster.c:5149
#11 0x00000000008ee98a in ServerLoop () at postmaster.c:1770
#12 0x00000000008ee3bb in PostmasterMain (argc=3, argv=0x12c4af0) at postmaster.c:1463
#13 0x00000000007b6d3a in main (argc=3, argv=0x12c4af0) at main.c:198

The steps to reproduce:
Worker1, in TablesyncWorkerMain(), the relstate of new table to sync (obtained
by GetSubscriptionRelations) is SUBREL_STATE_INIT, and in the foreach loop,
before the following Check (it needs a breakpoint before locking),

LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
if (rstate->state != SUBREL_STATE_SYNCDONE &&
!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
{
/* Update worker state for the next table */
MyLogicalRepWorker->relid = rstate->relid;
MyLogicalRepWorker->relstate = rstate->state;
MyLogicalRepWorker->relstate_lsn = rstate->lsn;
LWLockRelease(LogicalRepWorkerLock);
break;
}
LWLockRelease(LogicalRepWorkerLock);

let this table to be synchronized by another table sync worker (Worker2), and
Worker2 has finished before logicalrep_worker_find was called(). Then Worker1
tried to sync a table whose state is SUBREL_STATE_READY and the segmentation
fault happened.

Regards,
Shi Yu

#67Hayato Kuroda (Fujitsu)
kuroda.hayato@fujitsu.com
In reply to: Melih Mutlu (#63)
RE: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Dear Melih,

Thank you for making the patch!
I'm also interested in the patchset. Here are the comments for 0001.

Some codes are not suit for our coding conventions, but followings do not contain them
because patches seems in the early statge.
Moreover, 0003 needs rebase.

01. general

Why do tablesync workers have to disconnect from publisher for every iterations?
I think connection initiation overhead cannot be negligible in the postgres's basic
architecture. I have not checked yet, but could we add a new replication message
like STOP_STREAMING or CLEANUP? Or, engineerings for it is quite larger than the benefit?

02. logicalrep_worker_launch()

```
-       else
+       else if (!OidIsValid(relid))
                snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
+       else
+               snprintf(bgw.bgw_function_name, BGW_MAXLEN, "TablesyncWorkerMain");
```

You changed the entry point of tablesync workers, but bgw_type is still the same.
Do you have any decisions about it?

03. process_syncing_tables_for_sync()

```
+               /*
+                * Sync worker is cleaned at this point. It's ready to sync next table,
+                * if needed.
+                */
+               SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+               MyLogicalRepWorker->ready_to_reuse = true;
+               SpinLockRelease(&MyLogicalRepWorker->relmutex);
```

Maybe acquiring the lock for modifying ready_to_reuse is not needed because all
the sync workers check only the own attribute. Moreover, other processes do not read.

04. sync_worker_exit()

```
+/*
+ * Exit routine for synchronization worker.
+ */
+void
+pg_attribute_noreturn()
+sync_worker_exit(void)
```

I think we do not have to rename the function from finish_sync_worker().

05. LogicalRepApplyLoop()

```
+                       if (MyLogicalRepWorker->ready_to_reuse)
+                       {
+                               endofstream = true;
+                       }
```

We should add comments here to clarify the reason.

06. stream_build_options()

I think we can set twophase attribute here.

07. TablesyncWorkerMain()

```
+ ListCell *lc;
```

This variable should be declared inside the loop.

08. TablesyncWorkerMain()

```
+               /*
+                * If a relation with INIT state is assigned, clean up the worker for
+                * the next iteration.
+                *
+                * If there is no more work left for this worker, break the loop to
+                * exit.
+                */
+               if ( MyLogicalRepWorker->relstate == SUBREL_STATE_INIT)
+                       clean_sync_worker();
```

The sync worker sends a signal to its leader per the iteration, but it may be too
often. Maybe it is added for changing the rstate to READY, however, it is OK to
change it when the next change have come because should_apply_changes_for_rel()
returns true even if rel->state == SUBREL_STATE_SYNCDONE. I think the notification
should be done only at the end of sync workers. How do you think?

Best Regards,
Hayato Kuroda
FUJITSU LIMITED

#68Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#63)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Here are some review comments for the patch v2-0001.

======
Commit message

1. General
Better to use consistent terms in this message. Either "relations" or
"tables" -- not a mixture of both.

~~~

2.
Before this commit, tablesync workers were capable of syncing only one
relation. For each table, a new sync worker was launched and the worker
would exit when the worker is done with the current table.

~

SUGGESTION (2nd sentence)
For each table, a new sync worker was launched and that worker would
exit when done processing the table.

~~~

3.
Now, tablesync workers are not only limited with one relation and can
move to another relation in the same subscription. This reduces the
overhead of launching a new background worker and exiting from that
worker for each relation.

~

SUGGESTION (1st sentence)
Now, tablesync workers are not limited to processing only one
relation. When done, they can move to processing another relation in
the same subscription.

~~~

4.
A new tablesync worker gets launched only if the number of tablesync
workers for the subscription does not exceed
max_sync_workers_per_subscription. If there is a table needs to be synced,
a tablesync worker picks that up and syncs it.The worker continues to
picking new tables to sync until there is no table left for synchronization
in the subscription.

~

This seems to be missing the point that only "available" workers go
looking for more tables to process. Maybe reword something like below:

SUGGESTION
If there is a table that needs to be synced, an "available" tablesync
worker picks up that table and syncs it. Each tablesync worker
continues to pick new tables to sync until there are no tables left
requiring synchronization. If there was no "available" worker to
process the table, then a new tablesync worker will be launched,
provided the number of tablesync workers for the subscription does not
exceed max_sync_workers_per_subscription.

======
src/backend/replication/logical/launcher.c

5. logicalrep_worker_launch

@@ -460,8 +461,10 @@ retry:

  if (is_parallel_apply_worker)
  snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ParallelApplyWorkerMain");
- else
+ else if (!OidIsValid(relid))
  snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
+ else
+ snprintf(bgw.bgw_function_name, BGW_MAXLEN, "TablesyncWorkerMain");

if (OidIsValid(relid))
snprintf(bgw.bgw_name, BGW_MAXLEN,

~

5a.
I felt at least these conditions can be rearranged, so you can use
OidIsValid(relid) instead of !OidIsValid(relid).

~

5b.
Probably it can all be simplified, if you are happy to do it in one line:

snprintf(bgw.bgw_function_name, BGW_MAXLEN,
OidIsValid(relid) ? "TablesyncWorkerMain" :
is_parallel_apply_worker ? "ParallelApplyWorkerMain" :
"ApplyWorkerMain");

======
src/backend/replication/logical/tablesync.c

6. finish_sync_worker

This function is removed/renamed but there are still commenting in
this file referring to 'finish-sync_worker'

~~~

7. clean_sync_worker

I agree with comment from Shi-san. There should still be logging
somewhere that say this tablesync worker has completed the processing
of the current table.

~~~

8. sync_worker_exit

There is inconsistent function naming for clean_sync_worker versus
sync_worker_exit.

How about: clean_sync_worker/exit_sync_worker?
Or: sync_worker_clean/sync_worker_exit?

~~~

9. process_syncing_tables_for_sync

@@ -378,7 +387,13 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
*/
replorigin_drop_by_name(originname, true, false);

- finish_sync_worker();
+ /*
+ * Sync worker is cleaned at this point. It's ready to sync next table,
+ * if needed.
+ */
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->ready_to_reuse = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);

9a.
I did not quite follow the logic. It says "Sync worker is cleaned at
this point", but who is doing that? -- more details are needed. But,
why not just call clean_sync_worker() right here like it use to call
finish_sync_worker?

~

9b.
Shouldn't this "ready_to_use" flag be assigned within the
clean_sync_worker() function, since that is the function making is
clean for next re-use. The function comment even says so: "Prepares
the synchronization worker for reuse or exit."

======
src/backend/replication/logical/worker.c

10. General -- run_tablesync_worker, TablesyncWorkerMain

IMO these functions would be more appropriately reside in the
tablesync.c instead of the (common) worker.c. Was there some reason
why they cannot be put there?

~~~

11. LogicalRepApplyLoop

+ /*
+ * apply_dispatch() may have gone into apply_handle_commit()
+ * which can go into process_syncing_tables_for_sync early.
+ * Before we were able to reuse tablesync workers, that
+ * process_syncing_tables_for_sync call would exit the worker
+ * instead of preparing for reuse. Now that tablesync workers
+ * can be reused and process_syncing_tables_for_sync is not
+ * responsible for exiting. We need to take care of memory
+ * contexts here before moving to sync the nex table or exit.
+ */

11a.
IMO it does not seem good to explain the reason by describing how the
logic USED to work, with code that is removed (e.g. "Before we
were..."). It's better to describe why this is needed here based on
all the CURRENT code logic.

~

11b.
/nex table/next table/

~

12.
+ if (MyLogicalRepWorker->ready_to_reuse)
+ {
+ endofstream = true;
+ }

Unnecessary parentheses.

~

13.
+ /*
+ * If it's still not ready to reuse, this is probably an apply worker.
+ * End streaming before exiting.
+ */
+ if (!MyLogicalRepWorker->ready_to_reuse)
+ {
+ /* All done */
+ walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+ }

How can we not be 100% sure of the kind of worker we are dealing with?
E.g. "probably" ??

Should this code be using macros like am_tablesync_worker() to have
some certainty what it is dealing with here?

~~~

14. stream_build_options

+ /* stream_build_options
+  * Build logical replication streaming options.
+  *
+  * This function sets streaming options including replication slot name
+  * and origin start position. Workers need these options for logical
replication.
+  */
+static void
+stream_build_options(WalRcvStreamOptions *options, char *slotname,
XLogRecPtr *origin_startpos)

The function name seem a bit strange -- it's not really "building"
anything. How about something like SetStreamOptions, or
set_stream_options.

~~~

15. run_tablesync_worker

+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+ char *slotname,
+ char *originname,
+ int originname_size,
+ XLogRecPtr *origin_startpos)
+{
+ /* Set this to false for safety, in case we're already reusing the worker */
+ MyLogicalRepWorker->ready_to_reuse = false;

Maybe reword the comment so it does not say set 'this' to false.

~

16.
+ /* Start applying changes to catcup. */
+ start_apply(*origin_startpos);

typo: catcup

~~~

17. run_apply_worker

+static void
+run_apply_worker(WalRcvStreamOptions *options,
+ char *slotname,
+ char *originname,
+ int originname_size,
+ XLogRecPtr *origin_startpos)
+{
+ /* This is the leader apply worker */
+ RepOriginId originid;
+ TimeLineID startpointTLI;
+ char    *err;
+ bool must_use_password;

The comment above the variable declarations seems redundant/misplaced.

~~

18. InitializeLogRepWorker

  if (am_tablesync_worker())
  ereport(LOG,
- (errmsg("logical replication table synchronization worker for
subscription \"%s\", table \"%s\" has started",
+ (errmsg("logical replication table synchronization worker for
subscription \"%s\", relation \"%s\" with relid %u has started",
  MySubscription->name,
- get_rel_name(MyLogicalRepWorker->relid))));
+ get_rel_name(MyLogicalRepWorker->relid),
+ MyLogicalRepWorker->relid)));
  else

I felt this code could be using get_worker_name() function like the
"else" does instead of the hardwired: "logical replication table
synchronization worker" string

~~~

19. TablesyncWorkerMain

+TablesyncWorkerMain(Datum main_arg)
+{
+ int worker_slot = DatumGetInt32(main_arg);
+ char originname[NAMEDATALEN];
+ XLogRecPtr origin_startpos = InvalidXLogRecPtr;
+ char    *myslotname = NULL;
+ WalRcvStreamOptions options;
+ List    *rstates;
+ SubscriptionRelState *rstate;
+ ListCell   *lc;
- /* Setup replication origin tracking. */
- StartTransactionCommand();
- ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-    originname, sizeof(originname));
- originid = replorigin_by_name(originname, true);
- if (!OidIsValid(originid))
- originid = replorigin_create(originname);
- replorigin_session_setup(originid, 0);
- replorigin_session_origin = originid;
- origin_startpos = replorigin_session_get_progress(false);
-
- /* Is the use of a password mandatory? */
- must_use_password = MySubscription->passwordrequired &&
- !superuser_arg(MySubscription->owner);
-
- /* Note that the superuser_arg call can access the DB */
- CommitTransactionCommand();
+ elog(LOG, "logical replication table synchronization worker has started");

Would it be better if that elog was using the common function get_worker_name()?

~~~

20.
+ if (MyLogicalRepWorker->ready_to_reuse)
+ {
+ /* This transaction will be committed by clean_sync_worker. */
+ StartTransactionCommand();

The indentation is broken.

~~~

21.
+ * Check if any table whose relation state is still INIT. If a table
+ * in INIT state is found, the worker will not be finished, it will be
+ * reused instead.
  */

First sentence is not meaningful. Should it say: "Check if there is
any table whose relation state is still INIT." ??

~~~

22.
+ /*
+ * Pick the table for the next run if it is not already picked up
+ * by another worker.
+ *
+ * Take exclusive lock to prevent any other sync worker from picking
+ * the same table.
+ */
+ LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
+ if (rstate->state != SUBREL_STATE_SYNCDONE &&
+ !logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+ {
+ /* Update worker state for the next table */
+ MyLogicalRepWorker->relid = rstate->relid;
+ MyLogicalRepWorker->relstate = rstate->state;
+ MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+ LWLockRelease(LogicalRepWorkerLock);
+ break;
+ }
+ LWLockRelease(LogicalRepWorkerLock);
  }
+
+ /*
+ * If a relation with INIT state is assigned, clean up the worker for
+ * the next iteration.
+ *
+ * If there is no more work left for this worker, break the loop to
+ * exit.
+ */
+ if ( MyLogicalRepWorker->relstate == SUBREL_STATE_INIT)
+ clean_sync_worker();
  else
- {
- walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
- }
+ break;

I was unsure about this logic, but shouldn't the
MyLogicalRepWorker->relstate be assigned a default value prior to all
these loops, so that there can be no chance for it to be
SUBREL_STATE_INIT by accident.

~

23.
+ /* If not exited yet, then the worker will sync another table. */
+ StartTransactionCommand();
+ ereport(LOG,
+ (errmsg("logical replication table synchronization worker for
subscription \"%s\" has moved to sync table \"%s\" with relid %u.",
+ MySubscription->name, get_rel_name(MyLogicalRepWorker->relid),
MyLogicalRepWorker->relid)));
+ CommitTransactionCommand();

23a
This code seems strangely structured. Why is the "not exited yet" part
not within the preceding "if" block where the clean_sync_worker was
done?

~~~

23b.
Wont it be better for that errmsg to use the common function
get_worker_name() instead of having the hardcoded string?

======
src/include/replication/worker_internal.h

24.
+ /*
+ * Used to indicate whether sync worker is ready for being reused
+ * to sync another relation.
+ */
+ bool ready_to_reuse;
+

IIUC this field has no meaning except for a tablesync worker, but the
fieldname give no indication of that at all.

To make this more obvious it might be better to put this with the
other tablesync fields:

/* Used for initial table synchronization. */
Oid relid;
char relstate;
XLogRecPtr relstate_lsn;
slock_t relmutex;
And maybe rename it according to that convention relXXX -- e.g.
'relworker_available' or something

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#69Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Peter Smith (#68)
4 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi hackers,

You can find the updated patchset attached.
I worked to address the reviews and made some additional changes.

Let me first explain the new patchset.
0001: Refactors the logical replication code, mostly worker.c and
tablesync.c. Although this patch makes it easier to reuse workers, I
believe that it's useful even by itself without other patches. It does
not improve performance or anything but aims to increase readability
and such.
0002: This is only to reuse worker processes, everything else stays
the same (replication slots/origins etc.).
0003: Adds a new command for streaming replication protocol to create
a snapshot by an existing replication slot.
0004: Reuses replication slots/origins together with workers.

Even only 0001 and 0002 are enough to improve table sync performance
at the rates previously shared on this thread. This also means that
currently 0004 (reusing replication slots/origins) does not improve as
much as I would expect, even though it does not harm either.
I just wanted to share what I did so far, while I'm continuing to
investigate it more to see what I'm missing in patch 0004.

Thanks,
--
Melih Mutlu
Microsoft

Attachments:

v14-0004-Reuse-Replication-Slot-and-Origin-in-Tablesync.patchapplication/octet-stream; name=v14-0004-Reuse-Replication-Slot-and-Origin-in-Tablesync.patchDownload
From cd58f9363144492ebc1112588b4ba6945c0f8ca0 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH 4/4] Reuse Replication Slot and Origin in Tablesync

This commit allows reusing replication slots and origins during tablesync.

Earlier, a tablesync worker was creating a new replication slot and origin
each time it syncs a new table. With this patch, replication
slots/origins can be reusable for tablesync.

This reduces the overhead of creating/dropping replication slots and origins
and improves tablesync speed significantly especially for empty or small tables.

If the state of the current table is INIT or DATASYNC, tablesync worker needs a
repliation slot/origin. If the worker has not created slot and origin in
its previous runs, it will create those first. Otherwise the worker reuses
slot and origin created by the same worker in previous iterations earlier.
Tables in FINISHEDCOPY are expected to have a replication slot and origin.
Slot and origin names for such tables are persisted in
pg_subscription_rel catalog. Tablesync worker can fetch them and proceed
with existing slot and origin of FINISHEDCOPY tables and does not need to
create new ones.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/catalogs.sgml                    |  31 ++
 src/backend/catalog/pg_subscription.c         | 247 +++++++++++-
 src/backend/commands/subscriptioncmds.c       | 229 +++++++----
 .../replication/logical/applyparallelworker.c |   3 +-
 src/backend/replication/logical/launcher.c    |   8 +-
 src/backend/replication/logical/tablesync.c   | 369 +++++++++++++-----
 src/backend/replication/logical/worker.c      |  62 ++-
 src/include/catalog/pg_subscription.h         |   6 +
 src/include/catalog/pg_subscription_rel.h     |  14 +-
 src/include/replication/slot.h                |   3 +-
 src/include/replication/worker_internal.h     |  23 +-
 11 files changed, 797 insertions(+), 198 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index ed32ca0349..0183ecd21e 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8030,6 +8030,19 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        origin.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>sublastusedid</structfield> <type>int8</type>
+      </para>
+      <para>
+      The last used ID for tablesync workers. It acts as an unique identifier
+      for replication slots which are created by tablesync workers.
+      The last used ID needs to be persisted to make logical replication safely
+      proceed after any interruption. If sublastusedid is 0, then no table has
+      been synced yet.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
@@ -8114,6 +8127,24 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        otherwise null
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srrelslotname</structfield> <type>name</type>
+      </para>
+      <para>
+       Replication slot name that is used for synchronization of relation
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srreloriginname</structfield> <type>name</type>
+      </para>
+      <para>
+       Origin name that is used for tracking synchronization of relation
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index d07f88ce28..152fdaa310 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -108,6 +108,14 @@ GetSubscription(Oid subid, bool missing_ok)
 								   Anum_pg_subscription_suborigin);
 	sub->origin = TextDatumGetCString(datum);
 
+	/* Get last used id */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
+							tup,
+							Anum_pg_subscription_sublastusedid,
+							&isnull);
+	Assert(!isnull);
+	sub->lastusedid = DatumGetInt64(datum);
+
 	ReleaseSysCache(tup);
 
 	return sub;
@@ -199,6 +207,44 @@ DisableSubscription(Oid subid)
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update the last used replication slot ID for the given subscription.
+ */
+void
+UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid)
+{
+	Relation	rel;
+	bool		nulls[Natts_pg_subscription];
+	bool		replaces[Natts_pg_subscription];
+	Datum		values[Natts_pg_subscription];
+	HeapTuple	tup;
+
+	/* Look up the subscription in the catalog */
+	rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+	tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	/* Form a new tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	replaces[Anum_pg_subscription_sublastusedid - 1] = true;
+	values[Anum_pg_subscription_sublastusedid- 1] = Int64GetDatum(lastusedid);
+
+	/* Update the catalog */
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+	heap_freetuple(tup);
+
+	table_close(rel, NoLock);
+}
+
 /*
  * Convert text array to list of strings.
  *
@@ -228,7 +274,7 @@ textarray_to_stringlist(ArrayType *textarray)
  */
 void
 AddSubscriptionRelState(Oid subid, Oid relid, char state,
-						XLogRecPtr sublsn)
+						XLogRecPtr sublsn, char *relslotname, char *reloriginname)
 {
 	Relation	rel;
 	HeapTuple	tup;
@@ -257,6 +303,16 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
 	else
 		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -269,6 +325,60 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Internal function to modify columns for relation state update
+ */
+static void
+UpdateSubscriptionRelState_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char state,
+									XLogRecPtr sublsn)
+{
+	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
+	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+
+	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (sublsn != InvalidXLogRecPtr)
+		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
+	else
+		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication slot update
+ */
+static void
+UpdateSubscriptionRelReplicationSlot_internal(Datum *values,
+											bool *nulls,
+											bool *replaces,
+											char *relslotname)
+{
+	replaces[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication origin update
+ */
+static void
+UpdateSubscriptionRelOrigin_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char *reloriginname)
+{
+	replaces[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+}
+
 /*
  * Update the state of a subscription table.
  */
@@ -299,14 +409,56 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	memset(nulls, false, sizeof(nulls));
 	memset(replaces, false, sizeof(replaces));
 
-	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
-	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
 
-	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
-	if (sublsn != InvalidXLogRecPtr)
-		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
-	else
-		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update replication slot name, origin name and state of
+ * a subscription table in one transaction.
+ */
+void
+UpdateSubscriptionRel(Oid subid,
+					  Oid relid,
+					  char state,
+					  XLogRecPtr sublsn,
+					  char *relslotname,
+					  char *reloriginname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
+	UpdateSubscriptionRelOrigin_internal(values, nulls, replaces, reloriginname);
 
 	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
 							replaces);
@@ -318,6 +470,85 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Get origin name of subscription table.
+ *
+ * reloriginname's value has the replication origin name if the origin exists.
+ */
+void
+GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*originname;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srreloriginname, isnull);
+	if (!*isnull)
+	{
+		originname = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(reloriginname, originname, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
+/*
+ * Get replication slot name of subscription table.
+ *
+ * slotname's value has the replication slot name if the subscription has any.
+ */
+void
+GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*relrepslot;
+	bool		isnull;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srrelslotname, &isnull);
+	if (!isnull)
+	{
+		relrepslot = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(slotname, relrepslot, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
 /*
  * Get state of subscription table.
  *
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 54895ba929..9f5a295ddc 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -710,6 +710,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 		publicationListToArray(publications);
 	values[Anum_pg_subscription_suborigin - 1] =
 		CStringGetTextDatum(opts.origin);
+	values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(0);
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -773,7 +774,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 										 rv->schemaname, rv->relname);
 
 				AddSubscriptionRelState(subid, relid, table_state,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 			}
 
 			/*
@@ -864,6 +865,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
 	bool		must_use_password;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -943,7 +946,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				AddSubscriptionRelState(sub->oid, relid,
 										copy_data ? SUBREL_STATE_INIT : SUBREL_STATE_READY,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 				ereport(DEBUG1,
 						(errmsg_internal("table \"%s.%s\" added to subscription \"%s\"",
 										 rv->schemaname, rv->relname, sub->name)));
@@ -967,6 +970,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				char		state;
 				XLogRecPtr	statelsn;
+				char		slotname[NAMEDATALEN] = {0};
 
 				/*
 				 * Lock pg_subscription_rel with AccessExclusiveLock to
@@ -993,13 +997,36 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/*
+				 * Find the logical replication sync worker. If exists, store
+				 * the slot number for dropping associated replication slots
+				 * later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot_name);
+				}
+				else
+				{
+					/*
+					 * Sync of this relation might be failed in an earlier
+					 * attempt, but the replication slot might still exist.
+					 */
+					GetSubscriptionRelReplicationSlot(sub->oid, relid, slotname);
+					if (strlen(slotname) > 0)
+						sub_remove_slots = lappend(sub_remove_slots, slotname);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
 				 * tablesync origin.
 				 */
-				if (state != SUBREL_STATE_READY)
+				if (state != SUBREL_STATE_READY &&
+					state != SUBREL_STATE_SYNCDONE)
 				{
 					char		originname[NAMEDATALEN];
 
@@ -1027,31 +1054,24 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
-		 * the database operations we won't be able to rollback dropped slots.
+		 * Drop the replication slots associated with tablesync workers for
+		 * removed tables. This has to be at the end because otherwise if
+		 * there is an error while doing the database operations we won't be
+		 * able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			memcpy(syncslotname, lfirst(lc), sizeof(NAMEDATALEN));
+
+			/*
+			 * There is no certainty, maybe the slot does not exist yet. Also,
+			 * if we fail after removing some of the slots, next time, it will
+			 * again try to drop already dropped slots and fail. For these
+			 * reasons, we allow missing_ok = true for the drop.
+			 */
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1474,6 +1494,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	char	   *subname;
 	char	   *conninfo;
 	char	   *slotname;
+	int64		lastusedid;
 	List	   *subworkers;
 	ListCell   *lc;
 	char		originname[NAMEDATALEN];
@@ -1546,6 +1567,14 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	else
 		slotname = NULL;
 
+	/* Get the last used identifier by the subscription */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID, tup,
+							Anum_pg_subscription_sublastusedid, &isnull);
+	if (!isnull)
+		lastusedid = DatumGetInt64(datum);
+	else
+		lastusedid = 0;
+
 	/*
 	 * Since dropping a replication slot is not transactional, the replication
 	 * slot stays dropped even if the transaction rolls back.  So we cannot
@@ -1595,6 +1624,8 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	}
 	list_free(subworkers);
 
+	rstates = GetSubscriptionRelations(subid, true);
+
 	/*
 	 * Remove the no-longer-useful entry in the launcher's table of apply
 	 * worker start times.
@@ -1606,36 +1637,26 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	ApplyLauncherForgetWorkerStartTime(subid);
 
 	/*
-	 * Cleanup of tablesync replication origins.
-	 *
-	 * Any READY-state relations would already have dealt with clean-ups.
+	 * Cleanup of tablesync replication origins associated with the
+	 * subscription, if exists. Try to drop origins by creating all origin
+	 * names created for this subscription.
 	 *
 	 * Note that the state can't change because we have already stopped both
 	 * the apply and tablesync workers and they can't restart because of
 	 * exclusive lock on the subscription.
+	 *
+	 * XXX: This can be handled better instead of looping through all possible
 	 */
-	rstates = GetSubscriptionRelations(subid, true);
-	foreach(lc, rstates)
+	for (int64 i = 1; i <= lastusedid; i++)
 	{
-		SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-		Oid			relid = rstate->relid;
-
-		/* Only cleanup resources of tablesync workers */
-		if (!OidIsValid(relid))
-			continue;
+		char		originname_to_drop[NAMEDATALEN] = {0};
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * It is possible that the origin is not yet created for tablesync
-		 * worker so passing missing_ok = true. This can happen for the states
-		 * before SUBREL_STATE_FINISHEDCOPY.
-		 */
-		ReplicationOriginNameForLogicalRep(subid, relid, originname,
-										   sizeof(originname));
-		replorigin_drop_by_name(originname, true, false);
+		snprintf(originname_to_drop, sizeof(originname_to_drop), "pg_%u_%lld", subid, (long long) i);
+		/* missing_ok = true, since the origin might be already dropped. */
+		replorigin_drop_by_name(originname_to_drop, true, false);
 	}
 
+
 	/* Clean up dependencies */
 	deleteSharedDependencyRecordsFor(SubscriptionRelationId, subid, 0);
 
@@ -1688,39 +1709,17 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
+		List	   *slots = NULL;
 
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
 
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char	   *syncslotname = (char *) lfirst(lc);
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1743,6 +1742,71 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * Get the replication slot names associated with the subscription.
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok)
+{
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+					 " FROM pg_replication_slots"
+					 " WHERE slot_name LIKE 'pg_%i_sync_%%';",
+					 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+					errmsg("could not receive list of slots associated with the subscription %u, error: %s",
+					subid, res->err));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();
+		return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -2155,6 +2219,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -2165,18 +2230,20 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
-			char		syncslotname[NAMEDATALEN] = {0};
-
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
-				 syncslotname);
+				 worker->slot_name);
 		}
 	}
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index f16e2377bf..a57f178087 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -440,7 +440,8 @@ pa_launch_parallel_worker(void)
 										MySubscription->name,
 										MyLogicalRepWorker->userid,
 										InvalidOid,
-										dsm_segment_handle(winfo->dsm_seg));
+										dsm_segment_handle(winfo->dsm_seg),
+										InvalidRepSlotId);
 
 	if (launched)
 	{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 0c6ce69c58..6dd5b9a7af 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -304,7 +304,7 @@ logicalrep_workers_find(Oid subid, bool only_running)
  */
 bool
 logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
-						 Oid relid, dsm_handle subworker_dsm)
+						 Oid relid, dsm_handle subworker_dsm, int64 slotid)
 {
 	BackgroundWorker bgw;
 	BackgroundWorkerHandle *bgw_handle;
@@ -430,6 +430,9 @@ retry:
 	worker->launch_time = now;
 	worker->in_use = true;
 	worker->generation++;
+	worker->created_slot = false;
+	worker->rep_slot_id = slotid;
+	worker->slot_name = (char *) palloc(NAMEDATALEN);
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -1180,7 +1183,8 @@ ApplyLauncherMain(Datum main_arg)
 				ApplyLauncherSetWorkerStartTime(sub->oid, now);
 				logicalrep_worker_launch(sub->dbid, sub->oid, sub->name,
 										 sub->owner, InvalidOid,
-										 DSM_HANDLE_INVALID);
+										 DSM_HANDLE_INVALID,
+										 InvalidRepSlotId);
 			}
 			else
 			{
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 37f073b968..7718287fe4 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -328,40 +328,29 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 								   MyLogicalRepWorker->relid,
 								   MyLogicalRepWorker->relstate,
 								   MyLogicalRepWorker->relstate_lsn);
+		CommitTransactionCommand();
 
 		/*
-		 * End streaming so that LogRepWorkerWalRcvConn can be used to drop
-		 * the slot.
-		 */
-		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
-
-		/*
-		 * Cleanup the tablesync slot.
+		 * Cleanup the tablesync slot. If the slot name used by this worker is
+		 * different from the default slot name for the worker, this means the
+		 * current table had started to being synchronized by another worker
+		 * and replication slot. And this worker is reusing a replication slot
+		 * from a previous attempt. We do not need that replication slot
+		 * anymore.
 		 *
 		 * This has to be done after updating the state because otherwise if
 		 * there is an error while doing the database operations we won't be
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
+										MyLogicalRepWorker->rep_slot_id,
 										syncslotname,
 										sizeof(syncslotname));
 
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
-		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
-
-		CommitTransactionCommand();
-		pgstat_report_stat(false);
-
-		/*
-		 * Start a new transaction to clean up the tablesync origin tracking.
-		 * This transaction will be ended within the finish_sync_worker().
-		 * Now, even, if we fail to remove this here, the apply worker will
-		 * ensure to clean it up afterward.
+		 * We are safe to drop the replication tracking origin after this
+		 * point. Now, even, if we fail to remove this here, the apply worker
+		 * will ensure to clean it up afterward.
 		 *
 		 * We need to do this after the table state is set to SYNCDONE.
 		 * Otherwise, if an error occurs while performing the database
@@ -370,32 +359,70 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 * have been cleared before restart. So, the restarted worker will use
 		 * invalid replication progress state resulting in replay of
 		 * transactions that have already been applied.
+		 *
+		 * Firstly reset the origin session to remove the ownership of the
+		 * slot. This is needed to allow the origin to be dropped or reused
+		 * later.
 		 */
+		replorigin_session_reset();
+		replorigin_session_origin = InvalidRepOriginId;
+		replorigin_session_origin_lsn = InvalidXLogRecPtr;
+		replorigin_session_origin_timestamp = 0;
+
 		StartTransactionCommand();
+		if (MyLogicalRepWorker->slot_name && strcmp(syncslotname, MyLogicalRepWorker->slot_name) != 0)
+		{
+			/*
+			 * End streaming so that LogRepWorkerWalRcvConn can be used to
+			 * drop the slot.
+			 */
+			walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
 
-		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
+			ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+											   MyLogicalRepWorker->relid,
+											   originname,
+											   sizeof(originname));
+
+			/*
+			 * Drop replication origin
+			 *
+			 * There is a chance that the user is concurrently performing refresh
+			 * for the subscription where we remove the table state and its origin
+			 * or the apply worker would have removed this origin. So passing
+			 * missing_ok = true.
+			 */
+			replorigin_drop_by_name(originname, true, false);
+		}
 
 		/*
-		 * Resetting the origin session removes the ownership of the slot.
-		 * This is needed to allow the origin to be dropped.
+		 * We are safe to remove persisted replication slot and origin data,
+		 * since it's already in SYNCDONE state. They will not be needed
+		 * anymore.
 		 */
-		replorigin_session_reset();
-		replorigin_session_origin = InvalidRepOriginId;
-		replorigin_session_origin_lsn = InvalidXLogRecPtr;
-		replorigin_session_origin_timestamp = 0;
+		UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+							  MyLogicalRepWorker->relid,
+							  MyLogicalRepWorker->relstate,
+							  MyLogicalRepWorker->relstate_lsn,
+							  NULL,
+							  NULL);
+		ereport(DEBUG2,
+			(errmsg("process_syncing_tables_for_sync: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					"NULL", "NULL", MyLogicalRepWorker->relstate,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+		CommitTransactionCommand();
+		pgstat_report_stat(false);
 
 		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * There is a chance that the user is concurrently performing refresh
-		 * for the subscription where we remove the table state and its origin
-		 * or the apply worker would have removed this origin. So passing
-		 * missing_ok = true.
+		 * This should return the default origin name for the worker. Even if
+		 * the worker used a different origin for this table, it should be
+		 * dropped and removed from the catalog so far.
 		 */
-		replorigin_drop_by_name(originname, true, false);
+		StartTransactionCommand();
+		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+										   MyLogicalRepWorker->relid,
+										   originname,
+										   sizeof(originname));
 
 		/* Sync worker has completed synchronization of the current table. */
 		MyLogicalRepWorker->is_sync_completed = true;
@@ -492,6 +519,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 			if (current_lsn >= rstate->lsn)
 			{
 				char		originname[NAMEDATALEN];
+				bool		is_origin_null = true;
 
 				rstate->state = SUBREL_STATE_READY;
 				rstate->lsn = current_lsn;
@@ -512,18 +540,31 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				 * error while dropping we won't restart it to drop the
 				 * origin. So passing missing_ok = true.
 				 */
-				ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-												   rstate->relid,
-												   originname,
-												   sizeof(originname));
-				replorigin_drop_by_name(originname, true, false);
+				GetSubscriptionRelOrigin(MyLogicalRepWorker->subid,
+										 rstate->relid, originname,
+										 &is_origin_null);
+
+				if (!is_origin_null)
+				{
+					replorigin_drop_by_name(originname, true, false);
+				}
 
 				/*
 				 * Update the state to READY only after the origin cleanup.
 				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-										   rstate->relid, rstate->state,
-										   rstate->lsn);
+				UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+									  rstate->relid,
+									  rstate->state,
+									  rstate->lsn,
+									  NULL,
+									  NULL);
+				ereport(DEBUG2,
+					(errmsg("process_syncing_tables_for_apply: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+							"NULL", "NULL", rstate->state,
+							rstate->relid, MyLogicalRepWorker->subid)));
+
+				CommitTransactionCommand();
+				started_tx = false;
 			}
 		}
 		else
@@ -612,12 +653,25 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						TimestampDifferenceExceeds(hentry->last_start_time, now,
 												   wal_retrieve_retry_interval))
 					{
+						if (IsTransactionState())
+							CommitTransactionCommand();
+						StartTransactionCommand();
+						started_tx = true;
+
+						MySubscription->lastusedid++;
+						UpdateSubscriptionLastSlotId(MyLogicalRepWorker->subid,
+													 MySubscription->lastusedid);
+						ereport(DEBUG2,
+								(errmsg("process_syncing_tables_for_apply: incremented lastusedid to %lld for subscription %u",
+										(long long) MySubscription->lastusedid, MySubscription->oid)));
+
 						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
 												 MySubscription->oid,
 												 MySubscription->name,
 												 MyLogicalRepWorker->userid,
 												 rstate->relid,
-												 DSM_HANDLE_INVALID);
+												 DSM_HANDLE_INVALID,
+												 MySubscription->lastusedid);
 						hentry->last_start_time = now;
 					}
 				}
@@ -1240,8 +1294,8 @@ copy_table(Relation rel)
  * The name must not exceed NAMEDATALEN - 1 because of remote node constraints
  * on slot name length. We append system_identifier to avoid slot_name
  * collision with subscriptions in other clusters. With the current scheme
- * pg_%u_sync_%u_UINT64_FORMAT (3 + 10 + 6 + 10 + 20 + '\0'), the maximum
- * length of slot_name will be 50.
+ * pg_%u_sync_%lu_UINT64_FORMAT (3 + 10 + 6 + 20 + 20 + '\0'), the maximum
+ * length of slot_name will be 45.
  *
  * The returned slot name is stored in the supplied buffer (syncslotname) with
  * the given size.
@@ -1252,11 +1306,11 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
+ReplicationSlotNameForTablesync(Oid suboid, int64 slotid,
 								char *syncslotname, Size szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%lld_" UINT64_FORMAT, suboid,
+			(long long) slotid, GetSystemIdentifier());
 }
 
 /*
@@ -1282,6 +1336,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	UserContext ucxt;
 	bool		must_use_password;
 	bool		run_as_owner;
+	char	   *prev_slotname;
 
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
@@ -1316,7 +1371,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
+									MyLogicalRepWorker->rep_slot_id,
 									slotname,
 									NAMEDATALEN);
 
@@ -1338,12 +1393,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC ||
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY);
 
+	/*
+	 * See if tablesync of the current relation has been started with another
+	 * replication slot.
+	 *
+	 * Read previous slot name from the catalog, if exists.
+	 */
+	prev_slotname = (char *) palloc(NAMEDATALEN);
+	StartTransactionCommand();
+	GetSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+									  MyLogicalRepWorker->relid,
+									  prev_slotname);
+
 	/* Assign the origin tracking record name. */
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   sizeof(originname));
 
+	CommitTransactionCommand();
+
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC)
 	{
 		/*
@@ -1357,10 +1426,53 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		 * breakdown then it wouldn't have succeeded so trying it next time
 		 * seems like a better bet.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, slotname, true);
+		if (strlen(prev_slotname) > 0)
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);
+
+			StartTransactionCommand();
+			/* Replication origin might still exist. Try to drop */
+			replorigin_drop_by_name(originname, true, false);
+
+			/*
+			 * Remove replication slot and origin name from the relation's
+			 * catalog record
+			 */
+			UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+								  MyLogicalRepWorker->relid,
+								  MyLogicalRepWorker->relstate,
+								  MyLogicalRepWorker->relstate_lsn,
+								  NULL,
+								  NULL);
+			CommitTransactionCommand();
+			ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+						"NULL", "NULL", MyLogicalRepWorker->relstate,
+						MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+		}
 	}
 	else if (MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY)
 	{
+		/*
+		 * At this point, the table that is currently being synchronized
+		 * should have its replication slot name filled in the catalog. The
+		 * tablesync process was started with another sync worker and
+		 * replication slot. We need to continue using the same replication
+		 * slot in this worker too.
+		 */
+		if (strlen(prev_slotname) == 0)
+		{
+			elog(ERROR, "Replication slot could not be found for subscription %u, relation %u",
+				 MyLogicalRepWorker->subid,
+				 MyLogicalRepWorker->relid);
+		}
+
+		/*
+		 * Proceed with the correct replication slot. Use previously created
+		 * replication slot to sync this table.
+		 */
+		memcpy(slotname, prev_slotname, NAMEDATALEN);
+
 		/*
 		 * The COPY phase was previously done, but tablesync then crashed
 		 * before it was able to finish normally.
@@ -1380,7 +1492,9 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 		goto copy_table_done;
 	}
+	pfree(prev_slotname);
 
+	/* Preparing for table copy operation */
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 	MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
 	MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
@@ -1388,11 +1502,31 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   MyLogicalRepWorker->relstate,
-							   MyLogicalRepWorker->relstate_lsn);
+
+	/*
+	 * Refresh the originname in case of having non-existing origin
+	 * from previous failed sync attempts.
+	 * If that's the case, it should be removed from the catalog so far.
+	 * Then, we can continue by reusing the origin created by the current
+	 * worker instead of .
+	 */
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									MyLogicalRepWorker->relid,
+									originname,
+									sizeof(originname));
+
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  MyLogicalRepWorker->relstate,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					slotname, originname, MyLogicalRepWorker->relstate,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+
 	pgstat_report_stat(true);
 
 	StartTransactionCommand();
@@ -1420,48 +1554,95 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						res->err)));
 	walrcv_clear_result(res);
 
+	originid = replorigin_by_name(originname, true);
+
 	/*
 	 * Create a new permanent logical decoding slot. This slot will be used
 	 * for the catchup phase after COPY is done, so tell it to use the
 	 * snapshot to make the final data consistent.
+	 *
+	 * Replication slot will only be created if either this is the first run
+	 * of the worker or we're not using a previous replication slot.
 	 */
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-
-	/*
-	 * Setup replication origin tracking. The purpose of doing this before the
-	 * copy is to avoid doing the copy again due to any error in setting up
-	 * origin tracking.
-	 */
-	originid = replorigin_by_name(originname, true);
-	if (!OidIsValid(originid))
+	if (!MyLogicalRepWorker->created_slot)
 	{
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						   slotname, false /* permanent */ , false /* two_phase */ ,
+						   CRS_USE_SNAPSHOT, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: created replication slot %s for subscription %u",
+						slotname, MyLogicalRepWorker->subid)));
+
 		/*
-		 * Origin tracking does not exist, so create it now.
-		 *
-		 * Then advance to the LSN got from walrcv_create_slot. This is WAL
-		 * logged for the purpose of recovery. Locks are to prevent the
-		 * replication origin from vanishing while advancing.
+		 * Remember that we created the slot so that we will not try to create
+		 * it again.
 		 */
-		originid = replorigin_create(originname);
-
-		LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
-		replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
-						   true /* go backward */ , true /* WAL log */ );
-		UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->created_slot = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
+		/*
+		 * Setup replication origin tracking. The purpose of doing this before
+		 * the copy is to avoid doing the copy again due to any error in
+		 * setting up origin tracking.
+		 */
+		if (!OidIsValid(originid))
+		{
+			/*
+			 * Origin tracking does not exist, so create it now.
+			 */
+			originid = replorigin_create(originname);
+		}
+		else
+		{
+			/*
+			 * At this point, there shouldn't be any existing replication
+			 * origin with the same name.
+			 */
+			ereport(ERROR,
+					(errcode(ERRCODE_DUPLICATE_OBJECT),
+					 errmsg("replication origin \"%s\" already exists",
+							originname)));
+		}
 	}
 	else
 	{
-		ereport(ERROR,
-				(errcode(ERRCODE_DUPLICATE_OBJECT),
-				 errmsg("replication origin \"%s\" already exists",
-						originname)));
+		/*
+		 * Do not create a new replication slot, reuse the existing one
+		 * instead. Use a new snapshot for the replication slot to ensure that
+		 * tablesync and apply proceses are consistent with each other.
+		 */
+		WalRcvStreamOptions options;
+		int			server_version;
+
+		server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+		options.proto.logical.proto_version =
+			server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+			server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+			LOGICALREP_PROTO_VERSION_NUM;
+		options.proto.logical.publication_names = MySubscription->publications;
+
+		walrcv_slot_snapshot(LogRepWorkerWalRcvConn, slotname, &options, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: reusing replication slot %s for relation %u in subscription %u",
+						slotname, MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
 	}
 
+	/*
+	 * Advance to the LSN got from walrcv_create_slot or walrcv_slot_snapshot.
+	 * This is WAL logged for the purpose of recovery. Locks are to prevent
+	 * the replication origin from vanishing while advancing.
+	 *
+	 * Then setup replication origin tracking.
+	 */
+	LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+	replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
+					   true /* go backward */ , true /* WAL log */ );
+	UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+
 	/*
 	 * Make sure that the copy command runs as the table owner, unless the
 	 * user has opted out of that behaviour.
@@ -1520,12 +1701,18 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	 * Update the persisted state to indicate the COPY phase is done; make it
 	 * visible to others.
 	 */
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   SUBREL_STATE_FINISHEDCOPY,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  SUBREL_STATE_FINISHEDCOPY,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					originname, slotname, SUBREL_STATE_FINISHEDCOPY,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
 
 copy_table_done:
 
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 4a0f402ad4..0b89bc4c5d 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -464,8 +464,16 @@ ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 {
 	if (OidIsValid(relid))
 	{
-		/* Replication origin name for tablesync workers. */
-		snprintf(originname, szoriginname, "pg_%u_%u", suboid, relid);
+		bool		is_null = true;
+
+		/*
+		 * Replication origin name for tablesync workers. First, look into the
+		 * catalog. If originname does not exist, then use the default name.
+		 */
+		GetSubscriptionRelOrigin(suboid, relid,
+								 originname, &is_null);
+		if (is_null)
+			snprintf(originname, szoriginname, "pg_%u_%lld", suboid, (long long) MyLogicalRepWorker->rep_slot_id);
 	}
 	else
 	{
@@ -3768,7 +3776,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* Tablesync workers should end streaming before exiting the main loop
+	/*
+	 * Tablesync workers should end streaming before exiting the main loop
 	 * to drop replication slot. Only end streaming here for apply workers.
 	 */
 	if (!am_tablesync_worker())
@@ -4494,6 +4503,9 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 
 	/* allocate slot name in long-lived context */
 	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+
+	/* Keep the replication slot name used for this sync. */
+	MyLogicalRepWorker->slot_name = *myslotname;
 	pfree(syncslotname);
 }
 
@@ -4548,10 +4560,12 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 	/* Start table synchronization. */
 	start_table_sync(origin_startpos, &slotname);
 
+	StartTransactionCommand();
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   originname_size);
+	CommitTransactionCommand();
 
 	set_apply_error_context_origin(originname);
 
@@ -4592,11 +4606,10 @@ run_apply_worker(WalRcvStreamOptions *options,
 				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
 					errmsg("subscription has no replication slot set")));
 
-	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-									   originname, originname_size);
-
 	/* Setup replication origin tracking. */
 	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
 	originid = replorigin_by_name(originname, true);
 	if (!OidIsValid(originid))
 		originid = replorigin_create(originname);
@@ -4873,12 +4886,12 @@ TablesyncWorkerMain(Datum main_arg)
 				memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
 
 				/*
-				* Pick the table for the next run if it is not already picked up
-				* by another worker.
-				*
-				* Take exclusive lock to prevent any other sync worker from picking
-				* the same table.
-				*/
+				 * Pick the table for the next run if it is not already picked up
+				 * by another worker.
+				 *
+				 * Take exclusive lock to prevent any other sync worker from picking
+				 * the same table.
+				 */
 				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
 				if (rstate->state != SUBREL_STATE_SYNCDONE &&
 					!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
@@ -4908,7 +4921,32 @@ TablesyncWorkerMain(Datum main_arg)
 			}
 
 			if (!is_table_found)
+			{
+				TimeLineID	tli;
+
+				/*
+				 * It is important to give an error if we are unable to drop the
+				 * slot, otherwise, it won't be dropped till the corresponding
+				 * subscription is dropped. So passing missing_ok = false.
+				 */
+				if (MyLogicalRepWorker->created_slot)
+				{
+					walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+					ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
+				}
+
+				/*
+				 * Drop replication origin before exiting.
+				 *
+				 * There is a chance that the user is concurrently performing refresh
+				 * for the subscription where we remove the table state and its origin
+				 * or the apply worker would have removed this origin. So passing
+				 * missing_ok = true.
+				 */
+				replorigin_drop_by_name(originname, true, false);
+
 				break;
+			}
 		}
 	}
 
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 1d40eebc78..7e13f59847 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -108,6 +108,9 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
 
 	/* Only publish data originating from the specified origin */
 	text		suborigin BKI_DEFAULT(LOGICALREP_ORIGIN_ANY);
+
+	/* The last used ID to create a replication slot for tablesync */
+	int64		sublastusedid BKI_DEFAULT(0);
 #endif
 } FormData_pg_subscription;
 
@@ -144,6 +147,8 @@ typedef struct Subscription
 	List	   *publications;	/* List of publication names to subscribe to */
 	char	   *origin;			/* Only publish data originating from the
 								 * specified origin */
+	int64		lastusedid;		/* Last used unique ID to create replication
+								 * slots in tablesync */
 } Subscription;
 
 /* Disallow streaming in-progress transactions. */
@@ -164,6 +169,7 @@ typedef struct Subscription
 extern Subscription *GetSubscription(Oid subid, bool missing_ok);
 extern void FreeSubscription(Subscription *sub);
 extern void DisableSubscription(Oid subid);
+extern void UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid);
 
 extern int	CountDBSubscriptions(Oid dbid);
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 60a2bcca23..185164d75e 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -44,6 +44,12 @@ CATALOG(pg_subscription_rel,6102,SubscriptionRelRelationId)
 											 * used for synchronization
 											 * coordination, or NULL if not
 											 * valid */
+	NameData	srrelslotname BKI_FORCE_NULL;	/* name of the replication
+												 * slot for relation in
+												 * subscription */
+	NameData	srreloriginname BKI_FORCE_NULL; /* origin name for relation in
+												 * subscription */
+
 #endif
 } FormData_pg_subscription_rel;
 
@@ -81,10 +87,16 @@ typedef struct SubscriptionRelState
 } SubscriptionRelState;
 
 extern void AddSubscriptionRelState(Oid subid, Oid relid, char state,
-									XLogRecPtr sublsn);
+									XLogRecPtr sublsn, char *relslotname, char *reloriginname);
 extern void UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 									   XLogRecPtr sublsn);
+extern void UpdateSubscriptionRel(Oid subid, Oid relid, char state,
+								  XLogRecPtr sublsn, char *relslotname, char *reloriginname);
+
 extern char GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn);
+extern void GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname);
+extern void GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull);
+
 extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 
 extern bool HasSubscriptionRelations(Oid subid);
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index a8a89dc784..31b2c41893 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -237,8 +237,9 @@ extern bool InvalidateObsoleteReplicationSlots(ReplicationSlotInvalidationCause
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
 extern bool ReplicationSlotName(int index, Name name);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, Size szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, int64 slotid, char *syncslotname, Size szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List *GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 1e9f8e6e72..24006240c9 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -35,6 +35,23 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/*
+	 * Indicates if the sync worker created a replication slot for itself
+	 * in any point of its lifetime.
+	 * False means that the worker has not created a slot yet, and has been
+	 * reusing replication slots created by other workers so far.
+	 */
+	bool		created_slot;
+
+	/*
+	 * Unique identifier for replication slot to be created by tablesnync
+	 * workers, if needed.
+	 */
+	int64		rep_slot_id;
+
+	/* Replication slot name used by the worker. */
+	char	   *slot_name;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
@@ -239,7 +256,8 @@ extern LogicalRepWorker *logicalrep_worker_find(Oid subid, Oid relid,
 extern List *logicalrep_workers_find(Oid subid, bool only_running);
 extern bool logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname,
 									 Oid userid, Oid relid,
-									 dsm_handle subworker_dsm);
+									 dsm_handle subworker_dsm,
+									 int64 slotid);
 extern void logicalrep_worker_stop(Oid subid, Oid relid);
 extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
 extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
@@ -335,4 +353,7 @@ am_parallel_apply_worker(void)
 	return isParallelApplyWorker(MyLogicalRepWorker);
 }
 
+/* Invalid identifier to be used for naming replication slots */
+#define InvalidRepSlotId	0
+
 #endif							/* WORKER_INTERNAL_H */
-- 
2.25.1

0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchapplication/octet-stream; name=0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchDownload
From 81c38df18c7038dcfcabcc396e484d16b4680d51 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Mon, 5 Jun 2023 15:04:41 +0300
Subject: [PATCH 1/4] Refactor to split Apply and Tablesync Workers

Both apply and tablesync workers were using ApplyWorkerMain() as entry
point. As the name implies, ApplyWorkerMain() should be considered as
the main function for apply workers. Tablesync worker's path was hidden
and does not have enough in common to share the same main function with
apply worker.

Also; most of the code shared by both worker types are already combined
in LogicalRepApplyLoop(). There is no need to combine the rest in
ApplyWorkerMain() anymore.

This commit introduces TablesyncWorkerMain() as a new entry point for
tablesync workers and separates both type of workers from each other.
This aims to increase code readability and help to maintain logical
replication workers separately.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/postmaster/bgworker.c             |   3 +
 .../replication/logical/applyparallelworker.c |   2 +-
 src/backend/replication/logical/launcher.c    |  25 +-
 src/backend/replication/logical/tablesync.c   |   2 +-
 src/backend/replication/logical/worker.c      | 381 +++++++++++-------
 src/include/replication/logicalworker.h       |   1 +
 src/include/replication/worker_internal.h     |   4 +-
 7 files changed, 258 insertions(+), 160 deletions(-)

diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 0dd22b2351..5609919edf 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -131,6 +131,9 @@ static const struct
 	},
 	{
 		"ParallelApplyWorkerMain", ParallelApplyWorkerMain
+	},
+	{
+		"TablesyncWorkerMain", TablesyncWorkerMain
 	}
 };
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 82c1ddcdcb..f16e2377bf 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -942,7 +942,7 @@ ParallelApplyWorkerMain(Datum main_arg)
 	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
 		MyLogicalRepWorker->reply_time = 0;
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 87b5593d2d..c2bba3ba69 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -459,24 +459,27 @@ retry:
 	snprintf(bgw.bgw_library_name, BGW_MAXLEN, "postgres");
 
 	if (is_parallel_apply_worker)
+	{
 		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ParallelApplyWorkerMain");
-	else
-		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
-
-	if (OidIsValid(relid))
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication worker for subscription %u sync %u", subid, relid);
-	else if (is_parallel_apply_worker)
+				 "logical replication parallel apply worker for subscription %u", subid);
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
 				 "logical replication parallel apply worker for subscription %u", subid);
+	}
+	else if (OidIsValid(relid))
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "TablesyncWorkerMain");
+		snprintf(bgw.bgw_name, BGW_MAXLEN,
+				 "logical replication tablesync worker for subscription %u sync %u", subid, relid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication tablesync worker");
+	}
 	else
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
 				 "logical replication apply worker for subscription %u", subid);
-
-	if (is_parallel_apply_worker)
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
-	else
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication worker");
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication apply worker");
+	}
 
 	bgw.bgw_restart_time = BGW_NEVER_RESTART;
 	bgw.bgw_notify_pid = MyProcPid;
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 6d461654ab..8125bbd170 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -131,7 +131,7 @@ static StringInfo copybuf = NULL;
 /*
  * Exit routine for synchronization worker.
  */
-static void
+void
 pg_attribute_noreturn()
 finish_sync_worker(void)
 {
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 0ee764d68f..b979a755ae 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -392,6 +392,7 @@ static void stream_open_file(Oid subid, TransactionId xid,
 static void stream_write_change(char action, StringInfo s);
 static void stream_open_and_write_change(TransactionId xid, char action, StringInfo s);
 static void stream_close_file(void);
+static void set_stream_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
@@ -4330,6 +4331,69 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+ /* set_stream_options
+  * 	Set logical replication streaming options.
+  *
+  * This function sets streaming options including replication slot name
+  * and origin start position. Workers need these options for logical replication.
+  */
+static void
+set_stream_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+		options->proto.logical.twophase = true;
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4442,13 +4506,134 @@ start_apply(XLogRecPtr origin_startpos)
 }
 
 /*
- * Common initialization for leader apply worker and parallel apply worker.
+ * Runs the tablesync worker.
+ * It starts syncing tables. After a successful sync,
+ * sets streaming options and starts streaming to catchup.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Start table synchronization. */
+	start_table_sync(origin_startpos, &slotname);
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   originname_size);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	/* Start applying changes to catchup. */
+	start_apply(*origin_startpos);
+}
+
+/*
+ * Runs the leader apply worker.
+ * It sets up replication origin, streaming options
+ * and then starts streaming.
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
+{
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+	bool		must_use_password;
+
+	slotname = MySubscription->slotname;
+
+	/*
+	 * This shouldn't happen if the subscription is enabled, but guard
+	 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+	 * crash if slot is NULL.)
+	 */
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+
+	/* Is the use of a password mandatory? */
+	must_use_password = MySubscription->passwordrequired &&
+		!superuser_arg(MySubscription->owner);
+
+	/* Note that the superuser_arg call can access the DB */
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											must_use_password,
+											MySubscription->name, &err);
+
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+	 * We don't really use the output identify_system for anything but it
+	 * does some initializations on the upstream so let's still call it.
+	 */
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+	{
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
+	}
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+
+	/* Run the main loop. */
+	start_apply(*origin_startpos);
+}
+
+/*
+ * Common initialization for logical replication workers; leader apply worker,
+ * parallel apply worker and tablesync worker.
  *
  * Initialize the database connection, in-memory subscription and necessary
  * config options.
  */
 void
-InitializeApplyWorker(void)
+InitializeLogRepWorker(void)
 {
 	MemoryContext oldctx;
 
@@ -4512,7 +4697,8 @@ InitializeApplyWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has started",
+				(errmsg("%s for subscription \"%s\", table \"%s\" has started",
+						get_worker_name(),
 						MySubscription->name,
 						get_rel_name(MyLogicalRepWorker->relid))));
 	else
@@ -4533,7 +4719,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	InitializingApplyWorker = true;
 
@@ -4557,7 +4742,7 @@ ApplyWorkerMain(Datum main_arg)
 	/* Load the libpq-specific functions */
 	load_file("libpqwalreceiver", false);
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
@@ -4565,165 +4750,69 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-		bool		must_use_password;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-
-		/* Is the use of a password mandatory? */
-		must_use_password = MySubscription->passwordrequired &&
-			!superuser_arg(MySubscription->owner);
-
-		/* Note that the superuser_arg call can access the DB */
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												must_use_password,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
-	 * the subscription relation state.
+	 * the subscription relation state. Do this outside the loop to avoid
+	 * exceeding MAX_SYSCACHE_CALLBACKS
 	 */
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
+	/* This is leader apply worker */
+	run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
+	proc_exit(0);
+}
 
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
+/* Logical Replication Tablesync worker entry point */
+void
+TablesyncWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *myslotname = NULL;
+	WalRcvStreamOptions options;
+
+	/* Attach to slot */
+	logicalrep_worker_attach(worker_slot);
+
+	/* Setup signal handling */
+	pqsignal(SIGHUP, SignalHandlerForConfigReload);
+	pqsignal(SIGTERM, die);
+	BackgroundWorkerUnblockSignals();
 
 	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 * We don't currently need any ResourceOwner in a walreceiver process, but
+	 * if we did, we could call CreateAuxProcessResourceOwner here.
 	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
 
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
+	/* Initialise stats to a sanish value */
+	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
+		MyLogicalRepWorker->reply_time = GetCurrentTimestamp();
 
-	if (!am_tablesync_worker())
-	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
-		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+	/* Load the libpq-specific functions */
+	load_file("libpqwalreceiver", false);
 
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
-		}
-		else
-		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-		}
+	InitializeLogRepWorker();
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-								 MySubscription->name,
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-								 "?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
+	/* Connect to the origin and start the replication. */
+	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
+		 MySubscription->conninfo);
 
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+	/*
+	 * Setup callback for syscache so that we know when something changes in
+	 * the subscription relation state. Do this outside the loop to avoid
+	 * exceeding MAX_SYSCACHE_CALLBACKS
+	 */
+	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
+								  invalidate_syncing_table_states,
+								  (Datum) 0);
 
-	proc_exit(0);
+	run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
+
+	finish_sync_worker();
 }
 
 /*
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 39588da79f..bbd71d0b42 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -18,6 +18,7 @@ extern PGDLLIMPORT volatile sig_atomic_t ParallelApplyMessagePending;
 
 extern void ApplyWorkerMain(Datum main_arg);
 extern void ParallelApplyWorkerMain(Datum main_arg);
+extern void TablesyncWorkerMain(Datum main_arg);
 
 extern bool IsLogicalWorker(void);
 extern bool IsLogicalParallelApplyWorker(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 343e781896..7aba034774 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -265,7 +265,7 @@ extern void maybe_reread_subscription(void);
 
 extern void stream_cleanup_files(Oid subid, TransactionId xid);
 
-extern void InitializeApplyWorker(void);
+extern void InitializeLogRepWorker(void);
 
 extern void store_flush_position(XLogRecPtr remote_lsn, XLogRecPtr local_lsn);
 
@@ -307,6 +307,8 @@ extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
 
 #define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)
 
+extern void finish_sync_worker(void);
+
 static inline bool
 am_tablesync_worker(void)
 {
-- 
2.25.1

v3-0002-Reuse-Tablesync-Workers.patchapplication/octet-stream; name=v3-0002-Reuse-Tablesync-Workers.patchDownload
From dcfc3094dbb40550a5055496d156a87b8e3e2065 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Mon, 5 Jun 2023 15:45:29 +0300
Subject: [PATCH 2/4] Reuse Tablesync Workers

This commit allows reusing tablesync workers for syncing more than one
table sequentially during their lifetime, instead of exiting after
only syncing one table.

Before this commit, tablesync workers were capable of syncing only one
table. For each table, a new sync worker was launched and that worker would
exit when done processing the table.

Now, tablesync workers are not limited to processing only one
table. When done, they can move to processing another table in
the same subscription.

If there is a table that needs to be synced, an available tablesync
worker picks up that table and syncs it. Each tablesync worker
continues to pick new tables to sync until there are no tables left
requiring synchronization. If there was no available worker to
process the table, then a new tablesync worker will be launched,
provided the number of tablesync workers for the subscription does not
exceed max_sync_workers_per_subscription.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/replication/logical/launcher.c  |   1 +
 src/backend/replication/logical/tablesync.c |  46 ++++++--
 src/backend/replication/logical/worker.c    | 110 +++++++++++++++++++-
 src/include/replication/worker_internal.h   |   7 ++
 4 files changed, 149 insertions(+), 15 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index c2bba3ba69..0c6ce69c58 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -440,6 +440,7 @@ retry:
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
+	worker->is_sync_completed = false;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 8125bbd170..37f073b968 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -129,11 +129,10 @@ static bool FetchTableStates(bool *started_tx);
 static StringInfo copybuf = NULL;
 
 /*
- * Exit routine for synchronization worker.
+ * Prepares the synchronization worker for reuse or exit.
  */
 void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+clean_sync_worker(void)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -145,19 +144,38 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
+	/*
+	 * Disconnect from publisher. Otherwise reused sync workers causes
+	 * exceeding max_wal_senders
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		walrcv_disconnect(LogRepWorkerWalRcvConn);
+		LogRepWorkerWalRcvConn = NULL;
+	}
+
+	/* Find the leader apply worker and signal it. */
+	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
+}
+
+/*
+ * Exit routine for synchronization worker.
+ */
+void
+pg_attribute_noreturn()
+finish_sync_worker(void)
+{
+	clean_sync_worker();
+
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
 	StartTransactionCommand();
 	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
+			(errmsg("logical replication table synchronization worker for subscription \"%s\" has finished",
+					MySubscription->name)));
 	CommitTransactionCommand();
 
-	/* Find the leader apply worker and signal it. */
-	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
-
 	/* Stop gracefully */
 	proc_exit(0);
 }
@@ -379,7 +397,15 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 */
 		replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
+		/* Sync worker has completed synchronization of the current table. */
+		MyLogicalRepWorker->is_sync_completed = true;
+
+		ereport(LOG,
+		(errmsg("logical replication table synchronization worker for subscription \"%s\", relation \"%s\" with relid %u has finished",
+				MySubscription->name,
+				get_rel_name(MyLogicalRepWorker->relid),
+				MyLogicalRepWorker->relid)));
+		CommitTransactionCommand();
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index b979a755ae..4a0f402ad4 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3624,6 +3624,20 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can call process_syncing_tables_for_sync.
+				 *
+				 * process_syncing_tables_for_sync decides whether the sync of the
+				 * current table is completed. If it is completed, streaming must
+				 * be already ended. So, we can break the loop.
+				 */
+				if (MyLogicalRepWorker->is_sync_completed)
+				{
+					endofstream = true;
+					break;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -3643,6 +3657,15 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+
+			/*
+			 * If is_sync_completed is true, this means that the tablesync worker
+			 * is done with synchronization. Streaming has already been ended by
+			 * process_syncing_tables_for_sync. We should move to the next table
+			 * if needed, or exit.
+			 */
+			if (MyLogicalRepWorker->is_sync_completed)
+				endofstream = true;
 		}
 
 		/* Cleanup the memory. */
@@ -3745,8 +3768,11 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/* Tablesync workers should end streaming before exiting the main loop
+	 * to drop replication slot. Only end streaming here for apply workers.
+	 */
+	if (!am_tablesync_worker())
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
 }
 
 /*
@@ -4517,6 +4543,8 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 					 int originname_size,
 					 XLogRecPtr *origin_startpos)
 {
+	MyLogicalRepWorker->is_sync_completed = false;
+
 	/* Start table synchronization. */
 	start_table_sync(origin_startpos, &slotname);
 
@@ -4697,10 +4725,11 @@ InitializeLogRepWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("%s for subscription \"%s\", table \"%s\" has started",
+				(errmsg("%s for subscription \"%s\", table \"%s\" with relid %u has started",
 						get_worker_name(),
 						MySubscription->name,
-						get_rel_name(MyLogicalRepWorker->relid))));
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
 	else
 		ereport(LOG,
 		/* translator: first %s is the name of logical replication worker */
@@ -4810,7 +4839,78 @@ TablesyncWorkerMain(Datum main_arg)
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
+	/*
+	 * The loop where worker does its job. It loops until there is no relation
+	 * left to sync.
+	 */
+	for (;;)
+	{
+		List	   *rstates;
+		SubscriptionRelState *rstate;
+		ListCell   *lc;
+		bool 	is_table_found = false;
+
+		run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
+
+		if (IsTransactionState())
+			CommitTransactionCommand();
+
+		if (MyLogicalRepWorker->is_sync_completed)
+		{
+			/* This transaction will be committed by clean_sync_worker. */
+			StartTransactionCommand();
+
+			/*
+			 * Check if there is any table whose relation state is still INIT.
+			 * If a table in INIT state is found, the worker will not be finished,
+			 * it will be reused instead.
+			 */
+			rstates = GetSubscriptionRelations(MySubscription->oid, true);
+			rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
+
+			foreach(lc, rstates)
+			{
+				memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
+
+				/*
+				* Pick the table for the next run if it is not already picked up
+				* by another worker.
+				*
+				* Take exclusive lock to prevent any other sync worker from picking
+				* the same table.
+				*/
+				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
+				if (rstate->state != SUBREL_STATE_SYNCDONE &&
+					!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+				{
+					/* Update worker state for the next table */
+					MyLogicalRepWorker->relid = rstate->relid;
+					MyLogicalRepWorker->relstate = rstate->state;
+					MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+					LWLockRelease(LogicalRepWorkerLock);
+
+					/* found a table for next iteration */
+					is_table_found = true;
+					clean_sync_worker();
+
+					StartTransactionCommand();
+					ereport(LOG,
+							(errmsg("%s for subscription \"%s\" has moved to sync table \"%s\" with relid %u.",
+									get_worker_name(),
+									MySubscription->name,
+									get_rel_name(MyLogicalRepWorker->relid),
+									MyLogicalRepWorker->relid)));
+					CommitTransactionCommand();
+
+					break;
+				}
+				LWLockRelease(LogicalRepWorkerLock);
+			}
+
+			if (!is_table_found)
+				break;
+		}
+	}
 
 	finish_sync_worker();
 }
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 7aba034774..1e9f8e6e72 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -56,6 +56,12 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/*
+	 * Indicates whether tablesync worker has completed sycning its assigned
+	 * table. If true, no need to continue with that table.
+	 */
+	bool		is_sync_completed;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction or
@@ -308,6 +314,7 @@ extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
 #define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)
 
 extern void finish_sync_worker(void);
+extern void clean_sync_worker(void);
 
 static inline bool
 am_tablesync_worker(void)
-- 
2.25.1

v11-0003-Add-replication-protocol-cmd-to-create-a-snapshot.patchapplication/octet-stream; name=v11-0003-Add-replication-protocol-cmd-to-create-a-snapshot.patchDownload
From 6a0fd4f6678fc6f5ee7f3792edf6ffcfcf10d26a Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 13 Oct 2022 17:05:45 +0300
Subject: [PATCH 3/4] Add replication protocol cmd to create a snapshot

Introduced CREATE_REPLICATION_SNAPSHOT to be able to create and use a
snapshot without creating a new replication slot, but by using an
existing slot.

CREATE_REPLICATION_SNAPSHOT simply does what CREATE_REPLICATION_SLOT does
without creating a new replication slot.

CREATE_REPLICATION_SNAPSHOT command imports the snapshot into the current
transaction and returns consistent_point. The changes earlier than the
consistent_point will be applied by importing the snapshot. All changes
later than the consistent_point will be available to be consumed from
the replication slot.

This is useful for reusing replication slots in logical replication.
Otherwise, tablesync workers cannot start from a consistent point to copy
a relation and then apply changes by consuming from replication slot.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/protocol.sgml                    |  31 ++++++
 .../libpqwalreceiver/libpqwalreceiver.c       |  69 +++++++++++-
 src/backend/replication/logical/logical.c     |  40 ++++++-
 .../replication/logical/logicalfuncs.c        |   1 +
 src/backend/replication/repl_gram.y           |  18 ++-
 src/backend/replication/repl_scanner.l        |   2 +
 src/backend/replication/slotfuncs.c           |   1 +
 src/backend/replication/walsender.c           | 104 +++++++++++++++++-
 src/include/nodes/replnodes.h                 |  11 ++
 src/include/replication/logical.h             |   1 +
 src/include/replication/walreceiver.h         |  13 +++
 src/tools/pgindent/typedefs.list              |   2 +
 12 files changed, 289 insertions(+), 4 deletions(-)

diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index b11d9a6ba3..f7d5acee93 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2595,6 +2595,37 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
      </listitem>
     </varlistentry>
 
+    <varlistentry id="protocol-replication-replication-slot-snapshot">
+     <term><literal>CREATE_REPLICATION_SNAPSHOT</literal> <replaceable class="parameter">slot_name</replaceable> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
+      <indexterm><primary>CREATE_REPLICATION_SNAPSHOT</primary></indexterm>
+     </term>
+     <listitem>
+      <para>
+       Creates a snapshot including all the changes from the replication slot until
+       the point at which the replication slot becomes consistent. Then the snapshot
+       is used in the current transaction. This command is currently only supported
+       for logical replication slots.
+      </para>
+
+      <para>
+       In response to this command, the server will return a one-row result set,
+       containing the following field:
+       <variablelist>
+        <varlistentry>
+         <term><literal>consistent_point</literal> (<type>text</type>)</term>
+         <listitem>
+          <para>
+           The WAL location at which the slot became consistent.  This is the
+           earliest location from which streaming can start on this replication
+           slot.
+          </para>
+         </listitem>
+        </varlistentry>
+       </variablelist>
+      </para>
+     </listitem>
+    </varlistentry>
+
     <varlistentry id="protocol-replication-base-backup" xreflabel="BASE_BACKUP">
      <term><literal>BASE_BACKUP</literal> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
       <indexterm><primary>BASE_BACKUP</primary></indexterm>
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index dc9c5c82d9..cf88a3b7d4 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -82,6 +82,8 @@ static WalRcvExecResult *libpqrcv_exec(WalReceiverConn *conn,
 									   const int nRetTypes,
 									   const Oid *retTypes);
 static void libpqrcv_disconnect(WalReceiverConn *conn);
+static void libpqrcv_slot_snapshot(WalReceiverConn *conn, char *slotname,
+								   const WalRcvStreamOptions *options, XLogRecPtr *lsn);
 
 static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_connect = libpqrcv_connect,
@@ -98,7 +100,8 @@ static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_create_slot = libpqrcv_create_slot,
 	.walrcv_get_backend_pid = libpqrcv_get_backend_pid,
 	.walrcv_exec = libpqrcv_exec,
-	.walrcv_disconnect = libpqrcv_disconnect
+	.walrcv_disconnect = libpqrcv_disconnect,
+	.walrcv_slot_snapshot = libpqrcv_slot_snapshot
 };
 
 /* Prototypes for private functions */
@@ -982,6 +985,70 @@ libpqrcv_create_slot(WalReceiverConn *conn, const char *slotname,
 	return snapshot;
 }
 
+/*
+ * TODO
+ */
+static void
+libpqrcv_slot_snapshot(WalReceiverConn *conn,
+					   char *slotname,
+					   const WalRcvStreamOptions *options,
+					   XLogRecPtr *lsn)
+{
+	StringInfoData cmd;
+	PGresult   *res;
+	char	   *pubnames_str;
+	List	   *pubnames;
+	char	   *pubnames_literal;
+
+	initStringInfo(&cmd);
+
+	/* Build the command. */
+	appendStringInfo(&cmd, "CREATE_REPLICATION_SNAPSHOT \"%s\"", slotname);
+	appendStringInfoString(&cmd, " (");
+	appendStringInfo(&cmd, " proto_version '%u'",
+					 options->proto.logical.proto_version);
+
+	/* Add publication names. */
+	pubnames = options->proto.logical.publication_names;
+	pubnames_str = stringlist_to_identifierstr(conn->streamConn, pubnames);
+	if (!pubnames_str)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	pubnames_literal = PQescapeLiteral(conn->streamConn, pubnames_str,
+									   strlen(pubnames_str));
+	if (!pubnames_literal)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	appendStringInfo(&cmd, ", publication_names %s", pubnames_literal);
+	PQfreemem(pubnames_literal);
+	pfree(pubnames_str);
+
+	appendStringInfoString(&cmd, " )");
+
+	/* Execute the command. */
+	res = libpqrcv_PQexec(conn->streamConn, cmd.data);
+	pfree(cmd.data);
+
+	if (PQresultStatus(res) != PGRES_TUPLES_OK)
+	{
+		PQclear(res);
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg("Could not create a snapshot by replication slot \"%s\": %s",
+						slotname, pchomp(PQerrorMessage(conn->streamConn)))));
+	}
+
+	if (lsn)
+		*lsn = DatumGetLSN(DirectFunctionCall1Coll(pg_lsn_in, InvalidOid,
+												   CStringGetDatum(PQgetvalue(res, 0, 0))));
+
+	PQclear(res);
+}
+
 /*
  * Return PID of remote backend process.
  */
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 41243d0187..3c11f8e1dd 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -476,6 +476,10 @@ CreateInitDecodingContext(const char *plugin,
  * fast_forward
  *		bypass the generation of logical changes.
  *
+ * need_full_snapshot
+ * 		if true, must obtain a snapshot able to read all tables;
+ *  	if false, one that can read only catalogs is acceptable.
+ *
  * xl_routine
  *		XLogReaderRoutine used by underlying xlogreader
  *
@@ -494,6 +498,7 @@ LogicalDecodingContext *
 CreateDecodingContext(XLogRecPtr start_lsn,
 					  List *output_plugin_options,
 					  bool fast_forward,
+					  bool need_full_snapshot,
 					  XLogReaderRoutine *xl_routine,
 					  LogicalOutputPluginWriterPrepareWrite prepare_write,
 					  LogicalOutputPluginWriterWrite do_write,
@@ -502,6 +507,7 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 	LogicalDecodingContext *ctx;
 	ReplicationSlot *slot;
 	MemoryContext old_context;
+	TransactionId xmin_horizon = InvalidTransactionId;
 
 	/* shorter lines... */
 	slot = MyReplicationSlot;
@@ -571,8 +577,40 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 		start_lsn = slot->data.confirmed_flush;
 	}
 
+
+	/*
+	 * We need to determine a safe xmin horizon to start decoding from if we
+	 * want to create a snapshot too. Otherwise we would end up with a
+	 * snapshot that cannot be imported since xmin value from the snapshot may
+	 * be less than the oldest safe xmin. To avoid this call
+	 * GetOldestSafeDecodingTransactionId() to return a safe xmin value, which
+	 * can be used while exporting/importing the snapshot.
+	 *
+	 * So we have to acquire the ProcArrayLock to prevent computation of new
+	 * xmin horizons by other backends, get the safe decoding xid, and inform
+	 * the slot machinery about the new limit. Once that's done the
+	 * ProcArrayLock can be released as the slot machinery now is protecting
+	 * against vacuum.
+	 */
+	if (need_full_snapshot)
+	{
+		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+		xmin_horizon = GetOldestSafeDecodingTransactionId(!need_full_snapshot);
+
+		SpinLockAcquire(&slot->mutex);
+		slot->effective_catalog_xmin = xmin_horizon;
+		slot->data.catalog_xmin = xmin_horizon;
+		slot->effective_xmin = xmin_horizon;
+		SpinLockRelease(&slot->mutex);
+
+		ReplicationSlotsComputeRequiredXmin(true);
+
+		LWLockRelease(ProcArrayLock);
+	}
+
 	ctx = StartupDecodingContext(output_plugin_options,
-								 start_lsn, InvalidTransactionId, false,
+								 start_lsn, xmin_horizon, need_full_snapshot,
 								 fast_forward, xl_routine, prepare_write,
 								 do_write, update_progress);
 
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index 55a24c02c9..85c5cdb633 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -208,6 +208,7 @@ pg_logical_slot_get_changes_guts(FunctionCallInfo fcinfo, bool confirm, bool bin
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									options,
 									false,
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/repl_gram.y b/src/backend/replication/repl_gram.y
index 0c874e33cf..957a5cc653 100644
--- a/src/backend/replication/repl_gram.y
+++ b/src/backend/replication/repl_gram.y
@@ -65,6 +65,7 @@ Node *replication_parse_result;
 %token K_CREATE_REPLICATION_SLOT
 %token K_DROP_REPLICATION_SLOT
 %token K_TIMELINE_HISTORY
+%token K_CREATE_REPLICATION_SNAPSHOT
 %token K_WAIT
 %token K_TIMELINE
 %token K_PHYSICAL
@@ -80,7 +81,7 @@ Node *replication_parse_result;
 %type <node>	command
 %type <node>	base_backup start_replication start_logical_replication
 				create_replication_slot drop_replication_slot identify_system
-				read_replication_slot timeline_history show
+				read_replication_slot timeline_history show create_replication_snapshot
 %type <list>	generic_option_list
 %type <defelt>	generic_option
 %type <uintval>	opt_timeline
@@ -114,6 +115,7 @@ command:
 			| read_replication_slot
 			| timeline_history
 			| show
+			| create_replication_snapshot
 			;
 
 /*
@@ -307,6 +309,19 @@ timeline_history:
 				}
 			;
 
+/*
+ * CREATE_REPLICATION_SNAPSHOT %s options
+ */
+create_replication_snapshot:
+			K_CREATE_REPLICATION_SNAPSHOT var_name plugin_options
+				{
+					CreateReplicationSnapshotCmd *n = makeNode(CreateReplicationSnapshotCmd);
+					n->slotname = $2;
+					n->options = $3;
+					$$ = (Node *) n;
+				}
+			;
+
 opt_physical:
 			K_PHYSICAL
 			| /* EMPTY */
@@ -400,6 +415,7 @@ ident_or_keyword:
 			| K_CREATE_REPLICATION_SLOT	{ $$ = "create_replication_slot"; }
 			| K_DROP_REPLICATION_SLOT		{ $$ = "drop_replication_slot"; }
 			| K_TIMELINE_HISTORY			{ $$ = "timeline_history"; }
+			| K_CREATE_REPLICATION_SNAPSHOT	{ $$ = "create_replication_snapshot"; }
 			| K_WAIT						{ $$ = "wait"; }
 			| K_TIMELINE					{ $$ = "timeline"; }
 			| K_PHYSICAL					{ $$ = "physical"; }
diff --git a/src/backend/replication/repl_scanner.l b/src/backend/replication/repl_scanner.l
index cb467ca46f..5ba2e9c54b 100644
--- a/src/backend/replication/repl_scanner.l
+++ b/src/backend/replication/repl_scanner.l
@@ -126,6 +126,7 @@ START_REPLICATION	{ return K_START_REPLICATION; }
 CREATE_REPLICATION_SLOT		{ return K_CREATE_REPLICATION_SLOT; }
 DROP_REPLICATION_SLOT		{ return K_DROP_REPLICATION_SLOT; }
 TIMELINE_HISTORY	{ return K_TIMELINE_HISTORY; }
+CREATE_REPLICATION_SNAPSHOT	{ return K_CREATE_REPLICATION_SNAPSHOT; }
 PHYSICAL			{ return K_PHYSICAL; }
 RESERVE_WAL			{ return K_RESERVE_WAL; }
 LOGICAL				{ return K_LOGICAL; }
@@ -303,6 +304,7 @@ replication_scanner_is_replication_command(void)
 		case K_DROP_REPLICATION_SLOT:
 		case K_READ_REPLICATION_SLOT:
 		case K_TIMELINE_HISTORY:
+		case K_CREATE_REPLICATION_SNAPSHOT:
 		case K_SHOW:
 			/* Yes; push back the first token so we can parse later. */
 			repl_pushed_back_token = first_token;
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 6035cf4816..c5b2d5b61f 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -486,6 +486,7 @@ pg_logical_replication_slot_advance(XLogRecPtr moveto)
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									NIL,
 									true,	/* fast_forward */
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index d3a136b6f5..87392dea1f 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -238,6 +238,7 @@ static void CreateReplicationSlot(CreateReplicationSlotCmd *cmd);
 static void DropReplicationSlot(DropReplicationSlotCmd *cmd);
 static void StartReplication(StartReplicationCmd *cmd);
 static void StartLogicalReplication(StartReplicationCmd *cmd);
+static void CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd);
 static void ProcessStandbyMessage(void);
 static void ProcessStandbyReplyMessage(void);
 static void ProcessStandbyHSFeedbackMessage(void);
@@ -1284,7 +1285,7 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	 * are reported early.
 	 */
 	logical_decoding_ctx =
-		CreateDecodingContext(cmd->startpoint, cmd->options, false,
+		CreateDecodingContext(cmd->startpoint, cmd->options, false, false,
 							  XL_ROUTINE(.page_read = logical_read_xlog_page,
 										 .segment_open = WalSndSegmentOpen,
 										 .segment_close = wal_segment_close),
@@ -1336,6 +1337,98 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	EndCommand(&qc, DestRemote, false);
 }
 
+/*
+ * Create a snapshot from an existing replication slot.
+ */
+static void
+CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd)
+{
+	Snapshot	snap;
+	LogicalDecodingContext *ctx;
+	char		xloc[MAXFNAMELEN];
+	DestReceiver *dest;
+	TupOutputState *tstate;
+	TupleDesc	tupdesc;
+	Datum		values[1];
+	bool		nulls[1] = {0};
+
+	Assert(!MyReplicationSlot);
+
+	CheckLogicalDecodingRequirements();
+
+	if (!IsTransactionBlock())
+		ereport(ERROR,
+				(errmsg("%s must be called inside a transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (XactIsoLevel != XACT_REPEATABLE_READ)
+		ereport(ERROR,
+				(errmsg("%s must be called in REPEATABLE READ isolation mode transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (!XactReadOnly)
+		ereport(ERROR,
+				(errmsg("%s must be called in a read only transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (FirstSnapshotSet)
+		ereport(ERROR,
+				(errmsg("%s must be called before any query",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (IsSubTransaction())
+		ereport(ERROR,
+				(errmsg("%s must not be called in a subtransaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	ReplicationSlotAcquire(cmd->slotname, false);
+
+	ctx = CreateDecodingContext(MyReplicationSlot->data.restart_lsn,
+								cmd->options,
+								false,
+								true,
+								XL_ROUTINE(.page_read = logical_read_xlog_page,
+										   .segment_open = WalSndSegmentOpen,
+										   .segment_close = wal_segment_close),
+								WalSndPrepareWrite, WalSndWriteData,
+								WalSndUpdateProgress);
+
+	/*
+	 * Signal that we don't need the timeout mechanism. We're just creating
+	 * the snapshot with the replication slot and don't yet accept feedback
+	 * messages or send keepalives. As we possibly need to wait for further
+	 * WAL the walsender would otherwise possibly be killed too soon.
+	 */
+	last_reply_timestamp = 0;
+
+	/* build initial snapshot, might take a while */
+	DecodingContextFindStartpoint(ctx);
+
+	snap = SnapBuildInitialSnapshot(ctx->snapshot_builder);
+	RestoreTransactionSnapshot(snap, MyProc);
+
+	/* Don't need the decoding context anymore */
+	FreeDecodingContext(ctx);
+
+	/* Create a tuple to send consistent WAL location */
+	snprintf(xloc, sizeof(xloc), "%X/%X",
+			 LSN_FORMAT_ARGS(MyReplicationSlot->data.confirmed_flush));
+
+	dest = CreateDestReceiver(DestRemoteSimple);
+	tupdesc = CreateTemplateTupleDesc(1);
+	TupleDescInitBuiltinEntry(tupdesc, (AttrNumber) 1, "consistent_point",
+							  TEXTOID, -1, 0);
+	tstate = begin_tup_output_tupdesc(dest, tupdesc, &TTSOpsVirtual);
+
+	/* consistent wal location */
+	values[0] = CStringGetTextDatum(xloc);
+
+	do_tup_output(tstate, values, nulls);
+	end_tup_output(tstate);
+
+	ReplicationSlotRelease();
+}
+
 /*
  * LogicalDecodingContext 'prepare_write' callback.
  *
@@ -1864,6 +1957,15 @@ exec_replication_command(const char *cmd_string)
 			}
 			break;
 
+		case T_CreateReplicationSnapshotCmd:
+			{
+				cmdtag = "CREATE_REPLICATION_SNAPSHOT";
+				set_ps_display(cmdtag);
+				CreateReplicationSnapshot((CreateReplicationSnapshotCmd *) cmd_node);
+				EndReplicationCommand(cmdtag);
+				break;
+			}
+
 		default:
 			elog(ERROR, "unrecognized replication command node tag: %u",
 				 cmd_node->type);
diff --git a/src/include/nodes/replnodes.h b/src/include/nodes/replnodes.h
index 4321ba8f86..154ab74f33 100644
--- a/src/include/nodes/replnodes.h
+++ b/src/include/nodes/replnodes.h
@@ -108,4 +108,15 @@ typedef struct TimeLineHistoryCmd
 	TimeLineID	timeline;
 } TimeLineHistoryCmd;
 
+/* ----------------------
+ *		CREATE_REPLICATION_SNAPSHOT command
+ * ----------------------
+ */
+typedef struct CreateReplicationSnapshotCmd
+{
+	NodeTag		type;
+	char	   *slotname;
+	List	   *options;
+} CreateReplicationSnapshotCmd;
+
 #endif							/* REPLNODES_H */
diff --git a/src/include/replication/logical.h b/src/include/replication/logical.h
index 5f49554ea0..6535786a0e 100644
--- a/src/include/replication/logical.h
+++ b/src/include/replication/logical.h
@@ -125,6 +125,7 @@ extern LogicalDecodingContext *CreateInitDecodingContext(const char *plugin,
 extern LogicalDecodingContext *CreateDecodingContext(XLogRecPtr start_lsn,
 													 List *output_plugin_options,
 													 bool fast_forward,
+													 bool need_full_snapshot,
 													 XLogReaderRoutine *xl_routine,
 													 LogicalOutputPluginWriterPrepareWrite prepare_write,
 													 LogicalOutputPluginWriterWrite do_write,
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 281626fa6f..f01e5f23a7 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -386,6 +386,16 @@ typedef WalRcvExecResult *(*walrcv_exec_fn) (WalReceiverConn *conn,
  */
 typedef void (*walrcv_disconnect_fn) (WalReceiverConn *conn);
 
+/*
+ * walrcv_slot_snapshot_fn
+ *
+ * Create a snapshot by an existing replication slot
+ */
+typedef void (*walrcv_slot_snapshot_fn) (WalReceiverConn *conn,
+										 char *slotname,
+										 const WalRcvStreamOptions *options,
+										 XLogRecPtr *lsn);
+
 typedef struct WalReceiverFunctionsType
 {
 	walrcv_connect_fn walrcv_connect;
@@ -403,6 +413,7 @@ typedef struct WalReceiverFunctionsType
 	walrcv_get_backend_pid_fn walrcv_get_backend_pid;
 	walrcv_exec_fn walrcv_exec;
 	walrcv_disconnect_fn walrcv_disconnect;
+	walrcv_slot_snapshot_fn walrcv_slot_snapshot;
 } WalReceiverFunctionsType;
 
 extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
@@ -437,6 +448,8 @@ extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
 	WalReceiverFunctions->walrcv_exec(conn, exec, nRetTypes, retTypes)
 #define walrcv_disconnect(conn) \
 	WalReceiverFunctions->walrcv_disconnect(conn)
+#define walrcv_slot_snapshot(conn, slotname, options, lsn) \
+	WalReceiverFunctions->walrcv_slot_snapshot(conn, slotname, options, lsn)
 
 static inline void
 walrcv_clear_result(WalRcvExecResult *walres)
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 260854747b..985a4ab7fd 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2348,6 +2348,7 @@ ReplicationSlotInvalidationCause
 ReplicationSlotOnDisk
 ReplicationSlotPersistency
 ReplicationSlotPersistentData
+CreateReplicationSnapshotCmd
 ReplicationState
 ReplicationStateCtl
 ReplicationStateOnDisk
@@ -3853,6 +3854,7 @@ walrcv_receive_fn
 walrcv_send_fn
 walrcv_server_version_fn
 walrcv_startstreaming_fn
+walrcv_slot_snapshot_fn
 wchar2mb_with_len_converter
 wchar_t
 win32_deadchild_waitinfo
-- 
2.25.1

#70Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Hayato Kuroda (Fujitsu) (#67)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi,

Thanks for your reviews.

Hayato Kuroda (Fujitsu) <kuroda.hayato@fujitsu.com>, 13 Haz 2023 Sal,
13:06 tarihinde şunu yazdı:

01. general

Why do tablesync workers have to disconnect from publisher for every iterations?
I think connection initiation overhead cannot be negligible in the postgres's basic
architecture. I have not checked yet, but could we add a new replication message
like STOP_STREAMING or CLEANUP? Or, engineerings for it is quite larger than the benefit?

This actually makes sense. I quickly try to do that without adding any
new replication message. As you would expect, it did not work.
I don't really know what's needed to make a connection to last for
more than one iteration. Need to look into this. Happy to hear any
suggestions and thoughts.

The sync worker sends a signal to its leader per the iteration, but it may be too
often. Maybe it is added for changing the rstate to READY, however, it is OK to
change it when the next change have come because should_apply_changes_for_rel()
returns true even if rel->state == SUBREL_STATE_SYNCDONE. I think the notification
should be done only at the end of sync workers. How do you think?

I tried to move the logicalrep_worker_wakeup call from
clean_sync_worker (end of an iteration) to finish_sync_worker (end of
sync worker). I made table sync much slower for some reason, then I
reverted that change. Maybe I should look a bit more into the reason
why that happened some time.

Thanks,
--
Melih Mutlu
Microsoft

#71Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Peter Smith (#68)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Peter,

Thanks for your reviews. I tried to apply most of them. I just have
some comments below for some of them.

Peter Smith <smithpb2250@gmail.com>, 14 Haz 2023 Çar, 08:45 tarihinde
şunu yazdı:

9. process_syncing_tables_for_sync

@@ -378,7 +387,13 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
*/
replorigin_drop_by_name(originname, true, false);

- finish_sync_worker();
+ /*
+ * Sync worker is cleaned at this point. It's ready to sync next table,
+ * if needed.
+ */
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->ready_to_reuse = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);

9a.
I did not quite follow the logic. It says "Sync worker is cleaned at
this point", but who is doing that? -- more details are needed. But,
why not just call clean_sync_worker() right here like it use to call
finish_sync_worker?

I agree that these explanations at places where the worker decides to
not continue with the current table were confusing. Even the name of
ready_to_reuse was misleading. I renamed it and tried to improve
comments in such places.
Can you please check if those make more sense now?

======
src/backend/replication/logical/worker.c

10. General -- run_tablesync_worker, TablesyncWorkerMain

IMO these functions would be more appropriately reside in the
tablesync.c instead of the (common) worker.c. Was there some reason
why they cannot be put there?

I'm not really against moving those functions to tablesync.c. But
what's not clear to me is worker.c. Is it the places to put common
functions for all log. rep. workers? Then, what about apply worker?
Should we consider a separate file for apply worker too?

Thanks,
--
Melih Mutlu
Microsoft

#72Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#71)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Fri, Jun 23, 2023 at 11:50 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

src/backend/replication/logical/worker.c

10. General -- run_tablesync_worker, TablesyncWorkerMain

IMO these functions would be more appropriately reside in the
tablesync.c instead of the (common) worker.c. Was there some reason
why they cannot be put there?

I'm not really against moving those functions to tablesync.c. But
what's not clear to me is worker.c. Is it the places to put common
functions for all log. rep. workers? Then, what about apply worker?
Should we consider a separate file for apply worker too?

IIUC
- tablesync.c = for tablesync only
- applyparallelworker = for parallel apply worker only
- worker.c = for both normal apply worker, plus "common" worker code

Regarding making another file (e.g. applyworker.c). It sounds
sensible, but I guess you would need to first demonstrate the end
result will be much cleaner to get support for such a big refactor.

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#73Hayato Kuroda (Fujitsu)
kuroda.hayato@fujitsu.com
In reply to: Melih Mutlu (#70)
5 attachment(s)
RE: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Dear Melih,

Thank you for updating the patch! I have not reviewed yet, but I wanted
to reply your comments.

This actually makes sense. I quickly try to do that without adding any
new replication message. As you would expect, it did not work.
I don't really know what's needed to make a connection to last for
more than one iteration. Need to look into this. Happy to hear any
suggestions and thoughts.

I have analyzed how we handle this. Please see attached the patch (0003) which
allows reusing connection. The patchset passed tests on my CI.
To make cfbot happy I reassigned the patch number.

In this patch, the tablesync worker does not call clean_sync_worker() at the end
of iterations, and the establishment of the connection is done only once.
The creation of memory context is also suppressed.

Regarding the walsender, streamingDone{Sending|Receiving} is now initialized
before executing StartLogicalReplication(). These flags have been used to decide
when the process exits copy mode. The default value is false, and they are set
to true when the copy mode is finished.
I think there was no use-case that the same walsender executes START_REPLICATION
replication twice so there were no codes for restoring flags. Please tell me if any other
reasons.

Best Regards,
Hayato Kuroda
FUJITSU LIMITED

Attachments:

0004-Add-replication-protocol-cmd-to-create-a-snapshot.patchapplication/octet-stream; name=0004-Add-replication-protocol-cmd-to-create-a-snapshot.patchDownload
From 490498f5abef6a898576adf4a0dec0deef80ce61 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 13 Oct 2022 17:05:45 +0300
Subject: [PATCH 4/5] Add replication protocol cmd to create a snapshot

Introduced CREATE_REPLICATION_SNAPSHOT to be able to create and use a
snapshot without creating a new replication slot, but by using an
existing slot.

CREATE_REPLICATION_SNAPSHOT simply does what CREATE_REPLICATION_SLOT does
without creating a new replication slot.

CREATE_REPLICATION_SNAPSHOT command imports the snapshot into the current
transaction and returns consistent_point. The changes earlier than the
consistent_point will be applied by importing the snapshot. All changes
later than the consistent_point will be available to be consumed from
the replication slot.

This is useful for reusing replication slots in logical replication.
Otherwise, tablesync workers cannot start from a consistent point to copy
a relation and then apply changes by consuming from replication slot.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/protocol.sgml                    |  31 ++++++
 .../libpqwalreceiver/libpqwalreceiver.c       |  69 +++++++++++-
 src/backend/replication/logical/logical.c     |  40 ++++++-
 .../replication/logical/logicalfuncs.c        |   1 +
 src/backend/replication/repl_gram.y           |  18 ++-
 src/backend/replication/repl_scanner.l        |   2 +
 src/backend/replication/slotfuncs.c           |   1 +
 src/backend/replication/walsender.c           | 104 +++++++++++++++++-
 src/include/nodes/replnodes.h                 |  11 ++
 src/include/replication/logical.h             |   1 +
 src/include/replication/walreceiver.h         |  13 +++
 src/tools/pgindent/typedefs.list              |   2 +
 12 files changed, 289 insertions(+), 4 deletions(-)

diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index b11d9a6ba3..f7d5acee93 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2595,6 +2595,37 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
      </listitem>
     </varlistentry>
 
+    <varlistentry id="protocol-replication-replication-slot-snapshot">
+     <term><literal>CREATE_REPLICATION_SNAPSHOT</literal> <replaceable class="parameter">slot_name</replaceable> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
+      <indexterm><primary>CREATE_REPLICATION_SNAPSHOT</primary></indexterm>
+     </term>
+     <listitem>
+      <para>
+       Creates a snapshot including all the changes from the replication slot until
+       the point at which the replication slot becomes consistent. Then the snapshot
+       is used in the current transaction. This command is currently only supported
+       for logical replication slots.
+      </para>
+
+      <para>
+       In response to this command, the server will return a one-row result set,
+       containing the following field:
+       <variablelist>
+        <varlistentry>
+         <term><literal>consistent_point</literal> (<type>text</type>)</term>
+         <listitem>
+          <para>
+           The WAL location at which the slot became consistent.  This is the
+           earliest location from which streaming can start on this replication
+           slot.
+          </para>
+         </listitem>
+        </varlistentry>
+       </variablelist>
+      </para>
+     </listitem>
+    </varlistentry>
+
     <varlistentry id="protocol-replication-base-backup" xreflabel="BASE_BACKUP">
      <term><literal>BASE_BACKUP</literal> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
       <indexterm><primary>BASE_BACKUP</primary></indexterm>
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index dc9c5c82d9..cf88a3b7d4 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -82,6 +82,8 @@ static WalRcvExecResult *libpqrcv_exec(WalReceiverConn *conn,
 									   const int nRetTypes,
 									   const Oid *retTypes);
 static void libpqrcv_disconnect(WalReceiverConn *conn);
+static void libpqrcv_slot_snapshot(WalReceiverConn *conn, char *slotname,
+								   const WalRcvStreamOptions *options, XLogRecPtr *lsn);
 
 static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_connect = libpqrcv_connect,
@@ -98,7 +100,8 @@ static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_create_slot = libpqrcv_create_slot,
 	.walrcv_get_backend_pid = libpqrcv_get_backend_pid,
 	.walrcv_exec = libpqrcv_exec,
-	.walrcv_disconnect = libpqrcv_disconnect
+	.walrcv_disconnect = libpqrcv_disconnect,
+	.walrcv_slot_snapshot = libpqrcv_slot_snapshot
 };
 
 /* Prototypes for private functions */
@@ -982,6 +985,70 @@ libpqrcv_create_slot(WalReceiverConn *conn, const char *slotname,
 	return snapshot;
 }
 
+/*
+ * TODO
+ */
+static void
+libpqrcv_slot_snapshot(WalReceiverConn *conn,
+					   char *slotname,
+					   const WalRcvStreamOptions *options,
+					   XLogRecPtr *lsn)
+{
+	StringInfoData cmd;
+	PGresult   *res;
+	char	   *pubnames_str;
+	List	   *pubnames;
+	char	   *pubnames_literal;
+
+	initStringInfo(&cmd);
+
+	/* Build the command. */
+	appendStringInfo(&cmd, "CREATE_REPLICATION_SNAPSHOT \"%s\"", slotname);
+	appendStringInfoString(&cmd, " (");
+	appendStringInfo(&cmd, " proto_version '%u'",
+					 options->proto.logical.proto_version);
+
+	/* Add publication names. */
+	pubnames = options->proto.logical.publication_names;
+	pubnames_str = stringlist_to_identifierstr(conn->streamConn, pubnames);
+	if (!pubnames_str)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	pubnames_literal = PQescapeLiteral(conn->streamConn, pubnames_str,
+									   strlen(pubnames_str));
+	if (!pubnames_literal)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	appendStringInfo(&cmd, ", publication_names %s", pubnames_literal);
+	PQfreemem(pubnames_literal);
+	pfree(pubnames_str);
+
+	appendStringInfoString(&cmd, " )");
+
+	/* Execute the command. */
+	res = libpqrcv_PQexec(conn->streamConn, cmd.data);
+	pfree(cmd.data);
+
+	if (PQresultStatus(res) != PGRES_TUPLES_OK)
+	{
+		PQclear(res);
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg("Could not create a snapshot by replication slot \"%s\": %s",
+						slotname, pchomp(PQerrorMessage(conn->streamConn)))));
+	}
+
+	if (lsn)
+		*lsn = DatumGetLSN(DirectFunctionCall1Coll(pg_lsn_in, InvalidOid,
+												   CStringGetDatum(PQgetvalue(res, 0, 0))));
+
+	PQclear(res);
+}
+
 /*
  * Return PID of remote backend process.
  */
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 41243d0187..3c11f8e1dd 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -476,6 +476,10 @@ CreateInitDecodingContext(const char *plugin,
  * fast_forward
  *		bypass the generation of logical changes.
  *
+ * need_full_snapshot
+ * 		if true, must obtain a snapshot able to read all tables;
+ *  	if false, one that can read only catalogs is acceptable.
+ *
  * xl_routine
  *		XLogReaderRoutine used by underlying xlogreader
  *
@@ -494,6 +498,7 @@ LogicalDecodingContext *
 CreateDecodingContext(XLogRecPtr start_lsn,
 					  List *output_plugin_options,
 					  bool fast_forward,
+					  bool need_full_snapshot,
 					  XLogReaderRoutine *xl_routine,
 					  LogicalOutputPluginWriterPrepareWrite prepare_write,
 					  LogicalOutputPluginWriterWrite do_write,
@@ -502,6 +507,7 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 	LogicalDecodingContext *ctx;
 	ReplicationSlot *slot;
 	MemoryContext old_context;
+	TransactionId xmin_horizon = InvalidTransactionId;
 
 	/* shorter lines... */
 	slot = MyReplicationSlot;
@@ -571,8 +577,40 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 		start_lsn = slot->data.confirmed_flush;
 	}
 
+
+	/*
+	 * We need to determine a safe xmin horizon to start decoding from if we
+	 * want to create a snapshot too. Otherwise we would end up with a
+	 * snapshot that cannot be imported since xmin value from the snapshot may
+	 * be less than the oldest safe xmin. To avoid this call
+	 * GetOldestSafeDecodingTransactionId() to return a safe xmin value, which
+	 * can be used while exporting/importing the snapshot.
+	 *
+	 * So we have to acquire the ProcArrayLock to prevent computation of new
+	 * xmin horizons by other backends, get the safe decoding xid, and inform
+	 * the slot machinery about the new limit. Once that's done the
+	 * ProcArrayLock can be released as the slot machinery now is protecting
+	 * against vacuum.
+	 */
+	if (need_full_snapshot)
+	{
+		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+		xmin_horizon = GetOldestSafeDecodingTransactionId(!need_full_snapshot);
+
+		SpinLockAcquire(&slot->mutex);
+		slot->effective_catalog_xmin = xmin_horizon;
+		slot->data.catalog_xmin = xmin_horizon;
+		slot->effective_xmin = xmin_horizon;
+		SpinLockRelease(&slot->mutex);
+
+		ReplicationSlotsComputeRequiredXmin(true);
+
+		LWLockRelease(ProcArrayLock);
+	}
+
 	ctx = StartupDecodingContext(output_plugin_options,
-								 start_lsn, InvalidTransactionId, false,
+								 start_lsn, xmin_horizon, need_full_snapshot,
 								 fast_forward, xl_routine, prepare_write,
 								 do_write, update_progress);
 
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index 55a24c02c9..85c5cdb633 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -208,6 +208,7 @@ pg_logical_slot_get_changes_guts(FunctionCallInfo fcinfo, bool confirm, bool bin
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									options,
 									false,
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/repl_gram.y b/src/backend/replication/repl_gram.y
index 0c874e33cf..957a5cc653 100644
--- a/src/backend/replication/repl_gram.y
+++ b/src/backend/replication/repl_gram.y
@@ -65,6 +65,7 @@ Node *replication_parse_result;
 %token K_CREATE_REPLICATION_SLOT
 %token K_DROP_REPLICATION_SLOT
 %token K_TIMELINE_HISTORY
+%token K_CREATE_REPLICATION_SNAPSHOT
 %token K_WAIT
 %token K_TIMELINE
 %token K_PHYSICAL
@@ -80,7 +81,7 @@ Node *replication_parse_result;
 %type <node>	command
 %type <node>	base_backup start_replication start_logical_replication
 				create_replication_slot drop_replication_slot identify_system
-				read_replication_slot timeline_history show
+				read_replication_slot timeline_history show create_replication_snapshot
 %type <list>	generic_option_list
 %type <defelt>	generic_option
 %type <uintval>	opt_timeline
@@ -114,6 +115,7 @@ command:
 			| read_replication_slot
 			| timeline_history
 			| show
+			| create_replication_snapshot
 			;
 
 /*
@@ -307,6 +309,19 @@ timeline_history:
 				}
 			;
 
+/*
+ * CREATE_REPLICATION_SNAPSHOT %s options
+ */
+create_replication_snapshot:
+			K_CREATE_REPLICATION_SNAPSHOT var_name plugin_options
+				{
+					CreateReplicationSnapshotCmd *n = makeNode(CreateReplicationSnapshotCmd);
+					n->slotname = $2;
+					n->options = $3;
+					$$ = (Node *) n;
+				}
+			;
+
 opt_physical:
 			K_PHYSICAL
 			| /* EMPTY */
@@ -400,6 +415,7 @@ ident_or_keyword:
 			| K_CREATE_REPLICATION_SLOT	{ $$ = "create_replication_slot"; }
 			| K_DROP_REPLICATION_SLOT		{ $$ = "drop_replication_slot"; }
 			| K_TIMELINE_HISTORY			{ $$ = "timeline_history"; }
+			| K_CREATE_REPLICATION_SNAPSHOT	{ $$ = "create_replication_snapshot"; }
 			| K_WAIT						{ $$ = "wait"; }
 			| K_TIMELINE					{ $$ = "timeline"; }
 			| K_PHYSICAL					{ $$ = "physical"; }
diff --git a/src/backend/replication/repl_scanner.l b/src/backend/replication/repl_scanner.l
index cb467ca46f..5ba2e9c54b 100644
--- a/src/backend/replication/repl_scanner.l
+++ b/src/backend/replication/repl_scanner.l
@@ -126,6 +126,7 @@ START_REPLICATION	{ return K_START_REPLICATION; }
 CREATE_REPLICATION_SLOT		{ return K_CREATE_REPLICATION_SLOT; }
 DROP_REPLICATION_SLOT		{ return K_DROP_REPLICATION_SLOT; }
 TIMELINE_HISTORY	{ return K_TIMELINE_HISTORY; }
+CREATE_REPLICATION_SNAPSHOT	{ return K_CREATE_REPLICATION_SNAPSHOT; }
 PHYSICAL			{ return K_PHYSICAL; }
 RESERVE_WAL			{ return K_RESERVE_WAL; }
 LOGICAL				{ return K_LOGICAL; }
@@ -303,6 +304,7 @@ replication_scanner_is_replication_command(void)
 		case K_DROP_REPLICATION_SLOT:
 		case K_READ_REPLICATION_SLOT:
 		case K_TIMELINE_HISTORY:
+		case K_CREATE_REPLICATION_SNAPSHOT:
 		case K_SHOW:
 			/* Yes; push back the first token so we can parse later. */
 			repl_pushed_back_token = first_token;
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 6035cf4816..c5b2d5b61f 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -486,6 +486,7 @@ pg_logical_replication_slot_advance(XLogRecPtr moveto)
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									NIL,
 									true,	/* fast_forward */
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 429d00f2f0..974ab5ca35 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -238,6 +238,7 @@ static void CreateReplicationSlot(CreateReplicationSlotCmd *cmd);
 static void DropReplicationSlot(DropReplicationSlotCmd *cmd);
 static void StartReplication(StartReplicationCmd *cmd);
 static void StartLogicalReplication(StartReplicationCmd *cmd);
+static void CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd);
 static void ProcessStandbyMessage(void);
 static void ProcessStandbyReplyMessage(void);
 static void ProcessStandbyHSFeedbackMessage(void);
@@ -1284,7 +1285,7 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	 * are reported early.
 	 */
 	logical_decoding_ctx =
-		CreateDecodingContext(cmd->startpoint, cmd->options, false,
+		CreateDecodingContext(cmd->startpoint, cmd->options, false, false,
 							  XL_ROUTINE(.page_read = logical_read_xlog_page,
 										 .segment_open = WalSndSegmentOpen,
 										 .segment_close = wal_segment_close),
@@ -1336,6 +1337,98 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	EndCommand(&qc, DestRemote, false);
 }
 
+/*
+ * Create a snapshot from an existing replication slot.
+ */
+static void
+CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd)
+{
+	Snapshot	snap;
+	LogicalDecodingContext *ctx;
+	char		xloc[MAXFNAMELEN];
+	DestReceiver *dest;
+	TupOutputState *tstate;
+	TupleDesc	tupdesc;
+	Datum		values[1];
+	bool		nulls[1] = {0};
+
+	Assert(!MyReplicationSlot);
+
+	CheckLogicalDecodingRequirements();
+
+	if (!IsTransactionBlock())
+		ereport(ERROR,
+				(errmsg("%s must be called inside a transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (XactIsoLevel != XACT_REPEATABLE_READ)
+		ereport(ERROR,
+				(errmsg("%s must be called in REPEATABLE READ isolation mode transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (!XactReadOnly)
+		ereport(ERROR,
+				(errmsg("%s must be called in a read only transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (FirstSnapshotSet)
+		ereport(ERROR,
+				(errmsg("%s must be called before any query",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (IsSubTransaction())
+		ereport(ERROR,
+				(errmsg("%s must not be called in a subtransaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	ReplicationSlotAcquire(cmd->slotname, false);
+
+	ctx = CreateDecodingContext(MyReplicationSlot->data.restart_lsn,
+								cmd->options,
+								false,
+								true,
+								XL_ROUTINE(.page_read = logical_read_xlog_page,
+										   .segment_open = WalSndSegmentOpen,
+										   .segment_close = wal_segment_close),
+								WalSndPrepareWrite, WalSndWriteData,
+								WalSndUpdateProgress);
+
+	/*
+	 * Signal that we don't need the timeout mechanism. We're just creating
+	 * the snapshot with the replication slot and don't yet accept feedback
+	 * messages or send keepalives. As we possibly need to wait for further
+	 * WAL the walsender would otherwise possibly be killed too soon.
+	 */
+	last_reply_timestamp = 0;
+
+	/* build initial snapshot, might take a while */
+	DecodingContextFindStartpoint(ctx);
+
+	snap = SnapBuildInitialSnapshot(ctx->snapshot_builder);
+	RestoreTransactionSnapshot(snap, MyProc);
+
+	/* Don't need the decoding context anymore */
+	FreeDecodingContext(ctx);
+
+	/* Create a tuple to send consistent WAL location */
+	snprintf(xloc, sizeof(xloc), "%X/%X",
+			 LSN_FORMAT_ARGS(MyReplicationSlot->data.confirmed_flush));
+
+	dest = CreateDestReceiver(DestRemoteSimple);
+	tupdesc = CreateTemplateTupleDesc(1);
+	TupleDescInitBuiltinEntry(tupdesc, (AttrNumber) 1, "consistent_point",
+							  TEXTOID, -1, 0);
+	tstate = begin_tup_output_tupdesc(dest, tupdesc, &TTSOpsVirtual);
+
+	/* consistent wal location */
+	values[0] = CStringGetTextDatum(xloc);
+
+	do_tup_output(tstate, values, nulls);
+	end_tup_output(tstate);
+
+	ReplicationSlotRelease();
+}
+
 /*
  * LogicalDecodingContext 'prepare_write' callback.
  *
@@ -1870,6 +1963,15 @@ exec_replication_command(const char *cmd_string)
 			}
 			break;
 
+		case T_CreateReplicationSnapshotCmd:
+			{
+				cmdtag = "CREATE_REPLICATION_SNAPSHOT";
+				set_ps_display(cmdtag);
+				CreateReplicationSnapshot((CreateReplicationSnapshotCmd *) cmd_node);
+				EndReplicationCommand(cmdtag);
+				break;
+			}
+
 		default:
 			elog(ERROR, "unrecognized replication command node tag: %u",
 				 cmd_node->type);
diff --git a/src/include/nodes/replnodes.h b/src/include/nodes/replnodes.h
index 4321ba8f86..154ab74f33 100644
--- a/src/include/nodes/replnodes.h
+++ b/src/include/nodes/replnodes.h
@@ -108,4 +108,15 @@ typedef struct TimeLineHistoryCmd
 	TimeLineID	timeline;
 } TimeLineHistoryCmd;
 
+/* ----------------------
+ *		CREATE_REPLICATION_SNAPSHOT command
+ * ----------------------
+ */
+typedef struct CreateReplicationSnapshotCmd
+{
+	NodeTag		type;
+	char	   *slotname;
+	List	   *options;
+} CreateReplicationSnapshotCmd;
+
 #endif							/* REPLNODES_H */
diff --git a/src/include/replication/logical.h b/src/include/replication/logical.h
index 5f49554ea0..6535786a0e 100644
--- a/src/include/replication/logical.h
+++ b/src/include/replication/logical.h
@@ -125,6 +125,7 @@ extern LogicalDecodingContext *CreateInitDecodingContext(const char *plugin,
 extern LogicalDecodingContext *CreateDecodingContext(XLogRecPtr start_lsn,
 													 List *output_plugin_options,
 													 bool fast_forward,
+													 bool need_full_snapshot,
 													 XLogReaderRoutine *xl_routine,
 													 LogicalOutputPluginWriterPrepareWrite prepare_write,
 													 LogicalOutputPluginWriterWrite do_write,
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 281626fa6f..f01e5f23a7 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -386,6 +386,16 @@ typedef WalRcvExecResult *(*walrcv_exec_fn) (WalReceiverConn *conn,
  */
 typedef void (*walrcv_disconnect_fn) (WalReceiverConn *conn);
 
+/*
+ * walrcv_slot_snapshot_fn
+ *
+ * Create a snapshot by an existing replication slot
+ */
+typedef void (*walrcv_slot_snapshot_fn) (WalReceiverConn *conn,
+										 char *slotname,
+										 const WalRcvStreamOptions *options,
+										 XLogRecPtr *lsn);
+
 typedef struct WalReceiverFunctionsType
 {
 	walrcv_connect_fn walrcv_connect;
@@ -403,6 +413,7 @@ typedef struct WalReceiverFunctionsType
 	walrcv_get_backend_pid_fn walrcv_get_backend_pid;
 	walrcv_exec_fn walrcv_exec;
 	walrcv_disconnect_fn walrcv_disconnect;
+	walrcv_slot_snapshot_fn walrcv_slot_snapshot;
 } WalReceiverFunctionsType;
 
 extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
@@ -437,6 +448,8 @@ extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
 	WalReceiverFunctions->walrcv_exec(conn, exec, nRetTypes, retTypes)
 #define walrcv_disconnect(conn) \
 	WalReceiverFunctions->walrcv_disconnect(conn)
+#define walrcv_slot_snapshot(conn, slotname, options, lsn) \
+	WalReceiverFunctions->walrcv_slot_snapshot(conn, slotname, options, lsn)
 
 static inline void
 walrcv_clear_result(WalRcvExecResult *walres)
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 260854747b..985a4ab7fd 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2348,6 +2348,7 @@ ReplicationSlotInvalidationCause
 ReplicationSlotOnDisk
 ReplicationSlotPersistency
 ReplicationSlotPersistentData
+CreateReplicationSnapshotCmd
 ReplicationState
 ReplicationStateCtl
 ReplicationStateOnDisk
@@ -3853,6 +3854,7 @@ walrcv_receive_fn
 walrcv_send_fn
 walrcv_server_version_fn
 walrcv_startstreaming_fn
+walrcv_slot_snapshot_fn
 wchar2mb_with_len_converter
 wchar_t
 win32_deadchild_waitinfo
-- 
2.27.0

0005-Reuse-Replication-Slot-and-Origin-in-Tablesync.patchapplication/octet-stream; name=0005-Reuse-Replication-Slot-and-Origin-in-Tablesync.patchDownload
From fd7065669432bf177aa6bdbef36d6dcbcf6dea75 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH 5/5] Reuse Replication Slot and Origin in Tablesync

This commit allows reusing replication slots and origins during tablesync.

Earlier, a tablesync worker was creating a new replication slot and origin
each time it syncs a new table. With this patch, replication
slots/origins can be reusable for tablesync.

This reduces the overhead of creating/dropping replication slots and origins
and improves tablesync speed significantly especially for empty or small tables.

If the state of the current table is INIT or DATASYNC, tablesync worker needs a
repliation slot/origin. If the worker has not created slot and origin in
its previous runs, it will create those first. Otherwise the worker reuses
slot and origin created by the same worker in previous iterations earlier.
Tables in FINISHEDCOPY are expected to have a replication slot and origin.
Slot and origin names for such tables are persisted in
pg_subscription_rel catalog. Tablesync worker can fetch them and proceed
with existing slot and origin of FINISHEDCOPY tables and does not need to
create new ones.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/catalogs.sgml                    |  31 ++
 src/backend/catalog/pg_subscription.c         | 247 +++++++++++-
 src/backend/commands/subscriptioncmds.c       | 229 +++++++----
 .../replication/logical/applyparallelworker.c |   3 +-
 src/backend/replication/logical/launcher.c    |   8 +-
 src/backend/replication/logical/tablesync.c   | 369 +++++++++++++-----
 src/backend/replication/logical/worker.c      |  62 ++-
 src/include/catalog/pg_subscription.h         |   6 +
 src/include/catalog/pg_subscription_rel.h     |  14 +-
 src/include/replication/slot.h                |   3 +-
 src/include/replication/worker_internal.h     |  23 +-
 11 files changed, 797 insertions(+), 198 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index ed32ca0349..0183ecd21e 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8030,6 +8030,19 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        origin.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>sublastusedid</structfield> <type>int8</type>
+      </para>
+      <para>
+      The last used ID for tablesync workers. It acts as an unique identifier
+      for replication slots which are created by tablesync workers.
+      The last used ID needs to be persisted to make logical replication safely
+      proceed after any interruption. If sublastusedid is 0, then no table has
+      been synced yet.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
@@ -8114,6 +8127,24 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        otherwise null
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srrelslotname</structfield> <type>name</type>
+      </para>
+      <para>
+       Replication slot name that is used for synchronization of relation
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srreloriginname</structfield> <type>name</type>
+      </para>
+      <para>
+       Origin name that is used for tracking synchronization of relation
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index d07f88ce28..152fdaa310 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -108,6 +108,14 @@ GetSubscription(Oid subid, bool missing_ok)
 								   Anum_pg_subscription_suborigin);
 	sub->origin = TextDatumGetCString(datum);
 
+	/* Get last used id */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
+							tup,
+							Anum_pg_subscription_sublastusedid,
+							&isnull);
+	Assert(!isnull);
+	sub->lastusedid = DatumGetInt64(datum);
+
 	ReleaseSysCache(tup);
 
 	return sub;
@@ -199,6 +207,44 @@ DisableSubscription(Oid subid)
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update the last used replication slot ID for the given subscription.
+ */
+void
+UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid)
+{
+	Relation	rel;
+	bool		nulls[Natts_pg_subscription];
+	bool		replaces[Natts_pg_subscription];
+	Datum		values[Natts_pg_subscription];
+	HeapTuple	tup;
+
+	/* Look up the subscription in the catalog */
+	rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+	tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	/* Form a new tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	replaces[Anum_pg_subscription_sublastusedid - 1] = true;
+	values[Anum_pg_subscription_sublastusedid- 1] = Int64GetDatum(lastusedid);
+
+	/* Update the catalog */
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+	heap_freetuple(tup);
+
+	table_close(rel, NoLock);
+}
+
 /*
  * Convert text array to list of strings.
  *
@@ -228,7 +274,7 @@ textarray_to_stringlist(ArrayType *textarray)
  */
 void
 AddSubscriptionRelState(Oid subid, Oid relid, char state,
-						XLogRecPtr sublsn)
+						XLogRecPtr sublsn, char *relslotname, char *reloriginname)
 {
 	Relation	rel;
 	HeapTuple	tup;
@@ -257,6 +303,16 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
 	else
 		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -269,6 +325,60 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Internal function to modify columns for relation state update
+ */
+static void
+UpdateSubscriptionRelState_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char state,
+									XLogRecPtr sublsn)
+{
+	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
+	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+
+	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (sublsn != InvalidXLogRecPtr)
+		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
+	else
+		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication slot update
+ */
+static void
+UpdateSubscriptionRelReplicationSlot_internal(Datum *values,
+											bool *nulls,
+											bool *replaces,
+											char *relslotname)
+{
+	replaces[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication origin update
+ */
+static void
+UpdateSubscriptionRelOrigin_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char *reloriginname)
+{
+	replaces[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+}
+
 /*
  * Update the state of a subscription table.
  */
@@ -299,14 +409,56 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	memset(nulls, false, sizeof(nulls));
 	memset(replaces, false, sizeof(replaces));
 
-	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
-	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
 
-	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
-	if (sublsn != InvalidXLogRecPtr)
-		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
-	else
-		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update replication slot name, origin name and state of
+ * a subscription table in one transaction.
+ */
+void
+UpdateSubscriptionRel(Oid subid,
+					  Oid relid,
+					  char state,
+					  XLogRecPtr sublsn,
+					  char *relslotname,
+					  char *reloriginname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
+	UpdateSubscriptionRelOrigin_internal(values, nulls, replaces, reloriginname);
 
 	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
 							replaces);
@@ -318,6 +470,85 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Get origin name of subscription table.
+ *
+ * reloriginname's value has the replication origin name if the origin exists.
+ */
+void
+GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*originname;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srreloriginname, isnull);
+	if (!*isnull)
+	{
+		originname = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(reloriginname, originname, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
+/*
+ * Get replication slot name of subscription table.
+ *
+ * slotname's value has the replication slot name if the subscription has any.
+ */
+void
+GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*relrepslot;
+	bool		isnull;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srrelslotname, &isnull);
+	if (!isnull)
+	{
+		relrepslot = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(slotname, relrepslot, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
 /*
  * Get state of subscription table.
  *
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 54895ba929..9f5a295ddc 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -710,6 +710,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 		publicationListToArray(publications);
 	values[Anum_pg_subscription_suborigin - 1] =
 		CStringGetTextDatum(opts.origin);
+	values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(0);
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -773,7 +774,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 										 rv->schemaname, rv->relname);
 
 				AddSubscriptionRelState(subid, relid, table_state,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 			}
 
 			/*
@@ -864,6 +865,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
 	bool		must_use_password;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -943,7 +946,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				AddSubscriptionRelState(sub->oid, relid,
 										copy_data ? SUBREL_STATE_INIT : SUBREL_STATE_READY,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 				ereport(DEBUG1,
 						(errmsg_internal("table \"%s.%s\" added to subscription \"%s\"",
 										 rv->schemaname, rv->relname, sub->name)));
@@ -967,6 +970,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				char		state;
 				XLogRecPtr	statelsn;
+				char		slotname[NAMEDATALEN] = {0};
 
 				/*
 				 * Lock pg_subscription_rel with AccessExclusiveLock to
@@ -993,13 +997,36 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/*
+				 * Find the logical replication sync worker. If exists, store
+				 * the slot number for dropping associated replication slots
+				 * later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot_name);
+				}
+				else
+				{
+					/*
+					 * Sync of this relation might be failed in an earlier
+					 * attempt, but the replication slot might still exist.
+					 */
+					GetSubscriptionRelReplicationSlot(sub->oid, relid, slotname);
+					if (strlen(slotname) > 0)
+						sub_remove_slots = lappend(sub_remove_slots, slotname);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
 				 * tablesync origin.
 				 */
-				if (state != SUBREL_STATE_READY)
+				if (state != SUBREL_STATE_READY &&
+					state != SUBREL_STATE_SYNCDONE)
 				{
 					char		originname[NAMEDATALEN];
 
@@ -1027,31 +1054,24 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
-		 * the database operations we won't be able to rollback dropped slots.
+		 * Drop the replication slots associated with tablesync workers for
+		 * removed tables. This has to be at the end because otherwise if
+		 * there is an error while doing the database operations we won't be
+		 * able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			memcpy(syncslotname, lfirst(lc), sizeof(NAMEDATALEN));
+
+			/*
+			 * There is no certainty, maybe the slot does not exist yet. Also,
+			 * if we fail after removing some of the slots, next time, it will
+			 * again try to drop already dropped slots and fail. For these
+			 * reasons, we allow missing_ok = true for the drop.
+			 */
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1474,6 +1494,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	char	   *subname;
 	char	   *conninfo;
 	char	   *slotname;
+	int64		lastusedid;
 	List	   *subworkers;
 	ListCell   *lc;
 	char		originname[NAMEDATALEN];
@@ -1546,6 +1567,14 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	else
 		slotname = NULL;
 
+	/* Get the last used identifier by the subscription */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID, tup,
+							Anum_pg_subscription_sublastusedid, &isnull);
+	if (!isnull)
+		lastusedid = DatumGetInt64(datum);
+	else
+		lastusedid = 0;
+
 	/*
 	 * Since dropping a replication slot is not transactional, the replication
 	 * slot stays dropped even if the transaction rolls back.  So we cannot
@@ -1595,6 +1624,8 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	}
 	list_free(subworkers);
 
+	rstates = GetSubscriptionRelations(subid, true);
+
 	/*
 	 * Remove the no-longer-useful entry in the launcher's table of apply
 	 * worker start times.
@@ -1606,36 +1637,26 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	ApplyLauncherForgetWorkerStartTime(subid);
 
 	/*
-	 * Cleanup of tablesync replication origins.
-	 *
-	 * Any READY-state relations would already have dealt with clean-ups.
+	 * Cleanup of tablesync replication origins associated with the
+	 * subscription, if exists. Try to drop origins by creating all origin
+	 * names created for this subscription.
 	 *
 	 * Note that the state can't change because we have already stopped both
 	 * the apply and tablesync workers and they can't restart because of
 	 * exclusive lock on the subscription.
+	 *
+	 * XXX: This can be handled better instead of looping through all possible
 	 */
-	rstates = GetSubscriptionRelations(subid, true);
-	foreach(lc, rstates)
+	for (int64 i = 1; i <= lastusedid; i++)
 	{
-		SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-		Oid			relid = rstate->relid;
-
-		/* Only cleanup resources of tablesync workers */
-		if (!OidIsValid(relid))
-			continue;
+		char		originname_to_drop[NAMEDATALEN] = {0};
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * It is possible that the origin is not yet created for tablesync
-		 * worker so passing missing_ok = true. This can happen for the states
-		 * before SUBREL_STATE_FINISHEDCOPY.
-		 */
-		ReplicationOriginNameForLogicalRep(subid, relid, originname,
-										   sizeof(originname));
-		replorigin_drop_by_name(originname, true, false);
+		snprintf(originname_to_drop, sizeof(originname_to_drop), "pg_%u_%lld", subid, (long long) i);
+		/* missing_ok = true, since the origin might be already dropped. */
+		replorigin_drop_by_name(originname_to_drop, true, false);
 	}
 
+
 	/* Clean up dependencies */
 	deleteSharedDependencyRecordsFor(SubscriptionRelationId, subid, 0);
 
@@ -1688,39 +1709,17 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
+		List	   *slots = NULL;
 
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
 
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char	   *syncslotname = (char *) lfirst(lc);
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1743,6 +1742,71 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * Get the replication slot names associated with the subscription.
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok)
+{
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+					 " FROM pg_replication_slots"
+					 " WHERE slot_name LIKE 'pg_%i_sync_%%';",
+					 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+					errmsg("could not receive list of slots associated with the subscription %u, error: %s",
+					subid, res->err));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();
+		return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -2155,6 +2219,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -2165,18 +2230,20 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
-			char		syncslotname[NAMEDATALEN] = {0};
-
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
-				 syncslotname);
+				 worker->slot_name);
 		}
 	}
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index f16e2377bf..a57f178087 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -440,7 +440,8 @@ pa_launch_parallel_worker(void)
 										MySubscription->name,
 										MyLogicalRepWorker->userid,
 										InvalidOid,
-										dsm_segment_handle(winfo->dsm_seg));
+										dsm_segment_handle(winfo->dsm_seg),
+										InvalidRepSlotId);
 
 	if (launched)
 	{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index c24e56a11e..d0d8d95a68 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -304,7 +304,7 @@ logicalrep_workers_find(Oid subid, bool only_running)
  */
 bool
 logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
-						 Oid relid, dsm_handle subworker_dsm)
+						 Oid relid, dsm_handle subworker_dsm, int64 slotid)
 {
 	BackgroundWorker bgw;
 	BackgroundWorkerHandle *bgw_handle;
@@ -430,6 +430,9 @@ retry:
 	worker->launch_time = now;
 	worker->in_use = true;
 	worker->generation++;
+	worker->created_slot = false;
+	worker->rep_slot_id = slotid;
+	worker->slot_name = (char *) palloc(NAMEDATALEN);
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -1180,7 +1183,8 @@ ApplyLauncherMain(Datum main_arg)
 				ApplyLauncherSetWorkerStartTime(sub->oid, now);
 				logicalrep_worker_launch(sub->dbid, sub->oid, sub->name,
 										 sub->owner, InvalidOid,
-										 DSM_HANDLE_INVALID);
+										 DSM_HANDLE_INVALID,
+										 InvalidRepSlotId);
 			}
 			else
 			{
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 30fe9b78ac..c032214959 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -328,40 +328,29 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 								   MyLogicalRepWorker->relid,
 								   MyLogicalRepWorker->relstate,
 								   MyLogicalRepWorker->relstate_lsn);
+		CommitTransactionCommand();
 
 		/*
-		 * End streaming so that LogRepWorkerWalRcvConn can be used to drop
-		 * the slot.
-		 */
-		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
-
-		/*
-		 * Cleanup the tablesync slot.
+		 * Cleanup the tablesync slot. If the slot name used by this worker is
+		 * different from the default slot name for the worker, this means the
+		 * current table had started to being synchronized by another worker
+		 * and replication slot. And this worker is reusing a replication slot
+		 * from a previous attempt. We do not need that replication slot
+		 * anymore.
 		 *
 		 * This has to be done after updating the state because otherwise if
 		 * there is an error while doing the database operations we won't be
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
+										MyLogicalRepWorker->rep_slot_id,
 										syncslotname,
 										sizeof(syncslotname));
 
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
-		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
-
-		CommitTransactionCommand();
-		pgstat_report_stat(false);
-
-		/*
-		 * Start a new transaction to clean up the tablesync origin tracking.
-		 * This transaction will be ended within the finish_sync_worker().
-		 * Now, even, if we fail to remove this here, the apply worker will
-		 * ensure to clean it up afterward.
+		 * We are safe to drop the replication tracking origin after this
+		 * point. Now, even, if we fail to remove this here, the apply worker
+		 * will ensure to clean it up afterward.
 		 *
 		 * We need to do this after the table state is set to SYNCDONE.
 		 * Otherwise, if an error occurs while performing the database
@@ -370,32 +359,70 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 * have been cleared before restart. So, the restarted worker will use
 		 * invalid replication progress state resulting in replay of
 		 * transactions that have already been applied.
+		 *
+		 * Firstly reset the origin session to remove the ownership of the
+		 * slot. This is needed to allow the origin to be dropped or reused
+		 * later.
 		 */
+		replorigin_session_reset();
+		replorigin_session_origin = InvalidRepOriginId;
+		replorigin_session_origin_lsn = InvalidXLogRecPtr;
+		replorigin_session_origin_timestamp = 0;
+
 		StartTransactionCommand();
+		if (MyLogicalRepWorker->slot_name && strcmp(syncslotname, MyLogicalRepWorker->slot_name) != 0)
+		{
+			/*
+			 * End streaming so that LogRepWorkerWalRcvConn can be used to
+			 * drop the slot.
+			 */
+			walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
 
-		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
+			ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+											   MyLogicalRepWorker->relid,
+											   originname,
+											   sizeof(originname));
+
+			/*
+			 * Drop replication origin
+			 *
+			 * There is a chance that the user is concurrently performing refresh
+			 * for the subscription where we remove the table state and its origin
+			 * or the apply worker would have removed this origin. So passing
+			 * missing_ok = true.
+			 */
+			replorigin_drop_by_name(originname, true, false);
+		}
 
 		/*
-		 * Resetting the origin session removes the ownership of the slot.
-		 * This is needed to allow the origin to be dropped.
+		 * We are safe to remove persisted replication slot and origin data,
+		 * since it's already in SYNCDONE state. They will not be needed
+		 * anymore.
 		 */
-		replorigin_session_reset();
-		replorigin_session_origin = InvalidRepOriginId;
-		replorigin_session_origin_lsn = InvalidXLogRecPtr;
-		replorigin_session_origin_timestamp = 0;
+		UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+							  MyLogicalRepWorker->relid,
+							  MyLogicalRepWorker->relstate,
+							  MyLogicalRepWorker->relstate_lsn,
+							  NULL,
+							  NULL);
+		ereport(DEBUG2,
+			(errmsg("process_syncing_tables_for_sync: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					"NULL", "NULL", MyLogicalRepWorker->relstate,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+		CommitTransactionCommand();
+		pgstat_report_stat(false);
 
 		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * There is a chance that the user is concurrently performing refresh
-		 * for the subscription where we remove the table state and its origin
-		 * or the apply worker would have removed this origin. So passing
-		 * missing_ok = true.
+		 * This should return the default origin name for the worker. Even if
+		 * the worker used a different origin for this table, it should be
+		 * dropped and removed from the catalog so far.
 		 */
-		replorigin_drop_by_name(originname, true, false);
+		StartTransactionCommand();
+		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+										   MyLogicalRepWorker->relid,
+										   originname,
+										   sizeof(originname));
 
 		/* Sync worker has completed synchronization of the current table. */
 		MyLogicalRepWorker->is_sync_completed = true;
@@ -492,6 +519,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 			if (current_lsn >= rstate->lsn)
 			{
 				char		originname[NAMEDATALEN];
+				bool		is_origin_null = true;
 
 				rstate->state = SUBREL_STATE_READY;
 				rstate->lsn = current_lsn;
@@ -512,18 +540,31 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				 * error while dropping we won't restart it to drop the
 				 * origin. So passing missing_ok = true.
 				 */
-				ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-												   rstate->relid,
-												   originname,
-												   sizeof(originname));
-				replorigin_drop_by_name(originname, true, false);
+				GetSubscriptionRelOrigin(MyLogicalRepWorker->subid,
+										 rstate->relid, originname,
+										 &is_origin_null);
+
+				if (!is_origin_null)
+				{
+					replorigin_drop_by_name(originname, true, false);
+				}
 
 				/*
 				 * Update the state to READY only after the origin cleanup.
 				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-										   rstate->relid, rstate->state,
-										   rstate->lsn);
+				UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+									  rstate->relid,
+									  rstate->state,
+									  rstate->lsn,
+									  NULL,
+									  NULL);
+				ereport(DEBUG2,
+					(errmsg("process_syncing_tables_for_apply: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+							"NULL", "NULL", rstate->state,
+							rstate->relid, MyLogicalRepWorker->subid)));
+
+				CommitTransactionCommand();
+				started_tx = false;
 			}
 		}
 		else
@@ -612,12 +653,25 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						TimestampDifferenceExceeds(hentry->last_start_time, now,
 												   wal_retrieve_retry_interval))
 					{
+						if (IsTransactionState())
+							CommitTransactionCommand();
+						StartTransactionCommand();
+						started_tx = true;
+
+						MySubscription->lastusedid++;
+						UpdateSubscriptionLastSlotId(MyLogicalRepWorker->subid,
+													 MySubscription->lastusedid);
+						ereport(DEBUG2,
+								(errmsg("process_syncing_tables_for_apply: incremented lastusedid to %lld for subscription %u",
+										(long long) MySubscription->lastusedid, MySubscription->oid)));
+
 						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
 												 MySubscription->oid,
 												 MySubscription->name,
 												 MyLogicalRepWorker->userid,
 												 rstate->relid,
-												 DSM_HANDLE_INVALID);
+												 DSM_HANDLE_INVALID,
+												 MySubscription->lastusedid);
 						hentry->last_start_time = now;
 					}
 				}
@@ -1240,8 +1294,8 @@ copy_table(Relation rel)
  * The name must not exceed NAMEDATALEN - 1 because of remote node constraints
  * on slot name length. We append system_identifier to avoid slot_name
  * collision with subscriptions in other clusters. With the current scheme
- * pg_%u_sync_%u_UINT64_FORMAT (3 + 10 + 6 + 10 + 20 + '\0'), the maximum
- * length of slot_name will be 50.
+ * pg_%u_sync_%lu_UINT64_FORMAT (3 + 10 + 6 + 20 + 20 + '\0'), the maximum
+ * length of slot_name will be 45.
  *
  * The returned slot name is stored in the supplied buffer (syncslotname) with
  * the given size.
@@ -1252,11 +1306,11 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
+ReplicationSlotNameForTablesync(Oid suboid, int64 slotid,
 								char *syncslotname, Size szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%lld_" UINT64_FORMAT, suboid,
+			(long long) slotid, GetSystemIdentifier());
 }
 
 /*
@@ -1282,6 +1336,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	UserContext ucxt;
 	bool		must_use_password;
 	bool		run_as_owner;
+	char	   *prev_slotname;
 
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
@@ -1316,7 +1371,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
+									MyLogicalRepWorker->rep_slot_id,
 									slotname,
 									NAMEDATALEN);
 
@@ -1345,12 +1400,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC ||
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY);
 
+	/*
+	 * See if tablesync of the current relation has been started with another
+	 * replication slot.
+	 *
+	 * Read previous slot name from the catalog, if exists.
+	 */
+	prev_slotname = (char *) palloc(NAMEDATALEN);
+	StartTransactionCommand();
+	GetSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+									  MyLogicalRepWorker->relid,
+									  prev_slotname);
+
 	/* Assign the origin tracking record name. */
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   sizeof(originname));
 
+	CommitTransactionCommand();
+
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC)
 	{
 		/*
@@ -1364,10 +1433,53 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		 * breakdown then it wouldn't have succeeded so trying it next time
 		 * seems like a better bet.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, slotname, true);
+		if (strlen(prev_slotname) > 0)
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);
+
+			StartTransactionCommand();
+			/* Replication origin might still exist. Try to drop */
+			replorigin_drop_by_name(originname, true, false);
+
+			/*
+			 * Remove replication slot and origin name from the relation's
+			 * catalog record
+			 */
+			UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+								  MyLogicalRepWorker->relid,
+								  MyLogicalRepWorker->relstate,
+								  MyLogicalRepWorker->relstate_lsn,
+								  NULL,
+								  NULL);
+			CommitTransactionCommand();
+			ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+						"NULL", "NULL", MyLogicalRepWorker->relstate,
+						MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+		}
 	}
 	else if (MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY)
 	{
+		/*
+		 * At this point, the table that is currently being synchronized
+		 * should have its replication slot name filled in the catalog. The
+		 * tablesync process was started with another sync worker and
+		 * replication slot. We need to continue using the same replication
+		 * slot in this worker too.
+		 */
+		if (strlen(prev_slotname) == 0)
+		{
+			elog(ERROR, "Replication slot could not be found for subscription %u, relation %u",
+				 MyLogicalRepWorker->subid,
+				 MyLogicalRepWorker->relid);
+		}
+
+		/*
+		 * Proceed with the correct replication slot. Use previously created
+		 * replication slot to sync this table.
+		 */
+		memcpy(slotname, prev_slotname, NAMEDATALEN);
+
 		/*
 		 * The COPY phase was previously done, but tablesync then crashed
 		 * before it was able to finish normally.
@@ -1387,7 +1499,9 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 		goto copy_table_done;
 	}
+	pfree(prev_slotname);
 
+	/* Preparing for table copy operation */
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 	MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
 	MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
@@ -1395,11 +1509,31 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   MyLogicalRepWorker->relstate,
-							   MyLogicalRepWorker->relstate_lsn);
+
+	/*
+	 * Refresh the originname in case of having non-existing origin
+	 * from previous failed sync attempts.
+	 * If that's the case, it should be removed from the catalog so far.
+	 * Then, we can continue by reusing the origin created by the current
+	 * worker instead of .
+	 */
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									MyLogicalRepWorker->relid,
+									originname,
+									sizeof(originname));
+
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  MyLogicalRepWorker->relstate,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					slotname, originname, MyLogicalRepWorker->relstate,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+
 	pgstat_report_stat(true);
 
 	StartTransactionCommand();
@@ -1427,48 +1561,95 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						res->err)));
 	walrcv_clear_result(res);
 
+	originid = replorigin_by_name(originname, true);
+
 	/*
 	 * Create a new permanent logical decoding slot. This slot will be used
 	 * for the catchup phase after COPY is done, so tell it to use the
 	 * snapshot to make the final data consistent.
+	 *
+	 * Replication slot will only be created if either this is the first run
+	 * of the worker or we're not using a previous replication slot.
 	 */
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-
-	/*
-	 * Setup replication origin tracking. The purpose of doing this before the
-	 * copy is to avoid doing the copy again due to any error in setting up
-	 * origin tracking.
-	 */
-	originid = replorigin_by_name(originname, true);
-	if (!OidIsValid(originid))
+	if (!MyLogicalRepWorker->created_slot)
 	{
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						   slotname, false /* permanent */ , false /* two_phase */ ,
+						   CRS_USE_SNAPSHOT, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: created replication slot %s for subscription %u",
+						slotname, MyLogicalRepWorker->subid)));
+
 		/*
-		 * Origin tracking does not exist, so create it now.
-		 *
-		 * Then advance to the LSN got from walrcv_create_slot. This is WAL
-		 * logged for the purpose of recovery. Locks are to prevent the
-		 * replication origin from vanishing while advancing.
+		 * Remember that we created the slot so that we will not try to create
+		 * it again.
 		 */
-		originid = replorigin_create(originname);
-
-		LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
-		replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
-						   true /* go backward */ , true /* WAL log */ );
-		UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->created_slot = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
+		/*
+		 * Setup replication origin tracking. The purpose of doing this before
+		 * the copy is to avoid doing the copy again due to any error in
+		 * setting up origin tracking.
+		 */
+		if (!OidIsValid(originid))
+		{
+			/*
+			 * Origin tracking does not exist, so create it now.
+			 */
+			originid = replorigin_create(originname);
+		}
+		else
+		{
+			/*
+			 * At this point, there shouldn't be any existing replication
+			 * origin with the same name.
+			 */
+			ereport(ERROR,
+					(errcode(ERRCODE_DUPLICATE_OBJECT),
+					 errmsg("replication origin \"%s\" already exists",
+							originname)));
+		}
 	}
 	else
 	{
-		ereport(ERROR,
-				(errcode(ERRCODE_DUPLICATE_OBJECT),
-				 errmsg("replication origin \"%s\" already exists",
-						originname)));
+		/*
+		 * Do not create a new replication slot, reuse the existing one
+		 * instead. Use a new snapshot for the replication slot to ensure that
+		 * tablesync and apply proceses are consistent with each other.
+		 */
+		WalRcvStreamOptions options;
+		int			server_version;
+
+		server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+		options.proto.logical.proto_version =
+			server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+			server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+			LOGICALREP_PROTO_VERSION_NUM;
+		options.proto.logical.publication_names = MySubscription->publications;
+
+		walrcv_slot_snapshot(LogRepWorkerWalRcvConn, slotname, &options, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: reusing replication slot %s for relation %u in subscription %u",
+						slotname, MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
 	}
 
+	/*
+	 * Advance to the LSN got from walrcv_create_slot or walrcv_slot_snapshot.
+	 * This is WAL logged for the purpose of recovery. Locks are to prevent
+	 * the replication origin from vanishing while advancing.
+	 *
+	 * Then setup replication origin tracking.
+	 */
+	LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+	replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
+					   true /* go backward */ , true /* WAL log */ );
+	UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+
 	/*
 	 * Make sure that the copy command runs as the table owner, unless the
 	 * user has opted out of that behaviour.
@@ -1527,12 +1708,18 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	 * Update the persisted state to indicate the COPY phase is done; make it
 	 * visible to others.
 	 */
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   SUBREL_STATE_FINISHEDCOPY,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  SUBREL_STATE_FINISHEDCOPY,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					originname, slotname, SUBREL_STATE_FINISHEDCOPY,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
 
 copy_table_done:
 
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 8df960e343..c8f8b72aa7 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -464,8 +464,16 @@ ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 {
 	if (OidIsValid(relid))
 	{
-		/* Replication origin name for tablesync workers. */
-		snprintf(originname, szoriginname, "pg_%u_%u", suboid, relid);
+		bool		is_null = true;
+
+		/*
+		 * Replication origin name for tablesync workers. First, look into the
+		 * catalog. If originname does not exist, then use the default name.
+		 */
+		GetSubscriptionRelOrigin(suboid, relid,
+								 originname, &is_null);
+		if (is_null)
+			snprintf(originname, szoriginname, "pg_%u_%lld", suboid, (long long) MyLogicalRepWorker->rep_slot_id);
 	}
 	else
 	{
@@ -3770,7 +3778,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* Tablesync workers should end streaming before exiting the main loop
+	/*
+	 * Tablesync workers should end streaming before exiting the main loop
 	 * to drop replication slot. Only end streaming here for apply workers.
 	 */
 	if (!am_tablesync_worker())
@@ -4496,6 +4505,9 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 
 	/* allocate slot name in long-lived context */
 	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+
+	/* Keep the replication slot name used for this sync. */
+	MyLogicalRepWorker->slot_name = *myslotname;
 	pfree(syncslotname);
 }
 
@@ -4550,10 +4562,12 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 	/* Start table synchronization. */
 	start_table_sync(origin_startpos, &slotname);
 
+	StartTransactionCommand();
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   originname_size);
+	CommitTransactionCommand();
 
 	set_apply_error_context_origin(originname);
 
@@ -4594,11 +4608,10 @@ run_apply_worker(WalRcvStreamOptions *options,
 				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
 					errmsg("subscription has no replication slot set")));
 
-	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-									   originname, originname_size);
-
 	/* Setup replication origin tracking. */
 	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
 	originid = replorigin_by_name(originname, true);
 	if (!OidIsValid(originid))
 		originid = replorigin_create(originname);
@@ -4875,12 +4888,12 @@ TablesyncWorkerMain(Datum main_arg)
 				memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
 
 				/*
-				* Pick the table for the next run if it is not already picked up
-				* by another worker.
-				*
-				* Take exclusive lock to prevent any other sync worker from picking
-				* the same table.
-				*/
+				 * Pick the table for the next run if it is not already picked up
+				 * by another worker.
+				 *
+				 * Take exclusive lock to prevent any other sync worker from picking
+				 * the same table.
+				 */
 				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
 				if (rstate->state != SUBREL_STATE_SYNCDONE &&
 					!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
@@ -4912,7 +4925,32 @@ TablesyncWorkerMain(Datum main_arg)
 			}
 
 			if (!is_table_found)
+			{
+				TimeLineID	tli;
+
+				/*
+				 * It is important to give an error if we are unable to drop the
+				 * slot, otherwise, it won't be dropped till the corresponding
+				 * subscription is dropped. So passing missing_ok = false.
+				 */
+				if (MyLogicalRepWorker->created_slot)
+				{
+					walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+					ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
+				}
+
+				/*
+				 * Drop replication origin before exiting.
+				 *
+				 * There is a chance that the user is concurrently performing refresh
+				 * for the subscription where we remove the table state and its origin
+				 * or the apply worker would have removed this origin. So passing
+				 * missing_ok = true.
+				 */
+				replorigin_drop_by_name(originname, true, false);
+
 				break;
+			}
 		}
 	}
 
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 1d40eebc78..7e13f59847 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -108,6 +108,9 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
 
 	/* Only publish data originating from the specified origin */
 	text		suborigin BKI_DEFAULT(LOGICALREP_ORIGIN_ANY);
+
+	/* The last used ID to create a replication slot for tablesync */
+	int64		sublastusedid BKI_DEFAULT(0);
 #endif
 } FormData_pg_subscription;
 
@@ -144,6 +147,8 @@ typedef struct Subscription
 	List	   *publications;	/* List of publication names to subscribe to */
 	char	   *origin;			/* Only publish data originating from the
 								 * specified origin */
+	int64		lastusedid;		/* Last used unique ID to create replication
+								 * slots in tablesync */
 } Subscription;
 
 /* Disallow streaming in-progress transactions. */
@@ -164,6 +169,7 @@ typedef struct Subscription
 extern Subscription *GetSubscription(Oid subid, bool missing_ok);
 extern void FreeSubscription(Subscription *sub);
 extern void DisableSubscription(Oid subid);
+extern void UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid);
 
 extern int	CountDBSubscriptions(Oid dbid);
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 60a2bcca23..185164d75e 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -44,6 +44,12 @@ CATALOG(pg_subscription_rel,6102,SubscriptionRelRelationId)
 											 * used for synchronization
 											 * coordination, or NULL if not
 											 * valid */
+	NameData	srrelslotname BKI_FORCE_NULL;	/* name of the replication
+												 * slot for relation in
+												 * subscription */
+	NameData	srreloriginname BKI_FORCE_NULL; /* origin name for relation in
+												 * subscription */
+
 #endif
 } FormData_pg_subscription_rel;
 
@@ -81,10 +87,16 @@ typedef struct SubscriptionRelState
 } SubscriptionRelState;
 
 extern void AddSubscriptionRelState(Oid subid, Oid relid, char state,
-									XLogRecPtr sublsn);
+									XLogRecPtr sublsn, char *relslotname, char *reloriginname);
 extern void UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 									   XLogRecPtr sublsn);
+extern void UpdateSubscriptionRel(Oid subid, Oid relid, char state,
+								  XLogRecPtr sublsn, char *relslotname, char *reloriginname);
+
 extern char GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn);
+extern void GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname);
+extern void GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull);
+
 extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 
 extern bool HasSubscriptionRelations(Oid subid);
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index a8a89dc784..31b2c41893 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -237,8 +237,9 @@ extern bool InvalidateObsoleteReplicationSlots(ReplicationSlotInvalidationCause
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
 extern bool ReplicationSlotName(int index, Name name);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, Size szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, int64 slotid, char *syncslotname, Size szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List *GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 1e9f8e6e72..24006240c9 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -35,6 +35,23 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/*
+	 * Indicates if the sync worker created a replication slot for itself
+	 * in any point of its lifetime.
+	 * False means that the worker has not created a slot yet, and has been
+	 * reusing replication slots created by other workers so far.
+	 */
+	bool		created_slot;
+
+	/*
+	 * Unique identifier for replication slot to be created by tablesnync
+	 * workers, if needed.
+	 */
+	int64		rep_slot_id;
+
+	/* Replication slot name used by the worker. */
+	char	   *slot_name;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
@@ -239,7 +256,8 @@ extern LogicalRepWorker *logicalrep_worker_find(Oid subid, Oid relid,
 extern List *logicalrep_workers_find(Oid subid, bool only_running);
 extern bool logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname,
 									 Oid userid, Oid relid,
-									 dsm_handle subworker_dsm);
+									 dsm_handle subworker_dsm,
+									 int64 slotid);
 extern void logicalrep_worker_stop(Oid subid, Oid relid);
 extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
 extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
@@ -335,4 +353,7 @@ am_parallel_apply_worker(void)
 	return isParallelApplyWorker(MyLogicalRepWorker);
 }
 
+/* Invalid identifier to be used for naming replication slots */
+#define InvalidRepSlotId	0
+
 #endif							/* WORKER_INTERNAL_H */
-- 
2.27.0

0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchapplication/octet-stream; name=0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchDownload
From 9ba4e50c8e9884e1da18216092cc2418c5cae3ae Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Mon, 5 Jun 2023 15:04:41 +0300
Subject: [PATCH 1/5] Refactor to split Apply and Tablesync Workers

Both apply and tablesync workers were using ApplyWorkerMain() as entry
point. As the name implies, ApplyWorkerMain() should be considered as
the main function for apply workers. Tablesync worker's path was hidden
and does not have enough in common to share the same main function with
apply worker.

Also; most of the code shared by both worker types are already combined
in LogicalRepApplyLoop(). There is no need to combine the rest in
ApplyWorkerMain() anymore.

This commit introduces TablesyncWorkerMain() as a new entry point for
tablesync workers and separates both type of workers from each other.
This aims to increase code readability and help to maintain logical
replication workers separately.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/postmaster/bgworker.c             |   3 +
 .../replication/logical/applyparallelworker.c |   2 +-
 src/backend/replication/logical/launcher.c    |  25 +-
 src/backend/replication/logical/tablesync.c   |   2 +-
 src/backend/replication/logical/worker.c      | 381 +++++++++++-------
 src/include/replication/logicalworker.h       |   1 +
 src/include/replication/worker_internal.h     |   4 +-
 7 files changed, 258 insertions(+), 160 deletions(-)

diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 0dd22b2351..5609919edf 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -131,6 +131,9 @@ static const struct
 	},
 	{
 		"ParallelApplyWorkerMain", ParallelApplyWorkerMain
+	},
+	{
+		"TablesyncWorkerMain", TablesyncWorkerMain
 	}
 };
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 82c1ddcdcb..f16e2377bf 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -942,7 +942,7 @@ ParallelApplyWorkerMain(Datum main_arg)
 	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
 		MyLogicalRepWorker->reply_time = 0;
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 8395ae7b23..f562b1db41 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -459,24 +459,27 @@ retry:
 	snprintf(bgw.bgw_library_name, BGW_MAXLEN, "postgres");
 
 	if (is_parallel_apply_worker)
+	{
 		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ParallelApplyWorkerMain");
-	else
-		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
-
-	if (OidIsValid(relid))
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication worker for subscription %u sync %u", subid, relid);
-	else if (is_parallel_apply_worker)
+				 "logical replication parallel apply worker for subscription %u", subid);
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
 				 "logical replication parallel apply worker for subscription %u", subid);
+	}
+	else if (OidIsValid(relid))
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "TablesyncWorkerMain");
+		snprintf(bgw.bgw_name, BGW_MAXLEN,
+				 "logical replication tablesync worker for subscription %u sync %u", subid, relid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication tablesync worker");
+	}
 	else
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
 				 "logical replication apply worker for subscription %u", subid);
-
-	if (is_parallel_apply_worker)
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
-	else
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication worker");
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication apply worker");
+	}
 
 	bgw.bgw_restart_time = BGW_NEVER_RESTART;
 	bgw.bgw_notify_pid = MyProcPid;
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 6d461654ab..8125bbd170 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -131,7 +131,7 @@ static StringInfo copybuf = NULL;
 /*
  * Exit routine for synchronization worker.
  */
-static void
+void
 pg_attribute_noreturn()
 finish_sync_worker(void)
 {
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 0ee764d68f..b979a755ae 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -392,6 +392,7 @@ static void stream_open_file(Oid subid, TransactionId xid,
 static void stream_write_change(char action, StringInfo s);
 static void stream_open_and_write_change(TransactionId xid, char action, StringInfo s);
 static void stream_close_file(void);
+static void set_stream_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
@@ -4330,6 +4331,69 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+ /* set_stream_options
+  * 	Set logical replication streaming options.
+  *
+  * This function sets streaming options including replication slot name
+  * and origin start position. Workers need these options for logical replication.
+  */
+static void
+set_stream_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+		options->proto.logical.twophase = true;
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4442,13 +4506,134 @@ start_apply(XLogRecPtr origin_startpos)
 }
 
 /*
- * Common initialization for leader apply worker and parallel apply worker.
+ * Runs the tablesync worker.
+ * It starts syncing tables. After a successful sync,
+ * sets streaming options and starts streaming to catchup.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Start table synchronization. */
+	start_table_sync(origin_startpos, &slotname);
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   originname_size);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	/* Start applying changes to catchup. */
+	start_apply(*origin_startpos);
+}
+
+/*
+ * Runs the leader apply worker.
+ * It sets up replication origin, streaming options
+ * and then starts streaming.
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
+{
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+	bool		must_use_password;
+
+	slotname = MySubscription->slotname;
+
+	/*
+	 * This shouldn't happen if the subscription is enabled, but guard
+	 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+	 * crash if slot is NULL.)
+	 */
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+
+	/* Is the use of a password mandatory? */
+	must_use_password = MySubscription->passwordrequired &&
+		!superuser_arg(MySubscription->owner);
+
+	/* Note that the superuser_arg call can access the DB */
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											must_use_password,
+											MySubscription->name, &err);
+
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+	 * We don't really use the output identify_system for anything but it
+	 * does some initializations on the upstream so let's still call it.
+	 */
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+	{
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
+	}
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+
+	/* Run the main loop. */
+	start_apply(*origin_startpos);
+}
+
+/*
+ * Common initialization for logical replication workers; leader apply worker,
+ * parallel apply worker and tablesync worker.
  *
  * Initialize the database connection, in-memory subscription and necessary
  * config options.
  */
 void
-InitializeApplyWorker(void)
+InitializeLogRepWorker(void)
 {
 	MemoryContext oldctx;
 
@@ -4512,7 +4697,8 @@ InitializeApplyWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has started",
+				(errmsg("%s for subscription \"%s\", table \"%s\" has started",
+						get_worker_name(),
 						MySubscription->name,
 						get_rel_name(MyLogicalRepWorker->relid))));
 	else
@@ -4533,7 +4719,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	InitializingApplyWorker = true;
 
@@ -4557,7 +4742,7 @@ ApplyWorkerMain(Datum main_arg)
 	/* Load the libpq-specific functions */
 	load_file("libpqwalreceiver", false);
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
@@ -4565,165 +4750,69 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-		bool		must_use_password;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-
-		/* Is the use of a password mandatory? */
-		must_use_password = MySubscription->passwordrequired &&
-			!superuser_arg(MySubscription->owner);
-
-		/* Note that the superuser_arg call can access the DB */
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												must_use_password,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
-	 * the subscription relation state.
+	 * the subscription relation state. Do this outside the loop to avoid
+	 * exceeding MAX_SYSCACHE_CALLBACKS
 	 */
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
+	/* This is leader apply worker */
+	run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
+	proc_exit(0);
+}
 
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
+/* Logical Replication Tablesync worker entry point */
+void
+TablesyncWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *myslotname = NULL;
+	WalRcvStreamOptions options;
+
+	/* Attach to slot */
+	logicalrep_worker_attach(worker_slot);
+
+	/* Setup signal handling */
+	pqsignal(SIGHUP, SignalHandlerForConfigReload);
+	pqsignal(SIGTERM, die);
+	BackgroundWorkerUnblockSignals();
 
 	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 * We don't currently need any ResourceOwner in a walreceiver process, but
+	 * if we did, we could call CreateAuxProcessResourceOwner here.
 	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
 
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
+	/* Initialise stats to a sanish value */
+	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
+		MyLogicalRepWorker->reply_time = GetCurrentTimestamp();
 
-	if (!am_tablesync_worker())
-	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
-		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+	/* Load the libpq-specific functions */
+	load_file("libpqwalreceiver", false);
 
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
-		}
-		else
-		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-		}
+	InitializeLogRepWorker();
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-								 MySubscription->name,
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-								 "?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
+	/* Connect to the origin and start the replication. */
+	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
+		 MySubscription->conninfo);
 
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+	/*
+	 * Setup callback for syscache so that we know when something changes in
+	 * the subscription relation state. Do this outside the loop to avoid
+	 * exceeding MAX_SYSCACHE_CALLBACKS
+	 */
+	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
+								  invalidate_syncing_table_states,
+								  (Datum) 0);
 
-	proc_exit(0);
+	run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
+
+	finish_sync_worker();
 }
 
 /*
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 39588da79f..bbd71d0b42 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -18,6 +18,7 @@ extern PGDLLIMPORT volatile sig_atomic_t ParallelApplyMessagePending;
 
 extern void ApplyWorkerMain(Datum main_arg);
 extern void ParallelApplyWorkerMain(Datum main_arg);
+extern void TablesyncWorkerMain(Datum main_arg);
 
 extern bool IsLogicalWorker(void);
 extern bool IsLogicalParallelApplyWorker(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 343e781896..7aba034774 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -265,7 +265,7 @@ extern void maybe_reread_subscription(void);
 
 extern void stream_cleanup_files(Oid subid, TransactionId xid);
 
-extern void InitializeApplyWorker(void);
+extern void InitializeLogRepWorker(void);
 
 extern void store_flush_position(XLogRecPtr remote_lsn, XLogRecPtr local_lsn);
 
@@ -307,6 +307,8 @@ extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
 
 #define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)
 
+extern void finish_sync_worker(void);
+
 static inline bool
 am_tablesync_worker(void)
 {
-- 
2.27.0

0002-Reuse-Tablesync-Workers.patchapplication/octet-stream; name=0002-Reuse-Tablesync-Workers.patchDownload
From 9f97e6dd29264e48581c6480a411f2f1948c5da3 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Mon, 5 Jun 2023 15:45:29 +0300
Subject: [PATCH 2/5] Reuse Tablesync Workers

This commit allows reusing tablesync workers for syncing more than one
table sequentially during their lifetime, instead of exiting after
only syncing one table.

Before this commit, tablesync workers were capable of syncing only one
table. For each table, a new sync worker was launched and that worker would
exit when done processing the table.

Now, tablesync workers are not limited to processing only one
table. When done, they can move to processing another table in
the same subscription.

If there is a table that needs to be synced, an available tablesync
worker picks up that table and syncs it. Each tablesync worker
continues to pick new tables to sync until there are no tables left
requiring synchronization. If there was no available worker to
process the table, then a new tablesync worker will be launched,
provided the number of tablesync workers for the subscription does not
exceed max_sync_workers_per_subscription.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/replication/logical/launcher.c  |   1 +
 src/backend/replication/logical/tablesync.c |  46 ++++++--
 src/backend/replication/logical/worker.c    | 110 +++++++++++++++++++-
 src/include/replication/worker_internal.h   |   7 ++
 4 files changed, 149 insertions(+), 15 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index f562b1db41..c24e56a11e 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -440,6 +440,7 @@ retry:
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
+	worker->is_sync_completed = false;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 8125bbd170..37f073b968 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -129,11 +129,10 @@ static bool FetchTableStates(bool *started_tx);
 static StringInfo copybuf = NULL;
 
 /*
- * Exit routine for synchronization worker.
+ * Prepares the synchronization worker for reuse or exit.
  */
 void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+clean_sync_worker(void)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -145,19 +144,38 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
+	/*
+	 * Disconnect from publisher. Otherwise reused sync workers causes
+	 * exceeding max_wal_senders
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		walrcv_disconnect(LogRepWorkerWalRcvConn);
+		LogRepWorkerWalRcvConn = NULL;
+	}
+
+	/* Find the leader apply worker and signal it. */
+	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
+}
+
+/*
+ * Exit routine for synchronization worker.
+ */
+void
+pg_attribute_noreturn()
+finish_sync_worker(void)
+{
+	clean_sync_worker();
+
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
 	StartTransactionCommand();
 	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
+			(errmsg("logical replication table synchronization worker for subscription \"%s\" has finished",
+					MySubscription->name)));
 	CommitTransactionCommand();
 
-	/* Find the leader apply worker and signal it. */
-	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
-
 	/* Stop gracefully */
 	proc_exit(0);
 }
@@ -379,7 +397,15 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 */
 		replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
+		/* Sync worker has completed synchronization of the current table. */
+		MyLogicalRepWorker->is_sync_completed = true;
+
+		ereport(LOG,
+		(errmsg("logical replication table synchronization worker for subscription \"%s\", relation \"%s\" with relid %u has finished",
+				MySubscription->name,
+				get_rel_name(MyLogicalRepWorker->relid),
+				MyLogicalRepWorker->relid)));
+		CommitTransactionCommand();
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index b979a755ae..4a0f402ad4 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3624,6 +3624,20 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can call process_syncing_tables_for_sync.
+				 *
+				 * process_syncing_tables_for_sync decides whether the sync of the
+				 * current table is completed. If it is completed, streaming must
+				 * be already ended. So, we can break the loop.
+				 */
+				if (MyLogicalRepWorker->is_sync_completed)
+				{
+					endofstream = true;
+					break;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -3643,6 +3657,15 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+
+			/*
+			 * If is_sync_completed is true, this means that the tablesync worker
+			 * is done with synchronization. Streaming has already been ended by
+			 * process_syncing_tables_for_sync. We should move to the next table
+			 * if needed, or exit.
+			 */
+			if (MyLogicalRepWorker->is_sync_completed)
+				endofstream = true;
 		}
 
 		/* Cleanup the memory. */
@@ -3745,8 +3768,11 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/* Tablesync workers should end streaming before exiting the main loop
+	 * to drop replication slot. Only end streaming here for apply workers.
+	 */
+	if (!am_tablesync_worker())
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
 }
 
 /*
@@ -4517,6 +4543,8 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 					 int originname_size,
 					 XLogRecPtr *origin_startpos)
 {
+	MyLogicalRepWorker->is_sync_completed = false;
+
 	/* Start table synchronization. */
 	start_table_sync(origin_startpos, &slotname);
 
@@ -4697,10 +4725,11 @@ InitializeLogRepWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("%s for subscription \"%s\", table \"%s\" has started",
+				(errmsg("%s for subscription \"%s\", table \"%s\" with relid %u has started",
 						get_worker_name(),
 						MySubscription->name,
-						get_rel_name(MyLogicalRepWorker->relid))));
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
 	else
 		ereport(LOG,
 		/* translator: first %s is the name of logical replication worker */
@@ -4810,7 +4839,78 @@ TablesyncWorkerMain(Datum main_arg)
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
+	/*
+	 * The loop where worker does its job. It loops until there is no relation
+	 * left to sync.
+	 */
+	for (;;)
+	{
+		List	   *rstates;
+		SubscriptionRelState *rstate;
+		ListCell   *lc;
+		bool 	is_table_found = false;
+
+		run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
+
+		if (IsTransactionState())
+			CommitTransactionCommand();
+
+		if (MyLogicalRepWorker->is_sync_completed)
+		{
+			/* This transaction will be committed by clean_sync_worker. */
+			StartTransactionCommand();
+
+			/*
+			 * Check if there is any table whose relation state is still INIT.
+			 * If a table in INIT state is found, the worker will not be finished,
+			 * it will be reused instead.
+			 */
+			rstates = GetSubscriptionRelations(MySubscription->oid, true);
+			rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
+
+			foreach(lc, rstates)
+			{
+				memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
+
+				/*
+				* Pick the table for the next run if it is not already picked up
+				* by another worker.
+				*
+				* Take exclusive lock to prevent any other sync worker from picking
+				* the same table.
+				*/
+				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
+				if (rstate->state != SUBREL_STATE_SYNCDONE &&
+					!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+				{
+					/* Update worker state for the next table */
+					MyLogicalRepWorker->relid = rstate->relid;
+					MyLogicalRepWorker->relstate = rstate->state;
+					MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+					LWLockRelease(LogicalRepWorkerLock);
+
+					/* found a table for next iteration */
+					is_table_found = true;
+					clean_sync_worker();
+
+					StartTransactionCommand();
+					ereport(LOG,
+							(errmsg("%s for subscription \"%s\" has moved to sync table \"%s\" with relid %u.",
+									get_worker_name(),
+									MySubscription->name,
+									get_rel_name(MyLogicalRepWorker->relid),
+									MyLogicalRepWorker->relid)));
+					CommitTransactionCommand();
+
+					break;
+				}
+				LWLockRelease(LogicalRepWorkerLock);
+			}
+
+			if (!is_table_found)
+				break;
+		}
+	}
 
 	finish_sync_worker();
 }
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 7aba034774..1e9f8e6e72 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -56,6 +56,12 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/*
+	 * Indicates whether tablesync worker has completed sycning its assigned
+	 * table. If true, no need to continue with that table.
+	 */
+	bool		is_sync_completed;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction or
@@ -308,6 +314,7 @@ extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
 #define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)
 
 extern void finish_sync_worker(void);
+extern void clean_sync_worker(void);
 
 static inline bool
 am_tablesync_worker(void)
-- 
2.27.0

0003-reuse-connection-when-tablesync-workers-change-the-t.patchapplication/octet-stream; name=0003-reuse-connection-when-tablesync-workers-change-the-t.patchDownload
From 3ff63e4db6966194b03c86e77c1b312aab5e26cf Mon Sep 17 00:00:00 2001
From: Hayato Kuroda <kuroda.hayato@fujitsu.com>
Date: Tue, 27 Jun 2023 07:10:45 +0000
Subject: [PATCH 3/5] reuse connection when tablesync workers change the target

---
 src/backend/replication/logical/tablesync.c | 21 ++++++++++++++-------
 src/backend/replication/logical/worker.c    | 20 ++++++++++++--------
 src/backend/replication/walsender.c         |  6 ++++++
 3 files changed, 32 insertions(+), 15 deletions(-)

diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 37f073b968..30fe9b78ac 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -1321,14 +1321,21 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 									NAMEDATALEN);
 
 	/*
-	 * Here we use the slot name instead of the subscription name as the
-	 * application_name, so that it is different from the leader apply worker,
-	 * so that synchronous replication can distinguish them.
+	 * Connect to publisher if not yet. The application_name must be also
+	 * different from the leader apply worker.
 	 */
-	LogRepWorkerWalRcvConn =
-		walrcv_connect(MySubscription->conninfo, true,
-					   must_use_password,
-					   slotname, &err);
+	if (LogRepWorkerWalRcvConn == NULL)
+	{
+		char application_name[NAMEDATALEN];
+
+		snprintf(application_name, NAMEDATALEN, "tablesync for %s",
+				 MySubscription->name);
+		LogRepWorkerWalRcvConn =
+			walrcv_connect(MySubscription->conninfo, true,
+						   must_use_password,
+						   application_name, &err);
+	}
+
 	if (LogRepWorkerWalRcvConn == NULL)
 		ereport(ERROR,
 				(errcode(ERRCODE_CONNECTION_FAILURE),
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 4a0f402ad4..8df960e343 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3498,19 +3498,21 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 	/*
 	 * Init the ApplyMessageContext which we clean up after each replication
-	 * protocol message.
+	 * protocol message, if needed.
 	 */
-	ApplyMessageContext = AllocSetContextCreate(ApplyContext,
-												"ApplyMessageContext",
-												ALLOCSET_DEFAULT_SIZES);
+	if (!ApplyMessageContext)
+		ApplyMessageContext = AllocSetContextCreate(ApplyContext,
+													"ApplyMessageContext",
+													ALLOCSET_DEFAULT_SIZES);
 
 	/*
 	 * This memory context is used for per-stream data when the streaming mode
 	 * is enabled. This context is reset on each stream stop.
 	 */
-	LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
-													"LogicalStreamingContext",
-													ALLOCSET_DEFAULT_SIZES);
+	if (!LogicalStreamingContext)
+		LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
+														"LogicalStreamingContext",
+														ALLOCSET_DEFAULT_SIZES);
 
 	/* mark as idle, before starting to loop */
 	pgstat_report_activity(STATE_IDLE, NULL);
@@ -4891,7 +4893,9 @@ TablesyncWorkerMain(Datum main_arg)
 
 					/* found a table for next iteration */
 					is_table_found = true;
-					clean_sync_worker();
+
+					CommitTransactionCommand();
+					pgstat_report_stat(true);
 
 					StartTransactionCommand();
 					ereport(LOG,
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index d3a136b6f5..429d00f2f0 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1828,6 +1828,12 @@ exec_replication_command(const char *cmd_string)
 				set_ps_display(cmdtag);
 				PreventInTransactionBlock(true, cmdtag);
 
+				/*
+				 * Initialize the flag again because this streaming may be
+				 * second time.
+				 */
+				streamingDoneSending = streamingDoneReceiving = false;
+
 				if (cmd->kind == REPLICATION_KIND_PHYSICAL)
 					StartReplication(cmd);
 				else
-- 
2.27.0

#74Amit Kapila
amit.kapila16@gmail.com
In reply to: Melih Mutlu (#69)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Fri, Jun 23, 2023 at 7:03 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

You can find the updated patchset attached.
I worked to address the reviews and made some additional changes.

Let me first explain the new patchset.
0001: Refactors the logical replication code, mostly worker.c and
tablesync.c. Although this patch makes it easier to reuse workers, I
believe that it's useful even by itself without other patches. It does
not improve performance or anything but aims to increase readability
and such.
0002: This is only to reuse worker processes, everything else stays
the same (replication slots/origins etc.).
0003: Adds a new command for streaming replication protocol to create
a snapshot by an existing replication slot.
0004: Reuses replication slots/origins together with workers.

Even only 0001 and 0002 are enough to improve table sync performance
at the rates previously shared on this thread. This also means that
currently 0004 (reusing replication slots/origins) does not improve as
much as I would expect, even though it does not harm either.
I just wanted to share what I did so far, while I'm continuing to
investigate it more to see what I'm missing in patch 0004.

I think the reason why you don't see the benefit of the 0004 patches
is that it still pays the cost of disconnect/connect and we haven't
saved much on network transfer costs because of the new snapshot you
are creating in patch 0003. Is it possible to avoid disconnect/connect
each time the patch needs to reuse the same tablesync worker? Once, we
do that and save the cost of drop_slot and associated network round
trip, you may see the benefit of 0003 and 0004 patches.

--
With Regards,
Amit Kapila.

#75Hayato Kuroda (Fujitsu)
kuroda.hayato@fujitsu.com
In reply to: Melih Mutlu (#69)
RE: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Dear Melih,

Thanks for updating the patch. Followings are my comments.
Note that some lines exceeds 80 characters and some other lines seem too short.
And comments about coding conventions were skipped.

0001

01. logicalrep_worker_launch()

```
        if (is_parallel_apply_worker)
+       {
                snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ParallelApplyWorkerMain");
-       else
-               snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
-
-       if (OidIsValid(relid))
                snprintf(bgw.bgw_name, BGW_MAXLEN,
-                                "logical replication worker for subscription %u sync %u", subid, relid);
-       else if (is_parallel_apply_worker)
+                                "logical replication parallel apply worker for subscription %u", subid);
                snprintf(bgw.bgw_name, BGW_MAXLEN,
                                 "logical replication parallel apply worker for subscription %u", subid);
```

Latter snprintf(bgw.bgw_name...) should be snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication worker").

02. ApplyWorkerMain

```
        /*
         * Setup callback for syscache so that we know when something changes in
-        * the subscription relation state.
+        * the subscription relation state. Do this outside the loop to avoid
+        * exceeding MAX_SYSCACHE_CALLBACKS
         */
```

I'm not sure this change is really needed. CacheRegisterSyscacheCallback() must
be outside the loop to avoid duplicated register, and it seems trivial.

0002

03. TablesyncWorkerMain()

Regarding the inner loop, the exclusive lock is acquired even if the rstate is
SUBREL_STATE_SYNCDONE. Moreover, palloc() and memcpy() for rstate seemsed not
needed. How about following?

```
for (;;)
{
List *rstates;
- SubscriptionRelState *rstate;
ListCell *lc;
...
- rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));

                        foreach(lc, rstates)
                        {
-                               memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
+                               SubscriptionRelState *rstate =
+                                                                               (SubscriptionRelState *) lfirst(lc);
+
+                               if (rstate->state == SUBREL_STATE_SYNCDONE)
+                                       continue;
                                /*
-                               * Pick the table for the next run if it is not already picked up
-                               * by another worker.
-                               *
-                               * Take exclusive lock to prevent any other sync worker from picking
-                               * the same table.
-                               */
+                                * Take exclusive lock to prevent any other sync worker from
+                                * picking the same table.
+                                */
                                LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
-                               if (rstate->state != SUBREL_STATE_SYNCDONE &&
-                                       !logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+
+                               /*
+                                * Pick the table for the next run if it is not already picked up
+                                * by another worker.
+                                */
+                               if (!logicalrep_worker_find(MySubscription->oid,
+                                                                                       rstate->relid, false))
```

04. TablesyncWorkerMain

I think rstates should be pfree'd at the end of the outer loop, but it's OK
if other parts do not.

05. repsponse for for post

I tried to move the logicalrep_worker_wakeup call from
clean_sync_worker (end of an iteration) to finish_sync_worker (end of
sync worker). I made table sync much slower for some reason, then I
reverted that change. Maybe I should look a bit more into the reason
why that happened some time.

I want to see the testing method to reproduce the same issue, could you please
share it to -hackers?

0003, 0004

I did not checked yet but I could say same as above:
I want to see the testing method to reproduce the same issue.
Could you please share it to -hackers?
My previous post (an approach for reuse connection) may help the performance.

Best Regards,
Hayato Kuroda
FUJITSU LIMITED

#76Amit Kapila
amit.kapila16@gmail.com
In reply to: Hayato Kuroda (Fujitsu) (#73)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Tue, Jun 27, 2023 at 1:12 PM Hayato Kuroda (Fujitsu)
<kuroda.hayato@fujitsu.com> wrote:

This actually makes sense. I quickly try to do that without adding any
new replication message. As you would expect, it did not work.
I don't really know what's needed to make a connection to last for
more than one iteration. Need to look into this. Happy to hear any
suggestions and thoughts.

It is not clear to me what exactly you tried here which didn't work.
Can you please explain a bit more?

I have analyzed how we handle this. Please see attached the patch (0003) which
allows reusing connection.

Why did you change the application name during the connection?

--
With Regards,
Amit Kapila.

#77Hayato Kuroda (Fujitsu)
kuroda.hayato@fujitsu.com
In reply to: Amit Kapila (#76)
6 attachment(s)
RE: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Dear Amit,

This actually makes sense. I quickly try to do that without adding any
new replication message. As you would expect, it did not work.
I don't really know what's needed to make a connection to last for
more than one iteration. Need to look into this. Happy to hear any
suggestions and thoughts.

It is not clear to me what exactly you tried here which didn't work.
Can you please explain a bit more?

Just to confirm, this is not my part. Melih can answer this...

I have analyzed how we handle this. Please see attached the patch (0003) which
allows reusing connection.

Why did you change the application name during the connection?

It was because the lifetime of tablesync worker is longer than slots's one and
tablesync worker creates temporary replication slots many times, per the target
relation. The name of each slots has relid, so I thought that it was not suitable.
But in the later patch the tablesync worker tries to reuse the slot during the
synchronization, so in this case the application_name should be same as slotname.

I added comment in 0003, and new file 0006 file to use slot name as application_name
again. Note again that the separation was just for specifying changes, Melih can
include them to one part of files if needed.

Best Regards,
Hayato Kuroda
FUJITSU LIMITED

Attachments:

0005-Reuse-Replication-Slot-and-Origin-in-Tablesync.patchapplication/octet-stream; name=0005-Reuse-Replication-Slot-and-Origin-in-Tablesync.patchDownload
From 0092e4a0b5e67416e0bd0ed55d5dd5bbb001c7a8 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH 5/6] Reuse Replication Slot and Origin in Tablesync

This commit allows reusing replication slots and origins during tablesync.

Earlier, a tablesync worker was creating a new replication slot and origin
each time it syncs a new table. With this patch, replication
slots/origins can be reusable for tablesync.

This reduces the overhead of creating/dropping replication slots and origins
and improves tablesync speed significantly especially for empty or small tables.

If the state of the current table is INIT or DATASYNC, tablesync worker needs a
repliation slot/origin. If the worker has not created slot and origin in
its previous runs, it will create those first. Otherwise the worker reuses
slot and origin created by the same worker in previous iterations earlier.
Tables in FINISHEDCOPY are expected to have a replication slot and origin.
Slot and origin names for such tables are persisted in
pg_subscription_rel catalog. Tablesync worker can fetch them and proceed
with existing slot and origin of FINISHEDCOPY tables and does not need to
create new ones.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/catalogs.sgml                    |  31 ++
 src/backend/catalog/pg_subscription.c         | 247 +++++++++++-
 src/backend/commands/subscriptioncmds.c       | 229 +++++++----
 .../replication/logical/applyparallelworker.c |   3 +-
 src/backend/replication/logical/launcher.c    |   8 +-
 src/backend/replication/logical/tablesync.c   | 369 +++++++++++++-----
 src/backend/replication/logical/worker.c      |  62 ++-
 src/include/catalog/pg_subscription.h         |   6 +
 src/include/catalog/pg_subscription_rel.h     |  14 +-
 src/include/replication/slot.h                |   3 +-
 src/include/replication/worker_internal.h     |  23 +-
 11 files changed, 797 insertions(+), 198 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index ed32ca0349..0183ecd21e 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8030,6 +8030,19 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        origin.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>sublastusedid</structfield> <type>int8</type>
+      </para>
+      <para>
+      The last used ID for tablesync workers. It acts as an unique identifier
+      for replication slots which are created by tablesync workers.
+      The last used ID needs to be persisted to make logical replication safely
+      proceed after any interruption. If sublastusedid is 0, then no table has
+      been synced yet.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
@@ -8114,6 +8127,24 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        otherwise null
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srrelslotname</structfield> <type>name</type>
+      </para>
+      <para>
+       Replication slot name that is used for synchronization of relation
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srreloriginname</structfield> <type>name</type>
+      </para>
+      <para>
+       Origin name that is used for tracking synchronization of relation
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index d07f88ce28..152fdaa310 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -108,6 +108,14 @@ GetSubscription(Oid subid, bool missing_ok)
 								   Anum_pg_subscription_suborigin);
 	sub->origin = TextDatumGetCString(datum);
 
+	/* Get last used id */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
+							tup,
+							Anum_pg_subscription_sublastusedid,
+							&isnull);
+	Assert(!isnull);
+	sub->lastusedid = DatumGetInt64(datum);
+
 	ReleaseSysCache(tup);
 
 	return sub;
@@ -199,6 +207,44 @@ DisableSubscription(Oid subid)
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update the last used replication slot ID for the given subscription.
+ */
+void
+UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid)
+{
+	Relation	rel;
+	bool		nulls[Natts_pg_subscription];
+	bool		replaces[Natts_pg_subscription];
+	Datum		values[Natts_pg_subscription];
+	HeapTuple	tup;
+
+	/* Look up the subscription in the catalog */
+	rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+	tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	/* Form a new tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	replaces[Anum_pg_subscription_sublastusedid - 1] = true;
+	values[Anum_pg_subscription_sublastusedid- 1] = Int64GetDatum(lastusedid);
+
+	/* Update the catalog */
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+	heap_freetuple(tup);
+
+	table_close(rel, NoLock);
+}
+
 /*
  * Convert text array to list of strings.
  *
@@ -228,7 +274,7 @@ textarray_to_stringlist(ArrayType *textarray)
  */
 void
 AddSubscriptionRelState(Oid subid, Oid relid, char state,
-						XLogRecPtr sublsn)
+						XLogRecPtr sublsn, char *relslotname, char *reloriginname)
 {
 	Relation	rel;
 	HeapTuple	tup;
@@ -257,6 +303,16 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
 	else
 		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -269,6 +325,60 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Internal function to modify columns for relation state update
+ */
+static void
+UpdateSubscriptionRelState_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char state,
+									XLogRecPtr sublsn)
+{
+	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
+	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+
+	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (sublsn != InvalidXLogRecPtr)
+		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
+	else
+		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication slot update
+ */
+static void
+UpdateSubscriptionRelReplicationSlot_internal(Datum *values,
+											bool *nulls,
+											bool *replaces,
+											char *relslotname)
+{
+	replaces[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication origin update
+ */
+static void
+UpdateSubscriptionRelOrigin_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char *reloriginname)
+{
+	replaces[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+}
+
 /*
  * Update the state of a subscription table.
  */
@@ -299,14 +409,56 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	memset(nulls, false, sizeof(nulls));
 	memset(replaces, false, sizeof(replaces));
 
-	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
-	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
 
-	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
-	if (sublsn != InvalidXLogRecPtr)
-		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
-	else
-		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update replication slot name, origin name and state of
+ * a subscription table in one transaction.
+ */
+void
+UpdateSubscriptionRel(Oid subid,
+					  Oid relid,
+					  char state,
+					  XLogRecPtr sublsn,
+					  char *relslotname,
+					  char *reloriginname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
+	UpdateSubscriptionRelOrigin_internal(values, nulls, replaces, reloriginname);
 
 	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
 							replaces);
@@ -318,6 +470,85 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Get origin name of subscription table.
+ *
+ * reloriginname's value has the replication origin name if the origin exists.
+ */
+void
+GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*originname;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srreloriginname, isnull);
+	if (!*isnull)
+	{
+		originname = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(reloriginname, originname, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
+/*
+ * Get replication slot name of subscription table.
+ *
+ * slotname's value has the replication slot name if the subscription has any.
+ */
+void
+GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*relrepslot;
+	bool		isnull;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srrelslotname, &isnull);
+	if (!isnull)
+	{
+		relrepslot = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(slotname, relrepslot, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
 /*
  * Get state of subscription table.
  *
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 54895ba929..9f5a295ddc 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -710,6 +710,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 		publicationListToArray(publications);
 	values[Anum_pg_subscription_suborigin - 1] =
 		CStringGetTextDatum(opts.origin);
+	values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(0);
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -773,7 +774,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 										 rv->schemaname, rv->relname);
 
 				AddSubscriptionRelState(subid, relid, table_state,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 			}
 
 			/*
@@ -864,6 +865,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
 	bool		must_use_password;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -943,7 +946,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				AddSubscriptionRelState(sub->oid, relid,
 										copy_data ? SUBREL_STATE_INIT : SUBREL_STATE_READY,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 				ereport(DEBUG1,
 						(errmsg_internal("table \"%s.%s\" added to subscription \"%s\"",
 										 rv->schemaname, rv->relname, sub->name)));
@@ -967,6 +970,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				char		state;
 				XLogRecPtr	statelsn;
+				char		slotname[NAMEDATALEN] = {0};
 
 				/*
 				 * Lock pg_subscription_rel with AccessExclusiveLock to
@@ -993,13 +997,36 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/*
+				 * Find the logical replication sync worker. If exists, store
+				 * the slot number for dropping associated replication slots
+				 * later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot_name);
+				}
+				else
+				{
+					/*
+					 * Sync of this relation might be failed in an earlier
+					 * attempt, but the replication slot might still exist.
+					 */
+					GetSubscriptionRelReplicationSlot(sub->oid, relid, slotname);
+					if (strlen(slotname) > 0)
+						sub_remove_slots = lappend(sub_remove_slots, slotname);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
 				 * tablesync origin.
 				 */
-				if (state != SUBREL_STATE_READY)
+				if (state != SUBREL_STATE_READY &&
+					state != SUBREL_STATE_SYNCDONE)
 				{
 					char		originname[NAMEDATALEN];
 
@@ -1027,31 +1054,24 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
-		 * the database operations we won't be able to rollback dropped slots.
+		 * Drop the replication slots associated with tablesync workers for
+		 * removed tables. This has to be at the end because otherwise if
+		 * there is an error while doing the database operations we won't be
+		 * able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			memcpy(syncslotname, lfirst(lc), sizeof(NAMEDATALEN));
+
+			/*
+			 * There is no certainty, maybe the slot does not exist yet. Also,
+			 * if we fail after removing some of the slots, next time, it will
+			 * again try to drop already dropped slots and fail. For these
+			 * reasons, we allow missing_ok = true for the drop.
+			 */
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1474,6 +1494,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	char	   *subname;
 	char	   *conninfo;
 	char	   *slotname;
+	int64		lastusedid;
 	List	   *subworkers;
 	ListCell   *lc;
 	char		originname[NAMEDATALEN];
@@ -1546,6 +1567,14 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	else
 		slotname = NULL;
 
+	/* Get the last used identifier by the subscription */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID, tup,
+							Anum_pg_subscription_sublastusedid, &isnull);
+	if (!isnull)
+		lastusedid = DatumGetInt64(datum);
+	else
+		lastusedid = 0;
+
 	/*
 	 * Since dropping a replication slot is not transactional, the replication
 	 * slot stays dropped even if the transaction rolls back.  So we cannot
@@ -1595,6 +1624,8 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	}
 	list_free(subworkers);
 
+	rstates = GetSubscriptionRelations(subid, true);
+
 	/*
 	 * Remove the no-longer-useful entry in the launcher's table of apply
 	 * worker start times.
@@ -1606,36 +1637,26 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	ApplyLauncherForgetWorkerStartTime(subid);
 
 	/*
-	 * Cleanup of tablesync replication origins.
-	 *
-	 * Any READY-state relations would already have dealt with clean-ups.
+	 * Cleanup of tablesync replication origins associated with the
+	 * subscription, if exists. Try to drop origins by creating all origin
+	 * names created for this subscription.
 	 *
 	 * Note that the state can't change because we have already stopped both
 	 * the apply and tablesync workers and they can't restart because of
 	 * exclusive lock on the subscription.
+	 *
+	 * XXX: This can be handled better instead of looping through all possible
 	 */
-	rstates = GetSubscriptionRelations(subid, true);
-	foreach(lc, rstates)
+	for (int64 i = 1; i <= lastusedid; i++)
 	{
-		SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-		Oid			relid = rstate->relid;
-
-		/* Only cleanup resources of tablesync workers */
-		if (!OidIsValid(relid))
-			continue;
+		char		originname_to_drop[NAMEDATALEN] = {0};
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * It is possible that the origin is not yet created for tablesync
-		 * worker so passing missing_ok = true. This can happen for the states
-		 * before SUBREL_STATE_FINISHEDCOPY.
-		 */
-		ReplicationOriginNameForLogicalRep(subid, relid, originname,
-										   sizeof(originname));
-		replorigin_drop_by_name(originname, true, false);
+		snprintf(originname_to_drop, sizeof(originname_to_drop), "pg_%u_%lld", subid, (long long) i);
+		/* missing_ok = true, since the origin might be already dropped. */
+		replorigin_drop_by_name(originname_to_drop, true, false);
 	}
 
+
 	/* Clean up dependencies */
 	deleteSharedDependencyRecordsFor(SubscriptionRelationId, subid, 0);
 
@@ -1688,39 +1709,17 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
+		List	   *slots = NULL;
 
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
 
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char	   *syncslotname = (char *) lfirst(lc);
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1743,6 +1742,71 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * Get the replication slot names associated with the subscription.
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok)
+{
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+					 " FROM pg_replication_slots"
+					 " WHERE slot_name LIKE 'pg_%i_sync_%%';",
+					 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+					errmsg("could not receive list of slots associated with the subscription %u, error: %s",
+					subid, res->err));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();
+		return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -2155,6 +2219,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -2165,18 +2230,20 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
-			char		syncslotname[NAMEDATALEN] = {0};
-
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
-				 syncslotname);
+				 worker->slot_name);
 		}
 	}
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index f16e2377bf..a57f178087 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -440,7 +440,8 @@ pa_launch_parallel_worker(void)
 										MySubscription->name,
 										MyLogicalRepWorker->userid,
 										InvalidOid,
-										dsm_segment_handle(winfo->dsm_seg));
+										dsm_segment_handle(winfo->dsm_seg),
+										InvalidRepSlotId);
 
 	if (launched)
 	{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index c24e56a11e..d0d8d95a68 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -304,7 +304,7 @@ logicalrep_workers_find(Oid subid, bool only_running)
  */
 bool
 logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
-						 Oid relid, dsm_handle subworker_dsm)
+						 Oid relid, dsm_handle subworker_dsm, int64 slotid)
 {
 	BackgroundWorker bgw;
 	BackgroundWorkerHandle *bgw_handle;
@@ -430,6 +430,9 @@ retry:
 	worker->launch_time = now;
 	worker->in_use = true;
 	worker->generation++;
+	worker->created_slot = false;
+	worker->rep_slot_id = slotid;
+	worker->slot_name = (char *) palloc(NAMEDATALEN);
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -1180,7 +1183,8 @@ ApplyLauncherMain(Datum main_arg)
 				ApplyLauncherSetWorkerStartTime(sub->oid, now);
 				logicalrep_worker_launch(sub->dbid, sub->oid, sub->name,
 										 sub->owner, InvalidOid,
-										 DSM_HANDLE_INVALID);
+										 DSM_HANDLE_INVALID,
+										 InvalidRepSlotId);
 			}
 			else
 			{
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 0fc81355bf..4c81a9b4aa 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -328,40 +328,29 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 								   MyLogicalRepWorker->relid,
 								   MyLogicalRepWorker->relstate,
 								   MyLogicalRepWorker->relstate_lsn);
+		CommitTransactionCommand();
 
 		/*
-		 * End streaming so that LogRepWorkerWalRcvConn can be used to drop
-		 * the slot.
-		 */
-		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
-
-		/*
-		 * Cleanup the tablesync slot.
+		 * Cleanup the tablesync slot. If the slot name used by this worker is
+		 * different from the default slot name for the worker, this means the
+		 * current table had started to being synchronized by another worker
+		 * and replication slot. And this worker is reusing a replication slot
+		 * from a previous attempt. We do not need that replication slot
+		 * anymore.
 		 *
 		 * This has to be done after updating the state because otherwise if
 		 * there is an error while doing the database operations we won't be
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
+										MyLogicalRepWorker->rep_slot_id,
 										syncslotname,
 										sizeof(syncslotname));
 
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
-		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
-
-		CommitTransactionCommand();
-		pgstat_report_stat(false);
-
-		/*
-		 * Start a new transaction to clean up the tablesync origin tracking.
-		 * This transaction will be ended within the finish_sync_worker().
-		 * Now, even, if we fail to remove this here, the apply worker will
-		 * ensure to clean it up afterward.
+		 * We are safe to drop the replication tracking origin after this
+		 * point. Now, even, if we fail to remove this here, the apply worker
+		 * will ensure to clean it up afterward.
 		 *
 		 * We need to do this after the table state is set to SYNCDONE.
 		 * Otherwise, if an error occurs while performing the database
@@ -370,32 +359,70 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 * have been cleared before restart. So, the restarted worker will use
 		 * invalid replication progress state resulting in replay of
 		 * transactions that have already been applied.
+		 *
+		 * Firstly reset the origin session to remove the ownership of the
+		 * slot. This is needed to allow the origin to be dropped or reused
+		 * later.
 		 */
+		replorigin_session_reset();
+		replorigin_session_origin = InvalidRepOriginId;
+		replorigin_session_origin_lsn = InvalidXLogRecPtr;
+		replorigin_session_origin_timestamp = 0;
+
 		StartTransactionCommand();
+		if (MyLogicalRepWorker->slot_name && strcmp(syncslotname, MyLogicalRepWorker->slot_name) != 0)
+		{
+			/*
+			 * End streaming so that LogRepWorkerWalRcvConn can be used to
+			 * drop the slot.
+			 */
+			walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
 
-		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
+			ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+											   MyLogicalRepWorker->relid,
+											   originname,
+											   sizeof(originname));
+
+			/*
+			 * Drop replication origin
+			 *
+			 * There is a chance that the user is concurrently performing refresh
+			 * for the subscription where we remove the table state and its origin
+			 * or the apply worker would have removed this origin. So passing
+			 * missing_ok = true.
+			 */
+			replorigin_drop_by_name(originname, true, false);
+		}
 
 		/*
-		 * Resetting the origin session removes the ownership of the slot.
-		 * This is needed to allow the origin to be dropped.
+		 * We are safe to remove persisted replication slot and origin data,
+		 * since it's already in SYNCDONE state. They will not be needed
+		 * anymore.
 		 */
-		replorigin_session_reset();
-		replorigin_session_origin = InvalidRepOriginId;
-		replorigin_session_origin_lsn = InvalidXLogRecPtr;
-		replorigin_session_origin_timestamp = 0;
+		UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+							  MyLogicalRepWorker->relid,
+							  MyLogicalRepWorker->relstate,
+							  MyLogicalRepWorker->relstate_lsn,
+							  NULL,
+							  NULL);
+		ereport(DEBUG2,
+			(errmsg("process_syncing_tables_for_sync: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					"NULL", "NULL", MyLogicalRepWorker->relstate,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+		CommitTransactionCommand();
+		pgstat_report_stat(false);
 
 		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * There is a chance that the user is concurrently performing refresh
-		 * for the subscription where we remove the table state and its origin
-		 * or the apply worker would have removed this origin. So passing
-		 * missing_ok = true.
+		 * This should return the default origin name for the worker. Even if
+		 * the worker used a different origin for this table, it should be
+		 * dropped and removed from the catalog so far.
 		 */
-		replorigin_drop_by_name(originname, true, false);
+		StartTransactionCommand();
+		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+										   MyLogicalRepWorker->relid,
+										   originname,
+										   sizeof(originname));
 
 		/* Sync worker has completed synchronization of the current table. */
 		MyLogicalRepWorker->is_sync_completed = true;
@@ -492,6 +519,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 			if (current_lsn >= rstate->lsn)
 			{
 				char		originname[NAMEDATALEN];
+				bool		is_origin_null = true;
 
 				rstate->state = SUBREL_STATE_READY;
 				rstate->lsn = current_lsn;
@@ -512,18 +540,31 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				 * error while dropping we won't restart it to drop the
 				 * origin. So passing missing_ok = true.
 				 */
-				ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-												   rstate->relid,
-												   originname,
-												   sizeof(originname));
-				replorigin_drop_by_name(originname, true, false);
+				GetSubscriptionRelOrigin(MyLogicalRepWorker->subid,
+										 rstate->relid, originname,
+										 &is_origin_null);
+
+				if (!is_origin_null)
+				{
+					replorigin_drop_by_name(originname, true, false);
+				}
 
 				/*
 				 * Update the state to READY only after the origin cleanup.
 				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-										   rstate->relid, rstate->state,
-										   rstate->lsn);
+				UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+									  rstate->relid,
+									  rstate->state,
+									  rstate->lsn,
+									  NULL,
+									  NULL);
+				ereport(DEBUG2,
+					(errmsg("process_syncing_tables_for_apply: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+							"NULL", "NULL", rstate->state,
+							rstate->relid, MyLogicalRepWorker->subid)));
+
+				CommitTransactionCommand();
+				started_tx = false;
 			}
 		}
 		else
@@ -612,12 +653,25 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						TimestampDifferenceExceeds(hentry->last_start_time, now,
 												   wal_retrieve_retry_interval))
 					{
+						if (IsTransactionState())
+							CommitTransactionCommand();
+						StartTransactionCommand();
+						started_tx = true;
+
+						MySubscription->lastusedid++;
+						UpdateSubscriptionLastSlotId(MyLogicalRepWorker->subid,
+													 MySubscription->lastusedid);
+						ereport(DEBUG2,
+								(errmsg("process_syncing_tables_for_apply: incremented lastusedid to %lld for subscription %u",
+										(long long) MySubscription->lastusedid, MySubscription->oid)));
+
 						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
 												 MySubscription->oid,
 												 MySubscription->name,
 												 MyLogicalRepWorker->userid,
 												 rstate->relid,
-												 DSM_HANDLE_INVALID);
+												 DSM_HANDLE_INVALID,
+												 MySubscription->lastusedid);
 						hentry->last_start_time = now;
 					}
 				}
@@ -1240,8 +1294,8 @@ copy_table(Relation rel)
  * The name must not exceed NAMEDATALEN - 1 because of remote node constraints
  * on slot name length. We append system_identifier to avoid slot_name
  * collision with subscriptions in other clusters. With the current scheme
- * pg_%u_sync_%u_UINT64_FORMAT (3 + 10 + 6 + 10 + 20 + '\0'), the maximum
- * length of slot_name will be 50.
+ * pg_%u_sync_%lu_UINT64_FORMAT (3 + 10 + 6 + 20 + 20 + '\0'), the maximum
+ * length of slot_name will be 45.
  *
  * The returned slot name is stored in the supplied buffer (syncslotname) with
  * the given size.
@@ -1252,11 +1306,11 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
+ReplicationSlotNameForTablesync(Oid suboid, int64 slotid,
 								char *syncslotname, Size szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%lld_" UINT64_FORMAT, suboid,
+			(long long) slotid, GetSystemIdentifier());
 }
 
 /*
@@ -1282,6 +1336,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	UserContext ucxt;
 	bool		must_use_password;
 	bool		run_as_owner;
+	char	   *prev_slotname;
 
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
@@ -1316,7 +1371,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
+									MyLogicalRepWorker->rep_slot_id,
 									slotname,
 									NAMEDATALEN);
 
@@ -1355,12 +1410,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC ||
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY);
 
+	/*
+	 * See if tablesync of the current relation has been started with another
+	 * replication slot.
+	 *
+	 * Read previous slot name from the catalog, if exists.
+	 */
+	prev_slotname = (char *) palloc(NAMEDATALEN);
+	StartTransactionCommand();
+	GetSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+									  MyLogicalRepWorker->relid,
+									  prev_slotname);
+
 	/* Assign the origin tracking record name. */
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   sizeof(originname));
 
+	CommitTransactionCommand();
+
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC)
 	{
 		/*
@@ -1374,10 +1443,53 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		 * breakdown then it wouldn't have succeeded so trying it next time
 		 * seems like a better bet.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, slotname, true);
+		if (strlen(prev_slotname) > 0)
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);
+
+			StartTransactionCommand();
+			/* Replication origin might still exist. Try to drop */
+			replorigin_drop_by_name(originname, true, false);
+
+			/*
+			 * Remove replication slot and origin name from the relation's
+			 * catalog record
+			 */
+			UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+								  MyLogicalRepWorker->relid,
+								  MyLogicalRepWorker->relstate,
+								  MyLogicalRepWorker->relstate_lsn,
+								  NULL,
+								  NULL);
+			CommitTransactionCommand();
+			ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+						"NULL", "NULL", MyLogicalRepWorker->relstate,
+						MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+		}
 	}
 	else if (MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY)
 	{
+		/*
+		 * At this point, the table that is currently being synchronized
+		 * should have its replication slot name filled in the catalog. The
+		 * tablesync process was started with another sync worker and
+		 * replication slot. We need to continue using the same replication
+		 * slot in this worker too.
+		 */
+		if (strlen(prev_slotname) == 0)
+		{
+			elog(ERROR, "Replication slot could not be found for subscription %u, relation %u",
+				 MyLogicalRepWorker->subid,
+				 MyLogicalRepWorker->relid);
+		}
+
+		/*
+		 * Proceed with the correct replication slot. Use previously created
+		 * replication slot to sync this table.
+		 */
+		memcpy(slotname, prev_slotname, NAMEDATALEN);
+
 		/*
 		 * The COPY phase was previously done, but tablesync then crashed
 		 * before it was able to finish normally.
@@ -1397,7 +1509,9 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 		goto copy_table_done;
 	}
+	pfree(prev_slotname);
 
+	/* Preparing for table copy operation */
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 	MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
 	MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
@@ -1405,11 +1519,31 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   MyLogicalRepWorker->relstate,
-							   MyLogicalRepWorker->relstate_lsn);
+
+	/*
+	 * Refresh the originname in case of having non-existing origin
+	 * from previous failed sync attempts.
+	 * If that's the case, it should be removed from the catalog so far.
+	 * Then, we can continue by reusing the origin created by the current
+	 * worker instead of .
+	 */
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									MyLogicalRepWorker->relid,
+									originname,
+									sizeof(originname));
+
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  MyLogicalRepWorker->relstate,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					slotname, originname, MyLogicalRepWorker->relstate,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+
 	pgstat_report_stat(true);
 
 	StartTransactionCommand();
@@ -1437,48 +1571,95 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						res->err)));
 	walrcv_clear_result(res);
 
+	originid = replorigin_by_name(originname, true);
+
 	/*
 	 * Create a new permanent logical decoding slot. This slot will be used
 	 * for the catchup phase after COPY is done, so tell it to use the
 	 * snapshot to make the final data consistent.
+	 *
+	 * Replication slot will only be created if either this is the first run
+	 * of the worker or we're not using a previous replication slot.
 	 */
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-
-	/*
-	 * Setup replication origin tracking. The purpose of doing this before the
-	 * copy is to avoid doing the copy again due to any error in setting up
-	 * origin tracking.
-	 */
-	originid = replorigin_by_name(originname, true);
-	if (!OidIsValid(originid))
+	if (!MyLogicalRepWorker->created_slot)
 	{
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						   slotname, false /* permanent */ , false /* two_phase */ ,
+						   CRS_USE_SNAPSHOT, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: created replication slot %s for subscription %u",
+						slotname, MyLogicalRepWorker->subid)));
+
 		/*
-		 * Origin tracking does not exist, so create it now.
-		 *
-		 * Then advance to the LSN got from walrcv_create_slot. This is WAL
-		 * logged for the purpose of recovery. Locks are to prevent the
-		 * replication origin from vanishing while advancing.
+		 * Remember that we created the slot so that we will not try to create
+		 * it again.
 		 */
-		originid = replorigin_create(originname);
-
-		LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
-		replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
-						   true /* go backward */ , true /* WAL log */ );
-		UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->created_slot = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
+		/*
+		 * Setup replication origin tracking. The purpose of doing this before
+		 * the copy is to avoid doing the copy again due to any error in
+		 * setting up origin tracking.
+		 */
+		if (!OidIsValid(originid))
+		{
+			/*
+			 * Origin tracking does not exist, so create it now.
+			 */
+			originid = replorigin_create(originname);
+		}
+		else
+		{
+			/*
+			 * At this point, there shouldn't be any existing replication
+			 * origin with the same name.
+			 */
+			ereport(ERROR,
+					(errcode(ERRCODE_DUPLICATE_OBJECT),
+					 errmsg("replication origin \"%s\" already exists",
+							originname)));
+		}
 	}
 	else
 	{
-		ereport(ERROR,
-				(errcode(ERRCODE_DUPLICATE_OBJECT),
-				 errmsg("replication origin \"%s\" already exists",
-						originname)));
+		/*
+		 * Do not create a new replication slot, reuse the existing one
+		 * instead. Use a new snapshot for the replication slot to ensure that
+		 * tablesync and apply proceses are consistent with each other.
+		 */
+		WalRcvStreamOptions options;
+		int			server_version;
+
+		server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+		options.proto.logical.proto_version =
+			server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+			server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+			LOGICALREP_PROTO_VERSION_NUM;
+		options.proto.logical.publication_names = MySubscription->publications;
+
+		walrcv_slot_snapshot(LogRepWorkerWalRcvConn, slotname, &options, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: reusing replication slot %s for relation %u in subscription %u",
+						slotname, MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
 	}
 
+	/*
+	 * Advance to the LSN got from walrcv_create_slot or walrcv_slot_snapshot.
+	 * This is WAL logged for the purpose of recovery. Locks are to prevent
+	 * the replication origin from vanishing while advancing.
+	 *
+	 * Then setup replication origin tracking.
+	 */
+	LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+	replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
+					   true /* go backward */ , true /* WAL log */ );
+	UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+
 	/*
 	 * Make sure that the copy command runs as the table owner, unless the
 	 * user has opted out of that behaviour.
@@ -1537,12 +1718,18 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	 * Update the persisted state to indicate the COPY phase is done; make it
 	 * visible to others.
 	 */
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   SUBREL_STATE_FINISHEDCOPY,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  SUBREL_STATE_FINISHEDCOPY,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					originname, slotname, SUBREL_STATE_FINISHEDCOPY,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
 
 copy_table_done:
 
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 8df960e343..c8f8b72aa7 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -464,8 +464,16 @@ ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 {
 	if (OidIsValid(relid))
 	{
-		/* Replication origin name for tablesync workers. */
-		snprintf(originname, szoriginname, "pg_%u_%u", suboid, relid);
+		bool		is_null = true;
+
+		/*
+		 * Replication origin name for tablesync workers. First, look into the
+		 * catalog. If originname does not exist, then use the default name.
+		 */
+		GetSubscriptionRelOrigin(suboid, relid,
+								 originname, &is_null);
+		if (is_null)
+			snprintf(originname, szoriginname, "pg_%u_%lld", suboid, (long long) MyLogicalRepWorker->rep_slot_id);
 	}
 	else
 	{
@@ -3770,7 +3778,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* Tablesync workers should end streaming before exiting the main loop
+	/*
+	 * Tablesync workers should end streaming before exiting the main loop
 	 * to drop replication slot. Only end streaming here for apply workers.
 	 */
 	if (!am_tablesync_worker())
@@ -4496,6 +4505,9 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 
 	/* allocate slot name in long-lived context */
 	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+
+	/* Keep the replication slot name used for this sync. */
+	MyLogicalRepWorker->slot_name = *myslotname;
 	pfree(syncslotname);
 }
 
@@ -4550,10 +4562,12 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 	/* Start table synchronization. */
 	start_table_sync(origin_startpos, &slotname);
 
+	StartTransactionCommand();
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   originname_size);
+	CommitTransactionCommand();
 
 	set_apply_error_context_origin(originname);
 
@@ -4594,11 +4608,10 @@ run_apply_worker(WalRcvStreamOptions *options,
 				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
 					errmsg("subscription has no replication slot set")));
 
-	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-									   originname, originname_size);
-
 	/* Setup replication origin tracking. */
 	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
 	originid = replorigin_by_name(originname, true);
 	if (!OidIsValid(originid))
 		originid = replorigin_create(originname);
@@ -4875,12 +4888,12 @@ TablesyncWorkerMain(Datum main_arg)
 				memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
 
 				/*
-				* Pick the table for the next run if it is not already picked up
-				* by another worker.
-				*
-				* Take exclusive lock to prevent any other sync worker from picking
-				* the same table.
-				*/
+				 * Pick the table for the next run if it is not already picked up
+				 * by another worker.
+				 *
+				 * Take exclusive lock to prevent any other sync worker from picking
+				 * the same table.
+				 */
 				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
 				if (rstate->state != SUBREL_STATE_SYNCDONE &&
 					!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
@@ -4912,7 +4925,32 @@ TablesyncWorkerMain(Datum main_arg)
 			}
 
 			if (!is_table_found)
+			{
+				TimeLineID	tli;
+
+				/*
+				 * It is important to give an error if we are unable to drop the
+				 * slot, otherwise, it won't be dropped till the corresponding
+				 * subscription is dropped. So passing missing_ok = false.
+				 */
+				if (MyLogicalRepWorker->created_slot)
+				{
+					walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+					ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
+				}
+
+				/*
+				 * Drop replication origin before exiting.
+				 *
+				 * There is a chance that the user is concurrently performing refresh
+				 * for the subscription where we remove the table state and its origin
+				 * or the apply worker would have removed this origin. So passing
+				 * missing_ok = true.
+				 */
+				replorigin_drop_by_name(originname, true, false);
+
 				break;
+			}
 		}
 	}
 
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 1d40eebc78..7e13f59847 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -108,6 +108,9 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
 
 	/* Only publish data originating from the specified origin */
 	text		suborigin BKI_DEFAULT(LOGICALREP_ORIGIN_ANY);
+
+	/* The last used ID to create a replication slot for tablesync */
+	int64		sublastusedid BKI_DEFAULT(0);
 #endif
 } FormData_pg_subscription;
 
@@ -144,6 +147,8 @@ typedef struct Subscription
 	List	   *publications;	/* List of publication names to subscribe to */
 	char	   *origin;			/* Only publish data originating from the
 								 * specified origin */
+	int64		lastusedid;		/* Last used unique ID to create replication
+								 * slots in tablesync */
 } Subscription;
 
 /* Disallow streaming in-progress transactions. */
@@ -164,6 +169,7 @@ typedef struct Subscription
 extern Subscription *GetSubscription(Oid subid, bool missing_ok);
 extern void FreeSubscription(Subscription *sub);
 extern void DisableSubscription(Oid subid);
+extern void UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid);
 
 extern int	CountDBSubscriptions(Oid dbid);
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 60a2bcca23..185164d75e 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -44,6 +44,12 @@ CATALOG(pg_subscription_rel,6102,SubscriptionRelRelationId)
 											 * used for synchronization
 											 * coordination, or NULL if not
 											 * valid */
+	NameData	srrelslotname BKI_FORCE_NULL;	/* name of the replication
+												 * slot for relation in
+												 * subscription */
+	NameData	srreloriginname BKI_FORCE_NULL; /* origin name for relation in
+												 * subscription */
+
 #endif
 } FormData_pg_subscription_rel;
 
@@ -81,10 +87,16 @@ typedef struct SubscriptionRelState
 } SubscriptionRelState;
 
 extern void AddSubscriptionRelState(Oid subid, Oid relid, char state,
-									XLogRecPtr sublsn);
+									XLogRecPtr sublsn, char *relslotname, char *reloriginname);
 extern void UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 									   XLogRecPtr sublsn);
+extern void UpdateSubscriptionRel(Oid subid, Oid relid, char state,
+								  XLogRecPtr sublsn, char *relslotname, char *reloriginname);
+
 extern char GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn);
+extern void GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname);
+extern void GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull);
+
 extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 
 extern bool HasSubscriptionRelations(Oid subid);
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index a8a89dc784..31b2c41893 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -237,8 +237,9 @@ extern bool InvalidateObsoleteReplicationSlots(ReplicationSlotInvalidationCause
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
 extern bool ReplicationSlotName(int index, Name name);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, Size szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, int64 slotid, char *syncslotname, Size szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List *GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 1e9f8e6e72..24006240c9 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -35,6 +35,23 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/*
+	 * Indicates if the sync worker created a replication slot for itself
+	 * in any point of its lifetime.
+	 * False means that the worker has not created a slot yet, and has been
+	 * reusing replication slots created by other workers so far.
+	 */
+	bool		created_slot;
+
+	/*
+	 * Unique identifier for replication slot to be created by tablesnync
+	 * workers, if needed.
+	 */
+	int64		rep_slot_id;
+
+	/* Replication slot name used by the worker. */
+	char	   *slot_name;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
@@ -239,7 +256,8 @@ extern LogicalRepWorker *logicalrep_worker_find(Oid subid, Oid relid,
 extern List *logicalrep_workers_find(Oid subid, bool only_running);
 extern bool logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname,
 									 Oid userid, Oid relid,
-									 dsm_handle subworker_dsm);
+									 dsm_handle subworker_dsm,
+									 int64 slotid);
 extern void logicalrep_worker_stop(Oid subid, Oid relid);
 extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
 extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
@@ -335,4 +353,7 @@ am_parallel_apply_worker(void)
 	return isParallelApplyWorker(MyLogicalRepWorker);
 }
 
+/* Invalid identifier to be used for naming replication slots */
+#define InvalidRepSlotId	0
+
 #endif							/* WORKER_INTERNAL_H */
-- 
2.27.0

0006-Use-slot-name-as-application_name-again.patchapplication/octet-stream; name=0006-Use-slot-name-as-application_name-again.patchDownload
From c1c5c8bf5284a397b3a3efdc83be69285e98baa2 Mon Sep 17 00:00:00 2001
From: Hayato Kuroda <kuroda.hayato@fujitsu.com>
Date: Wed, 28 Jun 2023 06:19:34 +0000
Subject: [PATCH 6/6] Use slot name as application_name again

---
 src/backend/replication/logical/tablesync.c | 15 +++------------
 1 file changed, 3 insertions(+), 12 deletions(-)

diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 4c81a9b4aa..da0264c0b6 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -1382,23 +1382,14 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	 */
 	if (LogRepWorkerWalRcvConn == NULL)
 	{
-		char application_name[NAMEDATALEN];
-
 		/*
-		 * FIXME: set appropriate application_name. Previously, the slot name
-		 * was used becasue the lifetime of the tablesync worker was same as
-		 * that, but now the tablesync worker handles many slots during the
-		 * synchronization so that it is not suitable. So what should be?
-		 * Note that if the tablesync worker starts to reuse the replication
-		 * slot during synchronization, we should use the slot name as
-		 * application_name again.
+		 * Here we use the slot name instead of the subscription name as the
+		 * application_name,
 		 */
-		snprintf(application_name, NAMEDATALEN, "tablesync for %s",
-				 MySubscription->name);
 		LogRepWorkerWalRcvConn =
 			walrcv_connect(MySubscription->conninfo, true,
 						   must_use_password,
-						   application_name, &err);
+						   slotname, &err);
 	}
 
 	if (LogRepWorkerWalRcvConn == NULL)
-- 
2.27.0

0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchapplication/octet-stream; name=0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchDownload
From 46942df115ce0e0d653b932c61d9b4c7cfab7807 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Mon, 5 Jun 2023 15:04:41 +0300
Subject: [PATCH 1/6] Refactor to split Apply and Tablesync Workers

Both apply and tablesync workers were using ApplyWorkerMain() as entry
point. As the name implies, ApplyWorkerMain() should be considered as
the main function for apply workers. Tablesync worker's path was hidden
and does not have enough in common to share the same main function with
apply worker.

Also; most of the code shared by both worker types are already combined
in LogicalRepApplyLoop(). There is no need to combine the rest in
ApplyWorkerMain() anymore.

This commit introduces TablesyncWorkerMain() as a new entry point for
tablesync workers and separates both type of workers from each other.
This aims to increase code readability and help to maintain logical
replication workers separately.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/postmaster/bgworker.c             |   3 +
 .../replication/logical/applyparallelworker.c |   2 +-
 src/backend/replication/logical/launcher.c    |  25 +-
 src/backend/replication/logical/tablesync.c   |   2 +-
 src/backend/replication/logical/worker.c      | 381 +++++++++++-------
 src/include/replication/logicalworker.h       |   1 +
 src/include/replication/worker_internal.h     |   4 +-
 7 files changed, 258 insertions(+), 160 deletions(-)

diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 0dd22b2351..5609919edf 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -131,6 +131,9 @@ static const struct
 	},
 	{
 		"ParallelApplyWorkerMain", ParallelApplyWorkerMain
+	},
+	{
+		"TablesyncWorkerMain", TablesyncWorkerMain
 	}
 };
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 82c1ddcdcb..f16e2377bf 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -942,7 +942,7 @@ ParallelApplyWorkerMain(Datum main_arg)
 	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
 		MyLogicalRepWorker->reply_time = 0;
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 8395ae7b23..f562b1db41 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -459,24 +459,27 @@ retry:
 	snprintf(bgw.bgw_library_name, BGW_MAXLEN, "postgres");
 
 	if (is_parallel_apply_worker)
+	{
 		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ParallelApplyWorkerMain");
-	else
-		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
-
-	if (OidIsValid(relid))
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication worker for subscription %u sync %u", subid, relid);
-	else if (is_parallel_apply_worker)
+				 "logical replication parallel apply worker for subscription %u", subid);
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
 				 "logical replication parallel apply worker for subscription %u", subid);
+	}
+	else if (OidIsValid(relid))
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "TablesyncWorkerMain");
+		snprintf(bgw.bgw_name, BGW_MAXLEN,
+				 "logical replication tablesync worker for subscription %u sync %u", subid, relid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication tablesync worker");
+	}
 	else
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
 				 "logical replication apply worker for subscription %u", subid);
-
-	if (is_parallel_apply_worker)
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
-	else
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication worker");
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication apply worker");
+	}
 
 	bgw.bgw_restart_time = BGW_NEVER_RESTART;
 	bgw.bgw_notify_pid = MyProcPid;
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 6d461654ab..8125bbd170 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -131,7 +131,7 @@ static StringInfo copybuf = NULL;
 /*
  * Exit routine for synchronization worker.
  */
-static void
+void
 pg_attribute_noreturn()
 finish_sync_worker(void)
 {
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 0ee764d68f..b979a755ae 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -392,6 +392,7 @@ static void stream_open_file(Oid subid, TransactionId xid,
 static void stream_write_change(char action, StringInfo s);
 static void stream_open_and_write_change(TransactionId xid, char action, StringInfo s);
 static void stream_close_file(void);
+static void set_stream_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
@@ -4330,6 +4331,69 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+ /* set_stream_options
+  * 	Set logical replication streaming options.
+  *
+  * This function sets streaming options including replication slot name
+  * and origin start position. Workers need these options for logical replication.
+  */
+static void
+set_stream_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+		options->proto.logical.twophase = true;
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4442,13 +4506,134 @@ start_apply(XLogRecPtr origin_startpos)
 }
 
 /*
- * Common initialization for leader apply worker and parallel apply worker.
+ * Runs the tablesync worker.
+ * It starts syncing tables. After a successful sync,
+ * sets streaming options and starts streaming to catchup.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Start table synchronization. */
+	start_table_sync(origin_startpos, &slotname);
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   originname_size);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	/* Start applying changes to catchup. */
+	start_apply(*origin_startpos);
+}
+
+/*
+ * Runs the leader apply worker.
+ * It sets up replication origin, streaming options
+ * and then starts streaming.
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
+{
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+	bool		must_use_password;
+
+	slotname = MySubscription->slotname;
+
+	/*
+	 * This shouldn't happen if the subscription is enabled, but guard
+	 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+	 * crash if slot is NULL.)
+	 */
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+
+	/* Is the use of a password mandatory? */
+	must_use_password = MySubscription->passwordrequired &&
+		!superuser_arg(MySubscription->owner);
+
+	/* Note that the superuser_arg call can access the DB */
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											must_use_password,
+											MySubscription->name, &err);
+
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+	 * We don't really use the output identify_system for anything but it
+	 * does some initializations on the upstream so let's still call it.
+	 */
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+	{
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
+	}
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+
+	/* Run the main loop. */
+	start_apply(*origin_startpos);
+}
+
+/*
+ * Common initialization for logical replication workers; leader apply worker,
+ * parallel apply worker and tablesync worker.
  *
  * Initialize the database connection, in-memory subscription and necessary
  * config options.
  */
 void
-InitializeApplyWorker(void)
+InitializeLogRepWorker(void)
 {
 	MemoryContext oldctx;
 
@@ -4512,7 +4697,8 @@ InitializeApplyWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has started",
+				(errmsg("%s for subscription \"%s\", table \"%s\" has started",
+						get_worker_name(),
 						MySubscription->name,
 						get_rel_name(MyLogicalRepWorker->relid))));
 	else
@@ -4533,7 +4719,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	InitializingApplyWorker = true;
 
@@ -4557,7 +4742,7 @@ ApplyWorkerMain(Datum main_arg)
 	/* Load the libpq-specific functions */
 	load_file("libpqwalreceiver", false);
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
@@ -4565,165 +4750,69 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-		bool		must_use_password;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-
-		/* Is the use of a password mandatory? */
-		must_use_password = MySubscription->passwordrequired &&
-			!superuser_arg(MySubscription->owner);
-
-		/* Note that the superuser_arg call can access the DB */
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												must_use_password,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
-	 * the subscription relation state.
+	 * the subscription relation state. Do this outside the loop to avoid
+	 * exceeding MAX_SYSCACHE_CALLBACKS
 	 */
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
+	/* This is leader apply worker */
+	run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
+	proc_exit(0);
+}
 
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
+/* Logical Replication Tablesync worker entry point */
+void
+TablesyncWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *myslotname = NULL;
+	WalRcvStreamOptions options;
+
+	/* Attach to slot */
+	logicalrep_worker_attach(worker_slot);
+
+	/* Setup signal handling */
+	pqsignal(SIGHUP, SignalHandlerForConfigReload);
+	pqsignal(SIGTERM, die);
+	BackgroundWorkerUnblockSignals();
 
 	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 * We don't currently need any ResourceOwner in a walreceiver process, but
+	 * if we did, we could call CreateAuxProcessResourceOwner here.
 	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
 
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
+	/* Initialise stats to a sanish value */
+	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
+		MyLogicalRepWorker->reply_time = GetCurrentTimestamp();
 
-	if (!am_tablesync_worker())
-	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
-		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+	/* Load the libpq-specific functions */
+	load_file("libpqwalreceiver", false);
 
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
-		}
-		else
-		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-		}
+	InitializeLogRepWorker();
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-								 MySubscription->name,
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-								 "?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
+	/* Connect to the origin and start the replication. */
+	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
+		 MySubscription->conninfo);
 
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+	/*
+	 * Setup callback for syscache so that we know when something changes in
+	 * the subscription relation state. Do this outside the loop to avoid
+	 * exceeding MAX_SYSCACHE_CALLBACKS
+	 */
+	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
+								  invalidate_syncing_table_states,
+								  (Datum) 0);
 
-	proc_exit(0);
+	run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
+
+	finish_sync_worker();
 }
 
 /*
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 39588da79f..bbd71d0b42 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -18,6 +18,7 @@ extern PGDLLIMPORT volatile sig_atomic_t ParallelApplyMessagePending;
 
 extern void ApplyWorkerMain(Datum main_arg);
 extern void ParallelApplyWorkerMain(Datum main_arg);
+extern void TablesyncWorkerMain(Datum main_arg);
 
 extern bool IsLogicalWorker(void);
 extern bool IsLogicalParallelApplyWorker(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 343e781896..7aba034774 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -265,7 +265,7 @@ extern void maybe_reread_subscription(void);
 
 extern void stream_cleanup_files(Oid subid, TransactionId xid);
 
-extern void InitializeApplyWorker(void);
+extern void InitializeLogRepWorker(void);
 
 extern void store_flush_position(XLogRecPtr remote_lsn, XLogRecPtr local_lsn);
 
@@ -307,6 +307,8 @@ extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
 
 #define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)
 
+extern void finish_sync_worker(void);
+
 static inline bool
 am_tablesync_worker(void)
 {
-- 
2.27.0

0002-Reuse-Tablesync-Workers.patchapplication/octet-stream; name=0002-Reuse-Tablesync-Workers.patchDownload
From f04d0fd0099d8c3a3727341ecb868031fb9f93b3 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Mon, 5 Jun 2023 15:45:29 +0300
Subject: [PATCH 2/6] Reuse Tablesync Workers

This commit allows reusing tablesync workers for syncing more than one
table sequentially during their lifetime, instead of exiting after
only syncing one table.

Before this commit, tablesync workers were capable of syncing only one
table. For each table, a new sync worker was launched and that worker would
exit when done processing the table.

Now, tablesync workers are not limited to processing only one
table. When done, they can move to processing another table in
the same subscription.

If there is a table that needs to be synced, an available tablesync
worker picks up that table and syncs it. Each tablesync worker
continues to pick new tables to sync until there are no tables left
requiring synchronization. If there was no available worker to
process the table, then a new tablesync worker will be launched,
provided the number of tablesync workers for the subscription does not
exceed max_sync_workers_per_subscription.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/replication/logical/launcher.c  |   1 +
 src/backend/replication/logical/tablesync.c |  46 ++++++--
 src/backend/replication/logical/worker.c    | 110 +++++++++++++++++++-
 src/include/replication/worker_internal.h   |   7 ++
 4 files changed, 149 insertions(+), 15 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index f562b1db41..c24e56a11e 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -440,6 +440,7 @@ retry:
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
+	worker->is_sync_completed = false;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 8125bbd170..37f073b968 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -129,11 +129,10 @@ static bool FetchTableStates(bool *started_tx);
 static StringInfo copybuf = NULL;
 
 /*
- * Exit routine for synchronization worker.
+ * Prepares the synchronization worker for reuse or exit.
  */
 void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+clean_sync_worker(void)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -145,19 +144,38 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
+	/*
+	 * Disconnect from publisher. Otherwise reused sync workers causes
+	 * exceeding max_wal_senders
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		walrcv_disconnect(LogRepWorkerWalRcvConn);
+		LogRepWorkerWalRcvConn = NULL;
+	}
+
+	/* Find the leader apply worker and signal it. */
+	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
+}
+
+/*
+ * Exit routine for synchronization worker.
+ */
+void
+pg_attribute_noreturn()
+finish_sync_worker(void)
+{
+	clean_sync_worker();
+
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
 	StartTransactionCommand();
 	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
+			(errmsg("logical replication table synchronization worker for subscription \"%s\" has finished",
+					MySubscription->name)));
 	CommitTransactionCommand();
 
-	/* Find the leader apply worker and signal it. */
-	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
-
 	/* Stop gracefully */
 	proc_exit(0);
 }
@@ -379,7 +397,15 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 */
 		replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
+		/* Sync worker has completed synchronization of the current table. */
+		MyLogicalRepWorker->is_sync_completed = true;
+
+		ereport(LOG,
+		(errmsg("logical replication table synchronization worker for subscription \"%s\", relation \"%s\" with relid %u has finished",
+				MySubscription->name,
+				get_rel_name(MyLogicalRepWorker->relid),
+				MyLogicalRepWorker->relid)));
+		CommitTransactionCommand();
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index b979a755ae..4a0f402ad4 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3624,6 +3624,20 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can call process_syncing_tables_for_sync.
+				 *
+				 * process_syncing_tables_for_sync decides whether the sync of the
+				 * current table is completed. If it is completed, streaming must
+				 * be already ended. So, we can break the loop.
+				 */
+				if (MyLogicalRepWorker->is_sync_completed)
+				{
+					endofstream = true;
+					break;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -3643,6 +3657,15 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+
+			/*
+			 * If is_sync_completed is true, this means that the tablesync worker
+			 * is done with synchronization. Streaming has already been ended by
+			 * process_syncing_tables_for_sync. We should move to the next table
+			 * if needed, or exit.
+			 */
+			if (MyLogicalRepWorker->is_sync_completed)
+				endofstream = true;
 		}
 
 		/* Cleanup the memory. */
@@ -3745,8 +3768,11 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/* Tablesync workers should end streaming before exiting the main loop
+	 * to drop replication slot. Only end streaming here for apply workers.
+	 */
+	if (!am_tablesync_worker())
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
 }
 
 /*
@@ -4517,6 +4543,8 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 					 int originname_size,
 					 XLogRecPtr *origin_startpos)
 {
+	MyLogicalRepWorker->is_sync_completed = false;
+
 	/* Start table synchronization. */
 	start_table_sync(origin_startpos, &slotname);
 
@@ -4697,10 +4725,11 @@ InitializeLogRepWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("%s for subscription \"%s\", table \"%s\" has started",
+				(errmsg("%s for subscription \"%s\", table \"%s\" with relid %u has started",
 						get_worker_name(),
 						MySubscription->name,
-						get_rel_name(MyLogicalRepWorker->relid))));
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
 	else
 		ereport(LOG,
 		/* translator: first %s is the name of logical replication worker */
@@ -4810,7 +4839,78 @@ TablesyncWorkerMain(Datum main_arg)
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
+	/*
+	 * The loop where worker does its job. It loops until there is no relation
+	 * left to sync.
+	 */
+	for (;;)
+	{
+		List	   *rstates;
+		SubscriptionRelState *rstate;
+		ListCell   *lc;
+		bool 	is_table_found = false;
+
+		run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
+
+		if (IsTransactionState())
+			CommitTransactionCommand();
+
+		if (MyLogicalRepWorker->is_sync_completed)
+		{
+			/* This transaction will be committed by clean_sync_worker. */
+			StartTransactionCommand();
+
+			/*
+			 * Check if there is any table whose relation state is still INIT.
+			 * If a table in INIT state is found, the worker will not be finished,
+			 * it will be reused instead.
+			 */
+			rstates = GetSubscriptionRelations(MySubscription->oid, true);
+			rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
+
+			foreach(lc, rstates)
+			{
+				memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
+
+				/*
+				* Pick the table for the next run if it is not already picked up
+				* by another worker.
+				*
+				* Take exclusive lock to prevent any other sync worker from picking
+				* the same table.
+				*/
+				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
+				if (rstate->state != SUBREL_STATE_SYNCDONE &&
+					!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+				{
+					/* Update worker state for the next table */
+					MyLogicalRepWorker->relid = rstate->relid;
+					MyLogicalRepWorker->relstate = rstate->state;
+					MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+					LWLockRelease(LogicalRepWorkerLock);
+
+					/* found a table for next iteration */
+					is_table_found = true;
+					clean_sync_worker();
+
+					StartTransactionCommand();
+					ereport(LOG,
+							(errmsg("%s for subscription \"%s\" has moved to sync table \"%s\" with relid %u.",
+									get_worker_name(),
+									MySubscription->name,
+									get_rel_name(MyLogicalRepWorker->relid),
+									MyLogicalRepWorker->relid)));
+					CommitTransactionCommand();
+
+					break;
+				}
+				LWLockRelease(LogicalRepWorkerLock);
+			}
+
+			if (!is_table_found)
+				break;
+		}
+	}
 
 	finish_sync_worker();
 }
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 7aba034774..1e9f8e6e72 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -56,6 +56,12 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/*
+	 * Indicates whether tablesync worker has completed sycning its assigned
+	 * table. If true, no need to continue with that table.
+	 */
+	bool		is_sync_completed;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction or
@@ -308,6 +314,7 @@ extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
 #define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)
 
 extern void finish_sync_worker(void);
+extern void clean_sync_worker(void);
 
 static inline bool
 am_tablesync_worker(void)
-- 
2.27.0

0003-reuse-connection-when-tablesync-workers-change-the-t.patchapplication/octet-stream; name=0003-reuse-connection-when-tablesync-workers-change-the-t.patchDownload
From f621571b4406b71e8522cc7846030cb3d11fccef Mon Sep 17 00:00:00 2001
From: Hayato Kuroda <kuroda.hayato@fujitsu.com>
Date: Tue, 27 Jun 2023 07:10:45 +0000
Subject: [PATCH 3/6] reuse connection when tablesync workers change the target

---
 src/backend/replication/logical/tablesync.c | 31 ++++++++++++++++-----
 src/backend/replication/logical/worker.c    | 20 +++++++------
 src/backend/replication/walsender.c         |  6 ++++
 3 files changed, 42 insertions(+), 15 deletions(-)

diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 37f073b968..0fc81355bf 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -1321,14 +1321,31 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 									NAMEDATALEN);
 
 	/*
-	 * Here we use the slot name instead of the subscription name as the
-	 * application_name, so that it is different from the leader apply worker,
-	 * so that synchronous replication can distinguish them.
+	 * Connect to publisher if not yet. The application_name must be also
+	 * different from the leader apply worker because synchronous replication
+	 * must distinguish them.
 	 */
-	LogRepWorkerWalRcvConn =
-		walrcv_connect(MySubscription->conninfo, true,
-					   must_use_password,
-					   slotname, &err);
+	if (LogRepWorkerWalRcvConn == NULL)
+	{
+		char application_name[NAMEDATALEN];
+
+		/*
+		 * FIXME: set appropriate application_name. Previously, the slot name
+		 * was used becasue the lifetime of the tablesync worker was same as
+		 * that, but now the tablesync worker handles many slots during the
+		 * synchronization so that it is not suitable. So what should be?
+		 * Note that if the tablesync worker starts to reuse the replication
+		 * slot during synchronization, we should use the slot name as
+		 * application_name again.
+		 */
+		snprintf(application_name, NAMEDATALEN, "tablesync for %s",
+				 MySubscription->name);
+		LogRepWorkerWalRcvConn =
+			walrcv_connect(MySubscription->conninfo, true,
+						   must_use_password,
+						   application_name, &err);
+	}
+
 	if (LogRepWorkerWalRcvConn == NULL)
 		ereport(ERROR,
 				(errcode(ERRCODE_CONNECTION_FAILURE),
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 4a0f402ad4..8df960e343 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3498,19 +3498,21 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 	/*
 	 * Init the ApplyMessageContext which we clean up after each replication
-	 * protocol message.
+	 * protocol message, if needed.
 	 */
-	ApplyMessageContext = AllocSetContextCreate(ApplyContext,
-												"ApplyMessageContext",
-												ALLOCSET_DEFAULT_SIZES);
+	if (!ApplyMessageContext)
+		ApplyMessageContext = AllocSetContextCreate(ApplyContext,
+													"ApplyMessageContext",
+													ALLOCSET_DEFAULT_SIZES);
 
 	/*
 	 * This memory context is used for per-stream data when the streaming mode
 	 * is enabled. This context is reset on each stream stop.
 	 */
-	LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
-													"LogicalStreamingContext",
-													ALLOCSET_DEFAULT_SIZES);
+	if (!LogicalStreamingContext)
+		LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
+														"LogicalStreamingContext",
+														ALLOCSET_DEFAULT_SIZES);
 
 	/* mark as idle, before starting to loop */
 	pgstat_report_activity(STATE_IDLE, NULL);
@@ -4891,7 +4893,9 @@ TablesyncWorkerMain(Datum main_arg)
 
 					/* found a table for next iteration */
 					is_table_found = true;
-					clean_sync_worker();
+
+					CommitTransactionCommand();
+					pgstat_report_stat(true);
 
 					StartTransactionCommand();
 					ereport(LOG,
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index d3a136b6f5..429d00f2f0 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1828,6 +1828,12 @@ exec_replication_command(const char *cmd_string)
 				set_ps_display(cmdtag);
 				PreventInTransactionBlock(true, cmdtag);
 
+				/*
+				 * Initialize the flag again because this streaming may be
+				 * second time.
+				 */
+				streamingDoneSending = streamingDoneReceiving = false;
+
 				if (cmd->kind == REPLICATION_KIND_PHYSICAL)
 					StartReplication(cmd);
 				else
-- 
2.27.0

0004-Add-replication-protocol-cmd-to-create-a-snapshot.patchapplication/octet-stream; name=0004-Add-replication-protocol-cmd-to-create-a-snapshot.patchDownload
From 8c9494ccd472d710141f2aaf9b546658bb5486a7 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 13 Oct 2022 17:05:45 +0300
Subject: [PATCH 4/6] Add replication protocol cmd to create a snapshot

Introduced CREATE_REPLICATION_SNAPSHOT to be able to create and use a
snapshot without creating a new replication slot, but by using an
existing slot.

CREATE_REPLICATION_SNAPSHOT simply does what CREATE_REPLICATION_SLOT does
without creating a new replication slot.

CREATE_REPLICATION_SNAPSHOT command imports the snapshot into the current
transaction and returns consistent_point. The changes earlier than the
consistent_point will be applied by importing the snapshot. All changes
later than the consistent_point will be available to be consumed from
the replication slot.

This is useful for reusing replication slots in logical replication.
Otherwise, tablesync workers cannot start from a consistent point to copy
a relation and then apply changes by consuming from replication slot.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/protocol.sgml                    |  31 ++++++
 .../libpqwalreceiver/libpqwalreceiver.c       |  69 +++++++++++-
 src/backend/replication/logical/logical.c     |  40 ++++++-
 .../replication/logical/logicalfuncs.c        |   1 +
 src/backend/replication/repl_gram.y           |  18 ++-
 src/backend/replication/repl_scanner.l        |   2 +
 src/backend/replication/slotfuncs.c           |   1 +
 src/backend/replication/walsender.c           | 104 +++++++++++++++++-
 src/include/nodes/replnodes.h                 |  11 ++
 src/include/replication/logical.h             |   1 +
 src/include/replication/walreceiver.h         |  13 +++
 src/tools/pgindent/typedefs.list              |   2 +
 12 files changed, 289 insertions(+), 4 deletions(-)

diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index b11d9a6ba3..f7d5acee93 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2595,6 +2595,37 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
      </listitem>
     </varlistentry>
 
+    <varlistentry id="protocol-replication-replication-slot-snapshot">
+     <term><literal>CREATE_REPLICATION_SNAPSHOT</literal> <replaceable class="parameter">slot_name</replaceable> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
+      <indexterm><primary>CREATE_REPLICATION_SNAPSHOT</primary></indexterm>
+     </term>
+     <listitem>
+      <para>
+       Creates a snapshot including all the changes from the replication slot until
+       the point at which the replication slot becomes consistent. Then the snapshot
+       is used in the current transaction. This command is currently only supported
+       for logical replication slots.
+      </para>
+
+      <para>
+       In response to this command, the server will return a one-row result set,
+       containing the following field:
+       <variablelist>
+        <varlistentry>
+         <term><literal>consistent_point</literal> (<type>text</type>)</term>
+         <listitem>
+          <para>
+           The WAL location at which the slot became consistent.  This is the
+           earliest location from which streaming can start on this replication
+           slot.
+          </para>
+         </listitem>
+        </varlistentry>
+       </variablelist>
+      </para>
+     </listitem>
+    </varlistentry>
+
     <varlistentry id="protocol-replication-base-backup" xreflabel="BASE_BACKUP">
      <term><literal>BASE_BACKUP</literal> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
       <indexterm><primary>BASE_BACKUP</primary></indexterm>
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index dc9c5c82d9..cf88a3b7d4 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -82,6 +82,8 @@ static WalRcvExecResult *libpqrcv_exec(WalReceiverConn *conn,
 									   const int nRetTypes,
 									   const Oid *retTypes);
 static void libpqrcv_disconnect(WalReceiverConn *conn);
+static void libpqrcv_slot_snapshot(WalReceiverConn *conn, char *slotname,
+								   const WalRcvStreamOptions *options, XLogRecPtr *lsn);
 
 static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_connect = libpqrcv_connect,
@@ -98,7 +100,8 @@ static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_create_slot = libpqrcv_create_slot,
 	.walrcv_get_backend_pid = libpqrcv_get_backend_pid,
 	.walrcv_exec = libpqrcv_exec,
-	.walrcv_disconnect = libpqrcv_disconnect
+	.walrcv_disconnect = libpqrcv_disconnect,
+	.walrcv_slot_snapshot = libpqrcv_slot_snapshot
 };
 
 /* Prototypes for private functions */
@@ -982,6 +985,70 @@ libpqrcv_create_slot(WalReceiverConn *conn, const char *slotname,
 	return snapshot;
 }
 
+/*
+ * TODO
+ */
+static void
+libpqrcv_slot_snapshot(WalReceiverConn *conn,
+					   char *slotname,
+					   const WalRcvStreamOptions *options,
+					   XLogRecPtr *lsn)
+{
+	StringInfoData cmd;
+	PGresult   *res;
+	char	   *pubnames_str;
+	List	   *pubnames;
+	char	   *pubnames_literal;
+
+	initStringInfo(&cmd);
+
+	/* Build the command. */
+	appendStringInfo(&cmd, "CREATE_REPLICATION_SNAPSHOT \"%s\"", slotname);
+	appendStringInfoString(&cmd, " (");
+	appendStringInfo(&cmd, " proto_version '%u'",
+					 options->proto.logical.proto_version);
+
+	/* Add publication names. */
+	pubnames = options->proto.logical.publication_names;
+	pubnames_str = stringlist_to_identifierstr(conn->streamConn, pubnames);
+	if (!pubnames_str)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	pubnames_literal = PQescapeLiteral(conn->streamConn, pubnames_str,
+									   strlen(pubnames_str));
+	if (!pubnames_literal)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	appendStringInfo(&cmd, ", publication_names %s", pubnames_literal);
+	PQfreemem(pubnames_literal);
+	pfree(pubnames_str);
+
+	appendStringInfoString(&cmd, " )");
+
+	/* Execute the command. */
+	res = libpqrcv_PQexec(conn->streamConn, cmd.data);
+	pfree(cmd.data);
+
+	if (PQresultStatus(res) != PGRES_TUPLES_OK)
+	{
+		PQclear(res);
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg("Could not create a snapshot by replication slot \"%s\": %s",
+						slotname, pchomp(PQerrorMessage(conn->streamConn)))));
+	}
+
+	if (lsn)
+		*lsn = DatumGetLSN(DirectFunctionCall1Coll(pg_lsn_in, InvalidOid,
+												   CStringGetDatum(PQgetvalue(res, 0, 0))));
+
+	PQclear(res);
+}
+
 /*
  * Return PID of remote backend process.
  */
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 41243d0187..3c11f8e1dd 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -476,6 +476,10 @@ CreateInitDecodingContext(const char *plugin,
  * fast_forward
  *		bypass the generation of logical changes.
  *
+ * need_full_snapshot
+ * 		if true, must obtain a snapshot able to read all tables;
+ *  	if false, one that can read only catalogs is acceptable.
+ *
  * xl_routine
  *		XLogReaderRoutine used by underlying xlogreader
  *
@@ -494,6 +498,7 @@ LogicalDecodingContext *
 CreateDecodingContext(XLogRecPtr start_lsn,
 					  List *output_plugin_options,
 					  bool fast_forward,
+					  bool need_full_snapshot,
 					  XLogReaderRoutine *xl_routine,
 					  LogicalOutputPluginWriterPrepareWrite prepare_write,
 					  LogicalOutputPluginWriterWrite do_write,
@@ -502,6 +507,7 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 	LogicalDecodingContext *ctx;
 	ReplicationSlot *slot;
 	MemoryContext old_context;
+	TransactionId xmin_horizon = InvalidTransactionId;
 
 	/* shorter lines... */
 	slot = MyReplicationSlot;
@@ -571,8 +577,40 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 		start_lsn = slot->data.confirmed_flush;
 	}
 
+
+	/*
+	 * We need to determine a safe xmin horizon to start decoding from if we
+	 * want to create a snapshot too. Otherwise we would end up with a
+	 * snapshot that cannot be imported since xmin value from the snapshot may
+	 * be less than the oldest safe xmin. To avoid this call
+	 * GetOldestSafeDecodingTransactionId() to return a safe xmin value, which
+	 * can be used while exporting/importing the snapshot.
+	 *
+	 * So we have to acquire the ProcArrayLock to prevent computation of new
+	 * xmin horizons by other backends, get the safe decoding xid, and inform
+	 * the slot machinery about the new limit. Once that's done the
+	 * ProcArrayLock can be released as the slot machinery now is protecting
+	 * against vacuum.
+	 */
+	if (need_full_snapshot)
+	{
+		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+		xmin_horizon = GetOldestSafeDecodingTransactionId(!need_full_snapshot);
+
+		SpinLockAcquire(&slot->mutex);
+		slot->effective_catalog_xmin = xmin_horizon;
+		slot->data.catalog_xmin = xmin_horizon;
+		slot->effective_xmin = xmin_horizon;
+		SpinLockRelease(&slot->mutex);
+
+		ReplicationSlotsComputeRequiredXmin(true);
+
+		LWLockRelease(ProcArrayLock);
+	}
+
 	ctx = StartupDecodingContext(output_plugin_options,
-								 start_lsn, InvalidTransactionId, false,
+								 start_lsn, xmin_horizon, need_full_snapshot,
 								 fast_forward, xl_routine, prepare_write,
 								 do_write, update_progress);
 
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index 55a24c02c9..85c5cdb633 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -208,6 +208,7 @@ pg_logical_slot_get_changes_guts(FunctionCallInfo fcinfo, bool confirm, bool bin
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									options,
 									false,
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/repl_gram.y b/src/backend/replication/repl_gram.y
index 0c874e33cf..957a5cc653 100644
--- a/src/backend/replication/repl_gram.y
+++ b/src/backend/replication/repl_gram.y
@@ -65,6 +65,7 @@ Node *replication_parse_result;
 %token K_CREATE_REPLICATION_SLOT
 %token K_DROP_REPLICATION_SLOT
 %token K_TIMELINE_HISTORY
+%token K_CREATE_REPLICATION_SNAPSHOT
 %token K_WAIT
 %token K_TIMELINE
 %token K_PHYSICAL
@@ -80,7 +81,7 @@ Node *replication_parse_result;
 %type <node>	command
 %type <node>	base_backup start_replication start_logical_replication
 				create_replication_slot drop_replication_slot identify_system
-				read_replication_slot timeline_history show
+				read_replication_slot timeline_history show create_replication_snapshot
 %type <list>	generic_option_list
 %type <defelt>	generic_option
 %type <uintval>	opt_timeline
@@ -114,6 +115,7 @@ command:
 			| read_replication_slot
 			| timeline_history
 			| show
+			| create_replication_snapshot
 			;
 
 /*
@@ -307,6 +309,19 @@ timeline_history:
 				}
 			;
 
+/*
+ * CREATE_REPLICATION_SNAPSHOT %s options
+ */
+create_replication_snapshot:
+			K_CREATE_REPLICATION_SNAPSHOT var_name plugin_options
+				{
+					CreateReplicationSnapshotCmd *n = makeNode(CreateReplicationSnapshotCmd);
+					n->slotname = $2;
+					n->options = $3;
+					$$ = (Node *) n;
+				}
+			;
+
 opt_physical:
 			K_PHYSICAL
 			| /* EMPTY */
@@ -400,6 +415,7 @@ ident_or_keyword:
 			| K_CREATE_REPLICATION_SLOT	{ $$ = "create_replication_slot"; }
 			| K_DROP_REPLICATION_SLOT		{ $$ = "drop_replication_slot"; }
 			| K_TIMELINE_HISTORY			{ $$ = "timeline_history"; }
+			| K_CREATE_REPLICATION_SNAPSHOT	{ $$ = "create_replication_snapshot"; }
 			| K_WAIT						{ $$ = "wait"; }
 			| K_TIMELINE					{ $$ = "timeline"; }
 			| K_PHYSICAL					{ $$ = "physical"; }
diff --git a/src/backend/replication/repl_scanner.l b/src/backend/replication/repl_scanner.l
index cb467ca46f..5ba2e9c54b 100644
--- a/src/backend/replication/repl_scanner.l
+++ b/src/backend/replication/repl_scanner.l
@@ -126,6 +126,7 @@ START_REPLICATION	{ return K_START_REPLICATION; }
 CREATE_REPLICATION_SLOT		{ return K_CREATE_REPLICATION_SLOT; }
 DROP_REPLICATION_SLOT		{ return K_DROP_REPLICATION_SLOT; }
 TIMELINE_HISTORY	{ return K_TIMELINE_HISTORY; }
+CREATE_REPLICATION_SNAPSHOT	{ return K_CREATE_REPLICATION_SNAPSHOT; }
 PHYSICAL			{ return K_PHYSICAL; }
 RESERVE_WAL			{ return K_RESERVE_WAL; }
 LOGICAL				{ return K_LOGICAL; }
@@ -303,6 +304,7 @@ replication_scanner_is_replication_command(void)
 		case K_DROP_REPLICATION_SLOT:
 		case K_READ_REPLICATION_SLOT:
 		case K_TIMELINE_HISTORY:
+		case K_CREATE_REPLICATION_SNAPSHOT:
 		case K_SHOW:
 			/* Yes; push back the first token so we can parse later. */
 			repl_pushed_back_token = first_token;
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 6035cf4816..c5b2d5b61f 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -486,6 +486,7 @@ pg_logical_replication_slot_advance(XLogRecPtr moveto)
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									NIL,
 									true,	/* fast_forward */
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 429d00f2f0..974ab5ca35 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -238,6 +238,7 @@ static void CreateReplicationSlot(CreateReplicationSlotCmd *cmd);
 static void DropReplicationSlot(DropReplicationSlotCmd *cmd);
 static void StartReplication(StartReplicationCmd *cmd);
 static void StartLogicalReplication(StartReplicationCmd *cmd);
+static void CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd);
 static void ProcessStandbyMessage(void);
 static void ProcessStandbyReplyMessage(void);
 static void ProcessStandbyHSFeedbackMessage(void);
@@ -1284,7 +1285,7 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	 * are reported early.
 	 */
 	logical_decoding_ctx =
-		CreateDecodingContext(cmd->startpoint, cmd->options, false,
+		CreateDecodingContext(cmd->startpoint, cmd->options, false, false,
 							  XL_ROUTINE(.page_read = logical_read_xlog_page,
 										 .segment_open = WalSndSegmentOpen,
 										 .segment_close = wal_segment_close),
@@ -1336,6 +1337,98 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	EndCommand(&qc, DestRemote, false);
 }
 
+/*
+ * Create a snapshot from an existing replication slot.
+ */
+static void
+CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd)
+{
+	Snapshot	snap;
+	LogicalDecodingContext *ctx;
+	char		xloc[MAXFNAMELEN];
+	DestReceiver *dest;
+	TupOutputState *tstate;
+	TupleDesc	tupdesc;
+	Datum		values[1];
+	bool		nulls[1] = {0};
+
+	Assert(!MyReplicationSlot);
+
+	CheckLogicalDecodingRequirements();
+
+	if (!IsTransactionBlock())
+		ereport(ERROR,
+				(errmsg("%s must be called inside a transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (XactIsoLevel != XACT_REPEATABLE_READ)
+		ereport(ERROR,
+				(errmsg("%s must be called in REPEATABLE READ isolation mode transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (!XactReadOnly)
+		ereport(ERROR,
+				(errmsg("%s must be called in a read only transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (FirstSnapshotSet)
+		ereport(ERROR,
+				(errmsg("%s must be called before any query",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (IsSubTransaction())
+		ereport(ERROR,
+				(errmsg("%s must not be called in a subtransaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	ReplicationSlotAcquire(cmd->slotname, false);
+
+	ctx = CreateDecodingContext(MyReplicationSlot->data.restart_lsn,
+								cmd->options,
+								false,
+								true,
+								XL_ROUTINE(.page_read = logical_read_xlog_page,
+										   .segment_open = WalSndSegmentOpen,
+										   .segment_close = wal_segment_close),
+								WalSndPrepareWrite, WalSndWriteData,
+								WalSndUpdateProgress);
+
+	/*
+	 * Signal that we don't need the timeout mechanism. We're just creating
+	 * the snapshot with the replication slot and don't yet accept feedback
+	 * messages or send keepalives. As we possibly need to wait for further
+	 * WAL the walsender would otherwise possibly be killed too soon.
+	 */
+	last_reply_timestamp = 0;
+
+	/* build initial snapshot, might take a while */
+	DecodingContextFindStartpoint(ctx);
+
+	snap = SnapBuildInitialSnapshot(ctx->snapshot_builder);
+	RestoreTransactionSnapshot(snap, MyProc);
+
+	/* Don't need the decoding context anymore */
+	FreeDecodingContext(ctx);
+
+	/* Create a tuple to send consistent WAL location */
+	snprintf(xloc, sizeof(xloc), "%X/%X",
+			 LSN_FORMAT_ARGS(MyReplicationSlot->data.confirmed_flush));
+
+	dest = CreateDestReceiver(DestRemoteSimple);
+	tupdesc = CreateTemplateTupleDesc(1);
+	TupleDescInitBuiltinEntry(tupdesc, (AttrNumber) 1, "consistent_point",
+							  TEXTOID, -1, 0);
+	tstate = begin_tup_output_tupdesc(dest, tupdesc, &TTSOpsVirtual);
+
+	/* consistent wal location */
+	values[0] = CStringGetTextDatum(xloc);
+
+	do_tup_output(tstate, values, nulls);
+	end_tup_output(tstate);
+
+	ReplicationSlotRelease();
+}
+
 /*
  * LogicalDecodingContext 'prepare_write' callback.
  *
@@ -1870,6 +1963,15 @@ exec_replication_command(const char *cmd_string)
 			}
 			break;
 
+		case T_CreateReplicationSnapshotCmd:
+			{
+				cmdtag = "CREATE_REPLICATION_SNAPSHOT";
+				set_ps_display(cmdtag);
+				CreateReplicationSnapshot((CreateReplicationSnapshotCmd *) cmd_node);
+				EndReplicationCommand(cmdtag);
+				break;
+			}
+
 		default:
 			elog(ERROR, "unrecognized replication command node tag: %u",
 				 cmd_node->type);
diff --git a/src/include/nodes/replnodes.h b/src/include/nodes/replnodes.h
index 4321ba8f86..154ab74f33 100644
--- a/src/include/nodes/replnodes.h
+++ b/src/include/nodes/replnodes.h
@@ -108,4 +108,15 @@ typedef struct TimeLineHistoryCmd
 	TimeLineID	timeline;
 } TimeLineHistoryCmd;
 
+/* ----------------------
+ *		CREATE_REPLICATION_SNAPSHOT command
+ * ----------------------
+ */
+typedef struct CreateReplicationSnapshotCmd
+{
+	NodeTag		type;
+	char	   *slotname;
+	List	   *options;
+} CreateReplicationSnapshotCmd;
+
 #endif							/* REPLNODES_H */
diff --git a/src/include/replication/logical.h b/src/include/replication/logical.h
index 5f49554ea0..6535786a0e 100644
--- a/src/include/replication/logical.h
+++ b/src/include/replication/logical.h
@@ -125,6 +125,7 @@ extern LogicalDecodingContext *CreateInitDecodingContext(const char *plugin,
 extern LogicalDecodingContext *CreateDecodingContext(XLogRecPtr start_lsn,
 													 List *output_plugin_options,
 													 bool fast_forward,
+													 bool need_full_snapshot,
 													 XLogReaderRoutine *xl_routine,
 													 LogicalOutputPluginWriterPrepareWrite prepare_write,
 													 LogicalOutputPluginWriterWrite do_write,
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 281626fa6f..f01e5f23a7 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -386,6 +386,16 @@ typedef WalRcvExecResult *(*walrcv_exec_fn) (WalReceiverConn *conn,
  */
 typedef void (*walrcv_disconnect_fn) (WalReceiverConn *conn);
 
+/*
+ * walrcv_slot_snapshot_fn
+ *
+ * Create a snapshot by an existing replication slot
+ */
+typedef void (*walrcv_slot_snapshot_fn) (WalReceiverConn *conn,
+										 char *slotname,
+										 const WalRcvStreamOptions *options,
+										 XLogRecPtr *lsn);
+
 typedef struct WalReceiverFunctionsType
 {
 	walrcv_connect_fn walrcv_connect;
@@ -403,6 +413,7 @@ typedef struct WalReceiverFunctionsType
 	walrcv_get_backend_pid_fn walrcv_get_backend_pid;
 	walrcv_exec_fn walrcv_exec;
 	walrcv_disconnect_fn walrcv_disconnect;
+	walrcv_slot_snapshot_fn walrcv_slot_snapshot;
 } WalReceiverFunctionsType;
 
 extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
@@ -437,6 +448,8 @@ extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
 	WalReceiverFunctions->walrcv_exec(conn, exec, nRetTypes, retTypes)
 #define walrcv_disconnect(conn) \
 	WalReceiverFunctions->walrcv_disconnect(conn)
+#define walrcv_slot_snapshot(conn, slotname, options, lsn) \
+	WalReceiverFunctions->walrcv_slot_snapshot(conn, slotname, options, lsn)
 
 static inline void
 walrcv_clear_result(WalRcvExecResult *walres)
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 260854747b..985a4ab7fd 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2348,6 +2348,7 @@ ReplicationSlotInvalidationCause
 ReplicationSlotOnDisk
 ReplicationSlotPersistency
 ReplicationSlotPersistentData
+CreateReplicationSnapshotCmd
 ReplicationState
 ReplicationStateCtl
 ReplicationStateOnDisk
@@ -3853,6 +3854,7 @@ walrcv_receive_fn
 walrcv_send_fn
 walrcv_server_version_fn
 walrcv_startstreaming_fn
+walrcv_slot_snapshot_fn
 wchar2mb_with_len_converter
 wchar_t
 win32_deadchild_waitinfo
-- 
2.27.0

#78Amit Kapila
amit.kapila16@gmail.com
In reply to: Hayato Kuroda (Fujitsu) (#77)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Wed, Jun 28, 2023 at 12:02 PM Hayato Kuroda (Fujitsu)
<kuroda.hayato@fujitsu.com> wrote:

I have analyzed how we handle this. Please see attached the patch (0003) which
allows reusing connection.

Why did you change the application name during the connection?

It was because the lifetime of tablesync worker is longer than slots's one and
tablesync worker creates temporary replication slots many times, per the target
relation. The name of each slots has relid, so I thought that it was not suitable.

Okay, but let's try to give a unique application name to each
tablesync worker for the purpose of pg_stat_activity and synchronous
replication (as mentioned in existing comments as well). One idea is
to generate a name like pg_<sub_id>_sync_<worker_slot> but feel free
to suggest if you have any better ideas.

But in the later patch the tablesync worker tries to reuse the slot during the
synchronization, so in this case the application_name should be same as slotname.

Fair enough. I am slightly afraid that if we can't show the benefits
with later patches then we may need to drop them but at this stage I
feel we need to investigate why those are not helping?

--
With Regards,
Amit Kapila.

#79Amit Kapila
amit.kapila16@gmail.com
In reply to: Amit Kapila (#78)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Mon, Jul 3, 2023 at 9:42 AM Amit Kapila <amit.kapila16@gmail.com> wrote:

On Wed, Jun 28, 2023 at 12:02 PM Hayato Kuroda (Fujitsu)
<kuroda.hayato@fujitsu.com> wrote:

But in the later patch the tablesync worker tries to reuse the slot during the
synchronization, so in this case the application_name should be same as slotname.

Fair enough. I am slightly afraid that if we can't show the benefits
with later patches then we may need to drop them but at this stage I
feel we need to investigate why those are not helping?

On thinking about this, I think the primary benefit we were expecting
by saving network round trips for slot drop/create but now that we
anyway need an extra round trip to establish a snapshot, so such a
benefit was not visible. This is just a theory so we should validate
it. The another idea as discussed before [1]/messages/by-id/CAGPVpCRWEVhXa7ovrhuSQofx4to7o22oU9iKtrOgAOtz_=Y6vg@mail.gmail.com could be to try copying
multiple tables in a single transaction. Now, keeping a transaction
open for a longer time could have side-effects on the publisher node.
So, we probably need to ensure that we don't perform multiple large
syncs and even for smaller tables (and later sequences) perform it
only for some threshold number of tables which we can figure out by
some tests. Also, the other safety-check could be that anytime we need
to perform streaming (sync with apply worker), we won't copy more
tables in same transaction.

Thoughts?

[1]: /messages/by-id/CAGPVpCRWEVhXa7ovrhuSQofx4to7o22oU9iKtrOgAOtz_=Y6vg@mail.gmail.com

--
With Regards,
Amit Kapila.

#80vignesh C
vignesh21@gmail.com
In reply to: Hayato Kuroda (Fujitsu) (#77)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Wed, 28 Jun 2023 at 12:02, Hayato Kuroda (Fujitsu)
<kuroda.hayato@fujitsu.com> wrote:

Dear Amit,

This actually makes sense. I quickly try to do that without adding any
new replication message. As you would expect, it did not work.
I don't really know what's needed to make a connection to last for
more than one iteration. Need to look into this. Happy to hear any
suggestions and thoughts.

It is not clear to me what exactly you tried here which didn't work.
Can you please explain a bit more?

Just to confirm, this is not my part. Melih can answer this...

I have analyzed how we handle this. Please see attached the patch (0003) which
allows reusing connection.

Why did you change the application name during the connection?

It was because the lifetime of tablesync worker is longer than slots's one and
tablesync worker creates temporary replication slots many times, per the target
relation. The name of each slots has relid, so I thought that it was not suitable.
But in the later patch the tablesync worker tries to reuse the slot during the
synchronization, so in this case the application_name should be same as slotname.

I added comment in 0003, and new file 0006 file to use slot name as application_name
again. Note again that the separation was just for specifying changes, Melih can
include them to one part of files if needed.

Few comments:
1) Should these error messages say "Could not create a snapshot by
replication slot":
+       if (!pubnames_str)
+               ereport(ERROR,
+                               (errcode(ERRCODE_OUT_OF_MEMORY),
 /* likely guess */
+                                errmsg("could not start WAL streaming: %s",
+
pchomp(PQerrorMessage(conn->streamConn)))));
+       pubnames_literal = PQescapeLiteral(conn->streamConn, pubnames_str,
+
    strlen(pubnames_str));
+       if (!pubnames_literal)
+               ereport(ERROR,
+                               (errcode(ERRCODE_OUT_OF_MEMORY),
 /* likely guess */
+                                errmsg("could not start WAL streaming: %s",
+
pchomp(PQerrorMessage(conn->streamConn)))));
+       appendStringInfo(&cmd, ", publication_names %s", pubnames_literal);
+       PQfreemem(pubnames_literal);
+       pfree(pubnames_str);
2) These checks are present in CreateReplicationSlot too, can we have
a common function to check these for both CreateReplicationSlot and
CreateReplicationSnapshot:
+       if (!IsTransactionBlock())
+               ereport(ERROR,
+                               (errmsg("%s must be called inside a
transaction",
+
"CREATE_REPLICATION_SNAPSHOT ...")));
+
+       if (XactIsoLevel != XACT_REPEATABLE_READ)
+               ereport(ERROR,
+                               (errmsg("%s must be called in
REPEATABLE READ isolation mode transaction",
+
"CREATE_REPLICATION_SNAPSHOT ...")));
+
+       if (!XactReadOnly)
+               ereport(ERROR,
+                               (errmsg("%s must be called in a read
only transaction",
+
"CREATE_REPLICATION_SNAPSHOT ...")));
+
+       if (FirstSnapshotSet)
+               ereport(ERROR,
+                               (errmsg("%s must be called before any query",
+
"CREATE_REPLICATION_SNAPSHOT ...")));
+
+       if (IsSubTransaction())
+               ereport(ERROR,
+                               (errmsg("%s must not be called in a
subtransaction",
+
"CREATE_REPLICATION_SNAPSHOT ...")));
3) Probably we can add the function header at this point of time:
+/*
+ * TODO
+ */
+static void
+libpqrcv_slot_snapshot(WalReceiverConn *conn,
+                                          char *slotname,
+                                          const WalRcvStreamOptions *options,
+                                          XLogRecPtr *lsn)
4) Either or relation name or relid should be sufficient here, no need
to print both:
                                       StartTransactionCommand();
+                                       ereport(LOG,
+                                                       (errmsg("%s
for subscription \"%s\" has moved to sync table \"%s\" with relid
%u.",
+
 get_worker_name(),
+
 MySubscription->name,
+
 get_rel_name(MyLogicalRepWorker->relid),
+
 MyLogicalRepWorker->relid)));
+                                       CommitTransactionCommand();
5) Why is this check of logicalrep_worker_find is required required,
will it not be sufficient to pick the relations that are in
SUBREL_STATE_INIT state?
+                               /*
+                               * Pick the table for the next run if
it is not already picked up
+                               * by another worker.
+                               *
+                               * Take exclusive lock to prevent any
other sync worker from picking
+                               * the same table.
+                               */
+                               LWLockAcquire(LogicalRepWorkerLock,
LW_EXCLUSIVE);
+                               if (rstate->state != SUBREL_STATE_SYNCDONE &&
+
!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+                               {
+                                       /* Update worker state for the
next table */

6) This comment is missed while refactoring:
- /* Build logical replication streaming options. */
- options.logical = true;
- options.startpoint = origin_startpos;
- options.slotname = myslotname;

7) We could keep twophase and origin as the same order as it was
earlier so that it is easy to review that the existing code is kept as
is in this case:
+       options->proto.logical.publication_names = MySubscription->publications;
+       options->proto.logical.binary = MySubscription->binary;
+       options->proto.logical.twophase = false;
+       options->proto.logical.origin = pstrdup(MySubscription->origin);
+
+       /*
+        * Assign the appropriate option value for streaming option according to
+        * the 'streaming' mode and the publisher's ability to support
that mode.
+        */
+       if (server_version >= 160000 &&
8)  There are few indentation issues, we could run pgindent once:
8.a)
+               /* Sync worker has completed synchronization of the
current table. */
+               MyLogicalRepWorker->is_sync_completed = true;
+
+               ereport(LOG,
+               (errmsg("logical replication table synchronization
worker for subscription \"%s\", relation \"%s\" with relid %u has
finished",
+                               MySubscription->name,
+                               get_rel_name(MyLogicalRepWorker->relid),
+                               MyLogicalRepWorker->relid)));
+               CommitTransactionCommand();
8.b)
+               ereport(DEBUG2,
+                       (errmsg("process_syncing_tables_for_sync:
updated originname: %s, slotname: %s, state: %c for relation \"%u\" in
subscription \"%u\".",
+                                       "NULL", "NULL",
MyLogicalRepWorker->relstate,
+                                       MyLogicalRepWorker->relid,
MyLogicalRepWorker->subid)));
+               CommitTransactionCommand();
+               pgstat_report_stat(false);

Regards,
Vignesh

#81Hayato Kuroda (Fujitsu)
kuroda.hayato@fujitsu.com
In reply to: Amit Kapila (#78)
6 attachment(s)
RE: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Dear Amit,

I have analyzed how we handle this. Please see attached the patch (0003)

which

allows reusing connection.

Why did you change the application name during the connection?

It was because the lifetime of tablesync worker is longer than slots's one and
tablesync worker creates temporary replication slots many times, per the target
relation. The name of each slots has relid, so I thought that it was not suitable.

Okay, but let's try to give a unique application name to each
tablesync worker for the purpose of pg_stat_activity and synchronous
replication (as mentioned in existing comments as well). One idea is
to generate a name like pg_<sub_id>_sync_<worker_slot> but feel free
to suggest if you have any better ideas.

Good point. The slot id is passed as an argument of TablesyncWorkerMain(),
so I passed it to LogicalRepSyncTableStart(). PSA new set.

But in the later patch the tablesync worker tries to reuse the slot during the
synchronization, so in this case the application_name should be same as

slotname.

Fair enough. I am slightly afraid that if we can't show the benefits
with later patches then we may need to drop them but at this stage I
feel we need to investigate why those are not helping?

Agreed. Now I'm planning to do performance testing independently. We can discuss
based on that or Melih's one.

Best Regards,
Hayato Kuroda
FUJITSU LIMITED

Attachments:

v2-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchapplication/octet-stream; name=v2-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchDownload
From 510ed4e64a6985f251ececc72f3b1a4312a3a185 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Mon, 5 Jun 2023 15:04:41 +0300
Subject: [PATCH v2 1/6] Refactor to split Apply and Tablesync Workers

Both apply and tablesync workers were using ApplyWorkerMain() as entry
point. As the name implies, ApplyWorkerMain() should be considered as
the main function for apply workers. Tablesync worker's path was hidden
and does not have enough in common to share the same main function with
apply worker.

Also; most of the code shared by both worker types are already combined
in LogicalRepApplyLoop(). There is no need to combine the rest in
ApplyWorkerMain() anymore.

This commit introduces TablesyncWorkerMain() as a new entry point for
tablesync workers and separates both type of workers from each other.
This aims to increase code readability and help to maintain logical
replication workers separately.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/postmaster/bgworker.c             |   3 +
 .../replication/logical/applyparallelworker.c |   2 +-
 src/backend/replication/logical/launcher.c    |  25 +-
 src/backend/replication/logical/tablesync.c   |   2 +-
 src/backend/replication/logical/worker.c      | 381 +++++++++++-------
 src/include/replication/logicalworker.h       |   1 +
 src/include/replication/worker_internal.h     |   4 +-
 7 files changed, 258 insertions(+), 160 deletions(-)

diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 5b4bd71694..505e38376c 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -131,6 +131,9 @@ static const struct
 	},
 	{
 		"ParallelApplyWorkerMain", ParallelApplyWorkerMain
+	},
+	{
+		"TablesyncWorkerMain", TablesyncWorkerMain
 	}
 };
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 6fb96148f4..1d4e83c4c1 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -942,7 +942,7 @@ ParallelApplyWorkerMain(Datum main_arg)
 	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
 		MyLogicalRepWorker->reply_time = 0;
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 542af7d863..c49a7fe811 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -459,24 +459,27 @@ retry:
 	snprintf(bgw.bgw_library_name, MAXPGPATH, "postgres");
 
 	if (is_parallel_apply_worker)
+	{
 		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ParallelApplyWorkerMain");
-	else
-		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
-
-	if (OidIsValid(relid))
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication worker for subscription %u sync %u", subid, relid);
-	else if (is_parallel_apply_worker)
+				 "logical replication parallel apply worker for subscription %u", subid);
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
 				 "logical replication parallel apply worker for subscription %u", subid);
+	}
+	else if (OidIsValid(relid))
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "TablesyncWorkerMain");
+		snprintf(bgw.bgw_name, BGW_MAXLEN,
+				 "logical replication tablesync worker for subscription %u sync %u", subid, relid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication tablesync worker");
+	}
 	else
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
 				 "logical replication apply worker for subscription %u", subid);
-
-	if (is_parallel_apply_worker)
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
-	else
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication worker");
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication apply worker");
+	}
 
 	bgw.bgw_restart_time = BGW_NEVER_RESTART;
 	bgw.bgw_notify_pid = MyProcPid;
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 6d461654ab..8125bbd170 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -131,7 +131,7 @@ static StringInfo copybuf = NULL;
 /*
  * Exit routine for synchronization worker.
  */
-static void
+void
 pg_attribute_noreturn()
 finish_sync_worker(void)
 {
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 0ee764d68f..b979a755ae 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -392,6 +392,7 @@ static void stream_open_file(Oid subid, TransactionId xid,
 static void stream_write_change(char action, StringInfo s);
 static void stream_open_and_write_change(TransactionId xid, char action, StringInfo s);
 static void stream_close_file(void);
+static void set_stream_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
@@ -4330,6 +4331,69 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+ /* set_stream_options
+  * 	Set logical replication streaming options.
+  *
+  * This function sets streaming options including replication slot name
+  * and origin start position. Workers need these options for logical replication.
+  */
+static void
+set_stream_options(WalRcvStreamOptions *options, char *slotname, XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+		options->proto.logical.twophase = true;
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4442,13 +4506,134 @@ start_apply(XLogRecPtr origin_startpos)
 }
 
 /*
- * Common initialization for leader apply worker and parallel apply worker.
+ * Runs the tablesync worker.
+ * It starts syncing tables. After a successful sync,
+ * sets streaming options and starts streaming to catchup.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Start table synchronization. */
+	start_table_sync(origin_startpos, &slotname);
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   originname_size);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	/* Start applying changes to catchup. */
+	start_apply(*origin_startpos);
+}
+
+/*
+ * Runs the leader apply worker.
+ * It sets up replication origin, streaming options
+ * and then starts streaming.
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
+{
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+	bool		must_use_password;
+
+	slotname = MySubscription->slotname;
+
+	/*
+	 * This shouldn't happen if the subscription is enabled, but guard
+	 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+	 * crash if slot is NULL.)
+	 */
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+
+	/* Is the use of a password mandatory? */
+	must_use_password = MySubscription->passwordrequired &&
+		!superuser_arg(MySubscription->owner);
+
+	/* Note that the superuser_arg call can access the DB */
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											must_use_password,
+											MySubscription->name, &err);
+
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+	 * We don't really use the output identify_system for anything but it
+	 * does some initializations on the upstream so let's still call it.
+	 */
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+	{
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
+	}
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+
+	/* Run the main loop. */
+	start_apply(*origin_startpos);
+}
+
+/*
+ * Common initialization for logical replication workers; leader apply worker,
+ * parallel apply worker and tablesync worker.
  *
  * Initialize the database connection, in-memory subscription and necessary
  * config options.
  */
 void
-InitializeApplyWorker(void)
+InitializeLogRepWorker(void)
 {
 	MemoryContext oldctx;
 
@@ -4512,7 +4697,8 @@ InitializeApplyWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has started",
+				(errmsg("%s for subscription \"%s\", table \"%s\" has started",
+						get_worker_name(),
 						MySubscription->name,
 						get_rel_name(MyLogicalRepWorker->relid))));
 	else
@@ -4533,7 +4719,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	InitializingApplyWorker = true;
 
@@ -4557,7 +4742,7 @@ ApplyWorkerMain(Datum main_arg)
 	/* Load the libpq-specific functions */
 	load_file("libpqwalreceiver", false);
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
@@ -4565,165 +4750,69 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-		bool		must_use_password;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-
-		/* Is the use of a password mandatory? */
-		must_use_password = MySubscription->passwordrequired &&
-			!superuser_arg(MySubscription->owner);
-
-		/* Note that the superuser_arg call can access the DB */
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												must_use_password,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
-	 * the subscription relation state.
+	 * the subscription relation state. Do this outside the loop to avoid
+	 * exceeding MAX_SYSCACHE_CALLBACKS
 	 */
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
+	/* This is leader apply worker */
+	run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
+	proc_exit(0);
+}
 
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
+/* Logical Replication Tablesync worker entry point */
+void
+TablesyncWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *myslotname = NULL;
+	WalRcvStreamOptions options;
+
+	/* Attach to slot */
+	logicalrep_worker_attach(worker_slot);
+
+	/* Setup signal handling */
+	pqsignal(SIGHUP, SignalHandlerForConfigReload);
+	pqsignal(SIGTERM, die);
+	BackgroundWorkerUnblockSignals();
 
 	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 * We don't currently need any ResourceOwner in a walreceiver process, but
+	 * if we did, we could call CreateAuxProcessResourceOwner here.
 	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
 
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
+	/* Initialise stats to a sanish value */
+	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
+		MyLogicalRepWorker->reply_time = GetCurrentTimestamp();
 
-	if (!am_tablesync_worker())
-	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
-		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+	/* Load the libpq-specific functions */
+	load_file("libpqwalreceiver", false);
 
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
-		}
-		else
-		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-		}
+	InitializeLogRepWorker();
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-								 MySubscription->name,
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-								 "?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
+	/* Connect to the origin and start the replication. */
+	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
+		 MySubscription->conninfo);
 
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+	/*
+	 * Setup callback for syscache so that we know when something changes in
+	 * the subscription relation state. Do this outside the loop to avoid
+	 * exceeding MAX_SYSCACHE_CALLBACKS
+	 */
+	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
+								  invalidate_syncing_table_states,
+								  (Datum) 0);
 
-	proc_exit(0);
+	run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
+
+	finish_sync_worker();
 }
 
 /*
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 39588da79f..bbd71d0b42 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -18,6 +18,7 @@ extern PGDLLIMPORT volatile sig_atomic_t ParallelApplyMessagePending;
 
 extern void ApplyWorkerMain(Datum main_arg);
 extern void ParallelApplyWorkerMain(Datum main_arg);
+extern void TablesyncWorkerMain(Datum main_arg);
 
 extern bool IsLogicalWorker(void);
 extern bool IsLogicalParallelApplyWorker(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 343e781896..7aba034774 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -265,7 +265,7 @@ extern void maybe_reread_subscription(void);
 
 extern void stream_cleanup_files(Oid subid, TransactionId xid);
 
-extern void InitializeApplyWorker(void);
+extern void InitializeLogRepWorker(void);
 
 extern void store_flush_position(XLogRecPtr remote_lsn, XLogRecPtr local_lsn);
 
@@ -307,6 +307,8 @@ extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
 
 #define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)
 
+extern void finish_sync_worker(void);
+
 static inline bool
 am_tablesync_worker(void)
 {
-- 
2.27.0

v2-0002-Reuse-Tablesync-Workers.patchapplication/octet-stream; name=v2-0002-Reuse-Tablesync-Workers.patchDownload
From 8b416d01acd50ab0a7d97d64f122e7ca9903a2b7 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Mon, 5 Jun 2023 15:45:29 +0300
Subject: [PATCH v2 2/6] Reuse Tablesync Workers

This commit allows reusing tablesync workers for syncing more than one
table sequentially during their lifetime, instead of exiting after
only syncing one table.

Before this commit, tablesync workers were capable of syncing only one
table. For each table, a new sync worker was launched and that worker would
exit when done processing the table.

Now, tablesync workers are not limited to processing only one
table. When done, they can move to processing another table in
the same subscription.

If there is a table that needs to be synced, an available tablesync
worker picks up that table and syncs it. Each tablesync worker
continues to pick new tables to sync until there are no tables left
requiring synchronization. If there was no available worker to
process the table, then a new tablesync worker will be launched,
provided the number of tablesync workers for the subscription does not
exceed max_sync_workers_per_subscription.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/replication/logical/launcher.c  |   1 +
 src/backend/replication/logical/tablesync.c |  46 ++++++--
 src/backend/replication/logical/worker.c    | 110 +++++++++++++++++++-
 src/include/replication/worker_internal.h   |   7 ++
 4 files changed, 149 insertions(+), 15 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index c49a7fe811..4c3f72cb46 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -440,6 +440,7 @@ retry:
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
+	worker->is_sync_completed = false;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 8125bbd170..37f073b968 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -129,11 +129,10 @@ static bool FetchTableStates(bool *started_tx);
 static StringInfo copybuf = NULL;
 
 /*
- * Exit routine for synchronization worker.
+ * Prepares the synchronization worker for reuse or exit.
  */
 void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+clean_sync_worker(void)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -145,19 +144,38 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
+	/*
+	 * Disconnect from publisher. Otherwise reused sync workers causes
+	 * exceeding max_wal_senders
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		walrcv_disconnect(LogRepWorkerWalRcvConn);
+		LogRepWorkerWalRcvConn = NULL;
+	}
+
+	/* Find the leader apply worker and signal it. */
+	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
+}
+
+/*
+ * Exit routine for synchronization worker.
+ */
+void
+pg_attribute_noreturn()
+finish_sync_worker(void)
+{
+	clean_sync_worker();
+
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
 	StartTransactionCommand();
 	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
+			(errmsg("logical replication table synchronization worker for subscription \"%s\" has finished",
+					MySubscription->name)));
 	CommitTransactionCommand();
 
-	/* Find the leader apply worker and signal it. */
-	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
-
 	/* Stop gracefully */
 	proc_exit(0);
 }
@@ -379,7 +397,15 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 */
 		replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
+		/* Sync worker has completed synchronization of the current table. */
+		MyLogicalRepWorker->is_sync_completed = true;
+
+		ereport(LOG,
+		(errmsg("logical replication table synchronization worker for subscription \"%s\", relation \"%s\" with relid %u has finished",
+				MySubscription->name,
+				get_rel_name(MyLogicalRepWorker->relid),
+				MyLogicalRepWorker->relid)));
+		CommitTransactionCommand();
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index b979a755ae..4a0f402ad4 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3624,6 +3624,20 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can call process_syncing_tables_for_sync.
+				 *
+				 * process_syncing_tables_for_sync decides whether the sync of the
+				 * current table is completed. If it is completed, streaming must
+				 * be already ended. So, we can break the loop.
+				 */
+				if (MyLogicalRepWorker->is_sync_completed)
+				{
+					endofstream = true;
+					break;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -3643,6 +3657,15 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+
+			/*
+			 * If is_sync_completed is true, this means that the tablesync worker
+			 * is done with synchronization. Streaming has already been ended by
+			 * process_syncing_tables_for_sync. We should move to the next table
+			 * if needed, or exit.
+			 */
+			if (MyLogicalRepWorker->is_sync_completed)
+				endofstream = true;
 		}
 
 		/* Cleanup the memory. */
@@ -3745,8 +3768,11 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/* Tablesync workers should end streaming before exiting the main loop
+	 * to drop replication slot. Only end streaming here for apply workers.
+	 */
+	if (!am_tablesync_worker())
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
 }
 
 /*
@@ -4517,6 +4543,8 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 					 int originname_size,
 					 XLogRecPtr *origin_startpos)
 {
+	MyLogicalRepWorker->is_sync_completed = false;
+
 	/* Start table synchronization. */
 	start_table_sync(origin_startpos, &slotname);
 
@@ -4697,10 +4725,11 @@ InitializeLogRepWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("%s for subscription \"%s\", table \"%s\" has started",
+				(errmsg("%s for subscription \"%s\", table \"%s\" with relid %u has started",
 						get_worker_name(),
 						MySubscription->name,
-						get_rel_name(MyLogicalRepWorker->relid))));
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
 	else
 		ereport(LOG,
 		/* translator: first %s is the name of logical replication worker */
@@ -4810,7 +4839,78 @@ TablesyncWorkerMain(Datum main_arg)
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
+	/*
+	 * The loop where worker does its job. It loops until there is no relation
+	 * left to sync.
+	 */
+	for (;;)
+	{
+		List	   *rstates;
+		SubscriptionRelState *rstate;
+		ListCell   *lc;
+		bool 	is_table_found = false;
+
+		run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
+
+		if (IsTransactionState())
+			CommitTransactionCommand();
+
+		if (MyLogicalRepWorker->is_sync_completed)
+		{
+			/* This transaction will be committed by clean_sync_worker. */
+			StartTransactionCommand();
+
+			/*
+			 * Check if there is any table whose relation state is still INIT.
+			 * If a table in INIT state is found, the worker will not be finished,
+			 * it will be reused instead.
+			 */
+			rstates = GetSubscriptionRelations(MySubscription->oid, true);
+			rstate = (SubscriptionRelState *) palloc(sizeof(SubscriptionRelState));
+
+			foreach(lc, rstates)
+			{
+				memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
+
+				/*
+				* Pick the table for the next run if it is not already picked up
+				* by another worker.
+				*
+				* Take exclusive lock to prevent any other sync worker from picking
+				* the same table.
+				*/
+				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
+				if (rstate->state != SUBREL_STATE_SYNCDONE &&
+					!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+				{
+					/* Update worker state for the next table */
+					MyLogicalRepWorker->relid = rstate->relid;
+					MyLogicalRepWorker->relstate = rstate->state;
+					MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+					LWLockRelease(LogicalRepWorkerLock);
+
+					/* found a table for next iteration */
+					is_table_found = true;
+					clean_sync_worker();
+
+					StartTransactionCommand();
+					ereport(LOG,
+							(errmsg("%s for subscription \"%s\" has moved to sync table \"%s\" with relid %u.",
+									get_worker_name(),
+									MySubscription->name,
+									get_rel_name(MyLogicalRepWorker->relid),
+									MyLogicalRepWorker->relid)));
+					CommitTransactionCommand();
+
+					break;
+				}
+				LWLockRelease(LogicalRepWorkerLock);
+			}
+
+			if (!is_table_found)
+				break;
+		}
+	}
 
 	finish_sync_worker();
 }
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 7aba034774..1e9f8e6e72 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -56,6 +56,12 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/*
+	 * Indicates whether tablesync worker has completed sycning its assigned
+	 * table. If true, no need to continue with that table.
+	 */
+	bool		is_sync_completed;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction or
@@ -308,6 +314,7 @@ extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
 #define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)
 
 extern void finish_sync_worker(void);
+extern void clean_sync_worker(void);
 
 static inline bool
 am_tablesync_worker(void)
-- 
2.27.0

v2-0003-reuse-connection-when-tablesync-workers-change-th.patchapplication/octet-stream; name=v2-0003-reuse-connection-when-tablesync-workers-change-th.patchDownload
From 7e9731104d17056509df919b6b6122e6a3f1fce8 Mon Sep 17 00:00:00 2001
From: Hayato Kuroda <kuroda.hayato@fujitsu.com>
Date: Tue, 27 Jun 2023 07:10:45 +0000
Subject: [PATCH v2 3/6] reuse connection when tablesync workers change the
 target

---
 src/backend/replication/logical/tablesync.c | 33 ++++++++++++++++-----
 src/backend/replication/logical/worker.c    | 33 +++++++++++++--------
 src/backend/replication/walsender.c         |  6 ++++
 src/include/replication/worker_internal.h   |  2 +-
 4 files changed, 52 insertions(+), 22 deletions(-)

diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 37f073b968..8d44ed207f 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -1268,7 +1268,7 @@ ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
  * The returned slot name is palloc'ed in current memory context.
  */
 char *
-LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
+LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 {
 	char	   *slotname;
 	char	   *err;
@@ -1321,14 +1321,31 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 									NAMEDATALEN);
 
 	/*
-	 * Here we use the slot name instead of the subscription name as the
-	 * application_name, so that it is different from the leader apply worker,
-	 * so that synchronous replication can distinguish them.
+	 * Connect to publisher if not yet. The application_name must be also
+	 * different from the leader apply worker because synchronous replication
+	 * must distinguish them.
 	 */
-	LogRepWorkerWalRcvConn =
-		walrcv_connect(MySubscription->conninfo, true,
-					   must_use_password,
-					   slotname, &err);
+	if (LogRepWorkerWalRcvConn == NULL)
+	{
+		char application_name[NAMEDATALEN];
+
+		/*
+		 * FIXME: set appropriate application_name. Previously, the slot name
+		 * was used becasue the lifetime of the tablesync worker was same as
+		 * that, but now the tablesync worker handles many slots during the
+		 * synchronization so that it is not suitable. So what should be?
+		 * Note that if the tablesync worker starts to reuse the replication
+		 * slot during synchronization, we should use the slot name as
+		 * application_name again.
+		 */
+		snprintf(application_name, NAMEDATALEN, "pg_%u_sync_%i",
+				 MySubscription->oid, worker_slot);
+		LogRepWorkerWalRcvConn =
+			walrcv_connect(MySubscription->conninfo, true,
+						   must_use_password,
+						   application_name, &err);
+	}
+
 	if (LogRepWorkerWalRcvConn == NULL)
 		ereport(ERROR,
 				(errcode(ERRCODE_CONNECTION_FAILURE),
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 4a0f402ad4..5420675ce8 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3498,19 +3498,21 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 	/*
 	 * Init the ApplyMessageContext which we clean up after each replication
-	 * protocol message.
+	 * protocol message, if needed.
 	 */
-	ApplyMessageContext = AllocSetContextCreate(ApplyContext,
-												"ApplyMessageContext",
-												ALLOCSET_DEFAULT_SIZES);
+	if (!ApplyMessageContext)
+		ApplyMessageContext = AllocSetContextCreate(ApplyContext,
+													"ApplyMessageContext",
+													ALLOCSET_DEFAULT_SIZES);
 
 	/*
 	 * This memory context is used for per-stream data when the streaming mode
 	 * is enabled. This context is reset on each stream stop.
 	 */
-	LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
-													"LogicalStreamingContext",
-													ALLOCSET_DEFAULT_SIZES);
+	if (!LogicalStreamingContext)
+		LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
+														"LogicalStreamingContext",
+														ALLOCSET_DEFAULT_SIZES);
 
 	/* mark as idle, before starting to loop */
 	pgstat_report_activity(STATE_IDLE, NULL);
@@ -4462,7 +4464,7 @@ TwoPhaseTransactionGid(Oid subid, TransactionId xid, char *gid, int szgid)
  * are not repeatable.
  */
 static void
-start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+start_table_sync(XLogRecPtr *origin_startpos, char **myslotname, int worker_slot)
 {
 	char	   *syncslotname = NULL;
 
@@ -4471,7 +4473,7 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 	PG_TRY();
 	{
 		/* Call initial sync. */
-		syncslotname = LogicalRepSyncTableStart(origin_startpos);
+		syncslotname = LogicalRepSyncTableStart(origin_startpos, worker_slot);
 	}
 	PG_CATCH();
 	{
@@ -4541,12 +4543,13 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 					 char *slotname,
 					 char *originname,
 					 int originname_size,
-					 XLogRecPtr *origin_startpos)
+					 XLogRecPtr *origin_startpos,
+					 int worker_slot)
 {
 	MyLogicalRepWorker->is_sync_completed = false;
 
 	/* Start table synchronization. */
-	start_table_sync(origin_startpos, &slotname);
+	start_table_sync(origin_startpos, &slotname, worker_slot);
 
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
@@ -4850,7 +4853,9 @@ TablesyncWorkerMain(Datum main_arg)
 		ListCell   *lc;
 		bool 	is_table_found = false;
 
-		run_tablesync_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
+		run_tablesync_worker(&options, myslotname, originname,
+							 sizeof(originname), &origin_startpos,
+							 worker_slot);
 
 		if (IsTransactionState())
 			CommitTransactionCommand();
@@ -4891,7 +4896,9 @@ TablesyncWorkerMain(Datum main_arg)
 
 					/* found a table for next iteration */
 					is_table_found = true;
-					clean_sync_worker();
+
+					CommitTransactionCommand();
+					pgstat_report_stat(true);
 
 					StartTransactionCommand();
 					ereport(LOG,
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index d3a136b6f5..429d00f2f0 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1828,6 +1828,12 @@ exec_replication_command(const char *cmd_string)
 				set_ps_display(cmdtag);
 				PreventInTransactionBlock(true, cmdtag);
 
+				/*
+				 * Initialize the flag again because this streaming may be
+				 * second time.
+				 */
+				streamingDoneSending = streamingDoneReceiving = false;
+
 				if (cmd->kind == REPLICATION_KIND_PHYSICAL)
 					StartReplication(cmd);
 				else
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 1e9f8e6e72..d3cabf4033 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -249,7 +249,7 @@ extern int	logicalrep_sync_worker_count(Oid subid);
 
 extern void ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 											   char *originname, Size szoriginname);
-extern char *LogicalRepSyncTableStart(XLogRecPtr *origin_startpos);
+extern char *LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot);
 
 extern bool AllTablesyncsReady(void);
 extern void UpdateTwoPhaseState(Oid suboid, char new_state);
-- 
2.27.0

v2-0004-Add-replication-protocol-cmd-to-create-a-snapshot.patchapplication/octet-stream; name=v2-0004-Add-replication-protocol-cmd-to-create-a-snapshot.patchDownload
From 28f35dd8310a62c257d843bf69d1672717b80364 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 13 Oct 2022 17:05:45 +0300
Subject: [PATCH v2 4/6] Add replication protocol cmd to create a snapshot

Introduced CREATE_REPLICATION_SNAPSHOT to be able to create and use a
snapshot without creating a new replication slot, but by using an
existing slot.

CREATE_REPLICATION_SNAPSHOT simply does what CREATE_REPLICATION_SLOT does
without creating a new replication slot.

CREATE_REPLICATION_SNAPSHOT command imports the snapshot into the current
transaction and returns consistent_point. The changes earlier than the
consistent_point will be applied by importing the snapshot. All changes
later than the consistent_point will be available to be consumed from
the replication slot.

This is useful for reusing replication slots in logical replication.
Otherwise, tablesync workers cannot start from a consistent point to copy
a relation and then apply changes by consuming from replication slot.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/protocol.sgml                    |  31 ++++++
 .../libpqwalreceiver/libpqwalreceiver.c       |  69 +++++++++++-
 src/backend/replication/logical/logical.c     |  40 ++++++-
 .../replication/logical/logicalfuncs.c        |   1 +
 src/backend/replication/repl_gram.y           |  18 ++-
 src/backend/replication/repl_scanner.l        |   2 +
 src/backend/replication/slotfuncs.c           |   1 +
 src/backend/replication/walsender.c           | 104 +++++++++++++++++-
 src/include/nodes/replnodes.h                 |  11 ++
 src/include/replication/logical.h             |   1 +
 src/include/replication/walreceiver.h         |  13 +++
 src/tools/pgindent/typedefs.list              |   2 +
 12 files changed, 289 insertions(+), 4 deletions(-)

diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index b11d9a6ba3..f7d5acee93 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2595,6 +2595,37 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
      </listitem>
     </varlistentry>
 
+    <varlistentry id="protocol-replication-replication-slot-snapshot">
+     <term><literal>CREATE_REPLICATION_SNAPSHOT</literal> <replaceable class="parameter">slot_name</replaceable> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
+      <indexterm><primary>CREATE_REPLICATION_SNAPSHOT</primary></indexterm>
+     </term>
+     <listitem>
+      <para>
+       Creates a snapshot including all the changes from the replication slot until
+       the point at which the replication slot becomes consistent. Then the snapshot
+       is used in the current transaction. This command is currently only supported
+       for logical replication slots.
+      </para>
+
+      <para>
+       In response to this command, the server will return a one-row result set,
+       containing the following field:
+       <variablelist>
+        <varlistentry>
+         <term><literal>consistent_point</literal> (<type>text</type>)</term>
+         <listitem>
+          <para>
+           The WAL location at which the slot became consistent.  This is the
+           earliest location from which streaming can start on this replication
+           slot.
+          </para>
+         </listitem>
+        </varlistentry>
+       </variablelist>
+      </para>
+     </listitem>
+    </varlistentry>
+
     <varlistentry id="protocol-replication-base-backup" xreflabel="BASE_BACKUP">
      <term><literal>BASE_BACKUP</literal> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
       <indexterm><primary>BASE_BACKUP</primary></indexterm>
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index dc9c5c82d9..cf88a3b7d4 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -82,6 +82,8 @@ static WalRcvExecResult *libpqrcv_exec(WalReceiverConn *conn,
 									   const int nRetTypes,
 									   const Oid *retTypes);
 static void libpqrcv_disconnect(WalReceiverConn *conn);
+static void libpqrcv_slot_snapshot(WalReceiverConn *conn, char *slotname,
+								   const WalRcvStreamOptions *options, XLogRecPtr *lsn);
 
 static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_connect = libpqrcv_connect,
@@ -98,7 +100,8 @@ static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_create_slot = libpqrcv_create_slot,
 	.walrcv_get_backend_pid = libpqrcv_get_backend_pid,
 	.walrcv_exec = libpqrcv_exec,
-	.walrcv_disconnect = libpqrcv_disconnect
+	.walrcv_disconnect = libpqrcv_disconnect,
+	.walrcv_slot_snapshot = libpqrcv_slot_snapshot
 };
 
 /* Prototypes for private functions */
@@ -982,6 +985,70 @@ libpqrcv_create_slot(WalReceiverConn *conn, const char *slotname,
 	return snapshot;
 }
 
+/*
+ * TODO
+ */
+static void
+libpqrcv_slot_snapshot(WalReceiverConn *conn,
+					   char *slotname,
+					   const WalRcvStreamOptions *options,
+					   XLogRecPtr *lsn)
+{
+	StringInfoData cmd;
+	PGresult   *res;
+	char	   *pubnames_str;
+	List	   *pubnames;
+	char	   *pubnames_literal;
+
+	initStringInfo(&cmd);
+
+	/* Build the command. */
+	appendStringInfo(&cmd, "CREATE_REPLICATION_SNAPSHOT \"%s\"", slotname);
+	appendStringInfoString(&cmd, " (");
+	appendStringInfo(&cmd, " proto_version '%u'",
+					 options->proto.logical.proto_version);
+
+	/* Add publication names. */
+	pubnames = options->proto.logical.publication_names;
+	pubnames_str = stringlist_to_identifierstr(conn->streamConn, pubnames);
+	if (!pubnames_str)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	pubnames_literal = PQescapeLiteral(conn->streamConn, pubnames_str,
+									   strlen(pubnames_str));
+	if (!pubnames_literal)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("could not start WAL streaming: %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	appendStringInfo(&cmd, ", publication_names %s", pubnames_literal);
+	PQfreemem(pubnames_literal);
+	pfree(pubnames_str);
+
+	appendStringInfoString(&cmd, " )");
+
+	/* Execute the command. */
+	res = libpqrcv_PQexec(conn->streamConn, cmd.data);
+	pfree(cmd.data);
+
+	if (PQresultStatus(res) != PGRES_TUPLES_OK)
+	{
+		PQclear(res);
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg("Could not create a snapshot by replication slot \"%s\": %s",
+						slotname, pchomp(PQerrorMessage(conn->streamConn)))));
+	}
+
+	if (lsn)
+		*lsn = DatumGetLSN(DirectFunctionCall1Coll(pg_lsn_in, InvalidOid,
+												   CStringGetDatum(PQgetvalue(res, 0, 0))));
+
+	PQclear(res);
+}
+
 /*
  * Return PID of remote backend process.
  */
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 41243d0187..3c11f8e1dd 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -476,6 +476,10 @@ CreateInitDecodingContext(const char *plugin,
  * fast_forward
  *		bypass the generation of logical changes.
  *
+ * need_full_snapshot
+ * 		if true, must obtain a snapshot able to read all tables;
+ *  	if false, one that can read only catalogs is acceptable.
+ *
  * xl_routine
  *		XLogReaderRoutine used by underlying xlogreader
  *
@@ -494,6 +498,7 @@ LogicalDecodingContext *
 CreateDecodingContext(XLogRecPtr start_lsn,
 					  List *output_plugin_options,
 					  bool fast_forward,
+					  bool need_full_snapshot,
 					  XLogReaderRoutine *xl_routine,
 					  LogicalOutputPluginWriterPrepareWrite prepare_write,
 					  LogicalOutputPluginWriterWrite do_write,
@@ -502,6 +507,7 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 	LogicalDecodingContext *ctx;
 	ReplicationSlot *slot;
 	MemoryContext old_context;
+	TransactionId xmin_horizon = InvalidTransactionId;
 
 	/* shorter lines... */
 	slot = MyReplicationSlot;
@@ -571,8 +577,40 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 		start_lsn = slot->data.confirmed_flush;
 	}
 
+
+	/*
+	 * We need to determine a safe xmin horizon to start decoding from if we
+	 * want to create a snapshot too. Otherwise we would end up with a
+	 * snapshot that cannot be imported since xmin value from the snapshot may
+	 * be less than the oldest safe xmin. To avoid this call
+	 * GetOldestSafeDecodingTransactionId() to return a safe xmin value, which
+	 * can be used while exporting/importing the snapshot.
+	 *
+	 * So we have to acquire the ProcArrayLock to prevent computation of new
+	 * xmin horizons by other backends, get the safe decoding xid, and inform
+	 * the slot machinery about the new limit. Once that's done the
+	 * ProcArrayLock can be released as the slot machinery now is protecting
+	 * against vacuum.
+	 */
+	if (need_full_snapshot)
+	{
+		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+		xmin_horizon = GetOldestSafeDecodingTransactionId(!need_full_snapshot);
+
+		SpinLockAcquire(&slot->mutex);
+		slot->effective_catalog_xmin = xmin_horizon;
+		slot->data.catalog_xmin = xmin_horizon;
+		slot->effective_xmin = xmin_horizon;
+		SpinLockRelease(&slot->mutex);
+
+		ReplicationSlotsComputeRequiredXmin(true);
+
+		LWLockRelease(ProcArrayLock);
+	}
+
 	ctx = StartupDecodingContext(output_plugin_options,
-								 start_lsn, InvalidTransactionId, false,
+								 start_lsn, xmin_horizon, need_full_snapshot,
 								 fast_forward, xl_routine, prepare_write,
 								 do_write, update_progress);
 
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index 55a24c02c9..85c5cdb633 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -208,6 +208,7 @@ pg_logical_slot_get_changes_guts(FunctionCallInfo fcinfo, bool confirm, bool bin
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									options,
 									false,
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/repl_gram.y b/src/backend/replication/repl_gram.y
index 0c874e33cf..957a5cc653 100644
--- a/src/backend/replication/repl_gram.y
+++ b/src/backend/replication/repl_gram.y
@@ -65,6 +65,7 @@ Node *replication_parse_result;
 %token K_CREATE_REPLICATION_SLOT
 %token K_DROP_REPLICATION_SLOT
 %token K_TIMELINE_HISTORY
+%token K_CREATE_REPLICATION_SNAPSHOT
 %token K_WAIT
 %token K_TIMELINE
 %token K_PHYSICAL
@@ -80,7 +81,7 @@ Node *replication_parse_result;
 %type <node>	command
 %type <node>	base_backup start_replication start_logical_replication
 				create_replication_slot drop_replication_slot identify_system
-				read_replication_slot timeline_history show
+				read_replication_slot timeline_history show create_replication_snapshot
 %type <list>	generic_option_list
 %type <defelt>	generic_option
 %type <uintval>	opt_timeline
@@ -114,6 +115,7 @@ command:
 			| read_replication_slot
 			| timeline_history
 			| show
+			| create_replication_snapshot
 			;
 
 /*
@@ -307,6 +309,19 @@ timeline_history:
 				}
 			;
 
+/*
+ * CREATE_REPLICATION_SNAPSHOT %s options
+ */
+create_replication_snapshot:
+			K_CREATE_REPLICATION_SNAPSHOT var_name plugin_options
+				{
+					CreateReplicationSnapshotCmd *n = makeNode(CreateReplicationSnapshotCmd);
+					n->slotname = $2;
+					n->options = $3;
+					$$ = (Node *) n;
+				}
+			;
+
 opt_physical:
 			K_PHYSICAL
 			| /* EMPTY */
@@ -400,6 +415,7 @@ ident_or_keyword:
 			| K_CREATE_REPLICATION_SLOT	{ $$ = "create_replication_slot"; }
 			| K_DROP_REPLICATION_SLOT		{ $$ = "drop_replication_slot"; }
 			| K_TIMELINE_HISTORY			{ $$ = "timeline_history"; }
+			| K_CREATE_REPLICATION_SNAPSHOT	{ $$ = "create_replication_snapshot"; }
 			| K_WAIT						{ $$ = "wait"; }
 			| K_TIMELINE					{ $$ = "timeline"; }
 			| K_PHYSICAL					{ $$ = "physical"; }
diff --git a/src/backend/replication/repl_scanner.l b/src/backend/replication/repl_scanner.l
index cb467ca46f..5ba2e9c54b 100644
--- a/src/backend/replication/repl_scanner.l
+++ b/src/backend/replication/repl_scanner.l
@@ -126,6 +126,7 @@ START_REPLICATION	{ return K_START_REPLICATION; }
 CREATE_REPLICATION_SLOT		{ return K_CREATE_REPLICATION_SLOT; }
 DROP_REPLICATION_SLOT		{ return K_DROP_REPLICATION_SLOT; }
 TIMELINE_HISTORY	{ return K_TIMELINE_HISTORY; }
+CREATE_REPLICATION_SNAPSHOT	{ return K_CREATE_REPLICATION_SNAPSHOT; }
 PHYSICAL			{ return K_PHYSICAL; }
 RESERVE_WAL			{ return K_RESERVE_WAL; }
 LOGICAL				{ return K_LOGICAL; }
@@ -303,6 +304,7 @@ replication_scanner_is_replication_command(void)
 		case K_DROP_REPLICATION_SLOT:
 		case K_READ_REPLICATION_SLOT:
 		case K_TIMELINE_HISTORY:
+		case K_CREATE_REPLICATION_SNAPSHOT:
 		case K_SHOW:
 			/* Yes; push back the first token so we can parse later. */
 			repl_pushed_back_token = first_token;
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 6035cf4816..c5b2d5b61f 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -486,6 +486,7 @@ pg_logical_replication_slot_advance(XLogRecPtr moveto)
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									NIL,
 									true,	/* fast_forward */
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 429d00f2f0..974ab5ca35 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -238,6 +238,7 @@ static void CreateReplicationSlot(CreateReplicationSlotCmd *cmd);
 static void DropReplicationSlot(DropReplicationSlotCmd *cmd);
 static void StartReplication(StartReplicationCmd *cmd);
 static void StartLogicalReplication(StartReplicationCmd *cmd);
+static void CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd);
 static void ProcessStandbyMessage(void);
 static void ProcessStandbyReplyMessage(void);
 static void ProcessStandbyHSFeedbackMessage(void);
@@ -1284,7 +1285,7 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	 * are reported early.
 	 */
 	logical_decoding_ctx =
-		CreateDecodingContext(cmd->startpoint, cmd->options, false,
+		CreateDecodingContext(cmd->startpoint, cmd->options, false, false,
 							  XL_ROUTINE(.page_read = logical_read_xlog_page,
 										 .segment_open = WalSndSegmentOpen,
 										 .segment_close = wal_segment_close),
@@ -1336,6 +1337,98 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	EndCommand(&qc, DestRemote, false);
 }
 
+/*
+ * Create a snapshot from an existing replication slot.
+ */
+static void
+CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd)
+{
+	Snapshot	snap;
+	LogicalDecodingContext *ctx;
+	char		xloc[MAXFNAMELEN];
+	DestReceiver *dest;
+	TupOutputState *tstate;
+	TupleDesc	tupdesc;
+	Datum		values[1];
+	bool		nulls[1] = {0};
+
+	Assert(!MyReplicationSlot);
+
+	CheckLogicalDecodingRequirements();
+
+	if (!IsTransactionBlock())
+		ereport(ERROR,
+				(errmsg("%s must be called inside a transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (XactIsoLevel != XACT_REPEATABLE_READ)
+		ereport(ERROR,
+				(errmsg("%s must be called in REPEATABLE READ isolation mode transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (!XactReadOnly)
+		ereport(ERROR,
+				(errmsg("%s must be called in a read only transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (FirstSnapshotSet)
+		ereport(ERROR,
+				(errmsg("%s must be called before any query",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (IsSubTransaction())
+		ereport(ERROR,
+				(errmsg("%s must not be called in a subtransaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	ReplicationSlotAcquire(cmd->slotname, false);
+
+	ctx = CreateDecodingContext(MyReplicationSlot->data.restart_lsn,
+								cmd->options,
+								false,
+								true,
+								XL_ROUTINE(.page_read = logical_read_xlog_page,
+										   .segment_open = WalSndSegmentOpen,
+										   .segment_close = wal_segment_close),
+								WalSndPrepareWrite, WalSndWriteData,
+								WalSndUpdateProgress);
+
+	/*
+	 * Signal that we don't need the timeout mechanism. We're just creating
+	 * the snapshot with the replication slot and don't yet accept feedback
+	 * messages or send keepalives. As we possibly need to wait for further
+	 * WAL the walsender would otherwise possibly be killed too soon.
+	 */
+	last_reply_timestamp = 0;
+
+	/* build initial snapshot, might take a while */
+	DecodingContextFindStartpoint(ctx);
+
+	snap = SnapBuildInitialSnapshot(ctx->snapshot_builder);
+	RestoreTransactionSnapshot(snap, MyProc);
+
+	/* Don't need the decoding context anymore */
+	FreeDecodingContext(ctx);
+
+	/* Create a tuple to send consistent WAL location */
+	snprintf(xloc, sizeof(xloc), "%X/%X",
+			 LSN_FORMAT_ARGS(MyReplicationSlot->data.confirmed_flush));
+
+	dest = CreateDestReceiver(DestRemoteSimple);
+	tupdesc = CreateTemplateTupleDesc(1);
+	TupleDescInitBuiltinEntry(tupdesc, (AttrNumber) 1, "consistent_point",
+							  TEXTOID, -1, 0);
+	tstate = begin_tup_output_tupdesc(dest, tupdesc, &TTSOpsVirtual);
+
+	/* consistent wal location */
+	values[0] = CStringGetTextDatum(xloc);
+
+	do_tup_output(tstate, values, nulls);
+	end_tup_output(tstate);
+
+	ReplicationSlotRelease();
+}
+
 /*
  * LogicalDecodingContext 'prepare_write' callback.
  *
@@ -1870,6 +1963,15 @@ exec_replication_command(const char *cmd_string)
 			}
 			break;
 
+		case T_CreateReplicationSnapshotCmd:
+			{
+				cmdtag = "CREATE_REPLICATION_SNAPSHOT";
+				set_ps_display(cmdtag);
+				CreateReplicationSnapshot((CreateReplicationSnapshotCmd *) cmd_node);
+				EndReplicationCommand(cmdtag);
+				break;
+			}
+
 		default:
 			elog(ERROR, "unrecognized replication command node tag: %u",
 				 cmd_node->type);
diff --git a/src/include/nodes/replnodes.h b/src/include/nodes/replnodes.h
index 4321ba8f86..154ab74f33 100644
--- a/src/include/nodes/replnodes.h
+++ b/src/include/nodes/replnodes.h
@@ -108,4 +108,15 @@ typedef struct TimeLineHistoryCmd
 	TimeLineID	timeline;
 } TimeLineHistoryCmd;
 
+/* ----------------------
+ *		CREATE_REPLICATION_SNAPSHOT command
+ * ----------------------
+ */
+typedef struct CreateReplicationSnapshotCmd
+{
+	NodeTag		type;
+	char	   *slotname;
+	List	   *options;
+} CreateReplicationSnapshotCmd;
+
 #endif							/* REPLNODES_H */
diff --git a/src/include/replication/logical.h b/src/include/replication/logical.h
index 5f49554ea0..6535786a0e 100644
--- a/src/include/replication/logical.h
+++ b/src/include/replication/logical.h
@@ -125,6 +125,7 @@ extern LogicalDecodingContext *CreateInitDecodingContext(const char *plugin,
 extern LogicalDecodingContext *CreateDecodingContext(XLogRecPtr start_lsn,
 													 List *output_plugin_options,
 													 bool fast_forward,
+													 bool need_full_snapshot,
 													 XLogReaderRoutine *xl_routine,
 													 LogicalOutputPluginWriterPrepareWrite prepare_write,
 													 LogicalOutputPluginWriterWrite do_write,
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 281626fa6f..f01e5f23a7 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -386,6 +386,16 @@ typedef WalRcvExecResult *(*walrcv_exec_fn) (WalReceiverConn *conn,
  */
 typedef void (*walrcv_disconnect_fn) (WalReceiverConn *conn);
 
+/*
+ * walrcv_slot_snapshot_fn
+ *
+ * Create a snapshot by an existing replication slot
+ */
+typedef void (*walrcv_slot_snapshot_fn) (WalReceiverConn *conn,
+										 char *slotname,
+										 const WalRcvStreamOptions *options,
+										 XLogRecPtr *lsn);
+
 typedef struct WalReceiverFunctionsType
 {
 	walrcv_connect_fn walrcv_connect;
@@ -403,6 +413,7 @@ typedef struct WalReceiverFunctionsType
 	walrcv_get_backend_pid_fn walrcv_get_backend_pid;
 	walrcv_exec_fn walrcv_exec;
 	walrcv_disconnect_fn walrcv_disconnect;
+	walrcv_slot_snapshot_fn walrcv_slot_snapshot;
 } WalReceiverFunctionsType;
 
 extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
@@ -437,6 +448,8 @@ extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
 	WalReceiverFunctions->walrcv_exec(conn, exec, nRetTypes, retTypes)
 #define walrcv_disconnect(conn) \
 	WalReceiverFunctions->walrcv_disconnect(conn)
+#define walrcv_slot_snapshot(conn, slotname, options, lsn) \
+	WalReceiverFunctions->walrcv_slot_snapshot(conn, slotname, options, lsn)
 
 static inline void
 walrcv_clear_result(WalRcvExecResult *walres)
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index e941fb6c82..cc023b10fa 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2348,6 +2348,7 @@ ReplicationSlotInvalidationCause
 ReplicationSlotOnDisk
 ReplicationSlotPersistency
 ReplicationSlotPersistentData
+CreateReplicationSnapshotCmd
 ReplicationState
 ReplicationStateCtl
 ReplicationStateOnDisk
@@ -3855,6 +3856,7 @@ walrcv_receive_fn
 walrcv_send_fn
 walrcv_server_version_fn
 walrcv_startstreaming_fn
+walrcv_slot_snapshot_fn
 wchar2mb_with_len_converter
 wchar_t
 win32_deadchild_waitinfo
-- 
2.27.0

v2-0005-Reuse-Replication-Slot-and-Origin-in-Tablesync.patchapplication/octet-stream; name=v2-0005-Reuse-Replication-Slot-and-Origin-in-Tablesync.patchDownload
From 71307d7181a81adc871e892551c0eb8bf9ea5d3f Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH v2 5/6] Reuse Replication Slot and Origin in Tablesync

This commit allows reusing replication slots and origins during tablesync.

Earlier, a tablesync worker was creating a new replication slot and origin
each time it syncs a new table. With this patch, replication
slots/origins can be reusable for tablesync.

This reduces the overhead of creating/dropping replication slots and origins
and improves tablesync speed significantly especially for empty or small tables.

If the state of the current table is INIT or DATASYNC, tablesync worker needs a
repliation slot/origin. If the worker has not created slot and origin in
its previous runs, it will create those first. Otherwise the worker reuses
slot and origin created by the same worker in previous iterations earlier.
Tables in FINISHEDCOPY are expected to have a replication slot and origin.
Slot and origin names for such tables are persisted in
pg_subscription_rel catalog. Tablesync worker can fetch them and proceed
with existing slot and origin of FINISHEDCOPY tables and does not need to
create new ones.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/catalogs.sgml                    |  31 ++
 src/backend/catalog/pg_subscription.c         | 247 +++++++++++-
 src/backend/commands/subscriptioncmds.c       | 229 +++++++----
 .../replication/logical/applyparallelworker.c |   3 +-
 src/backend/replication/logical/launcher.c    |   8 +-
 src/backend/replication/logical/tablesync.c   | 369 +++++++++++++-----
 src/backend/replication/logical/worker.c      |  62 ++-
 src/include/catalog/pg_subscription.h         |   6 +
 src/include/catalog/pg_subscription_rel.h     |  14 +-
 src/include/replication/slot.h                |   3 +-
 src/include/replication/worker_internal.h     |  23 +-
 11 files changed, 797 insertions(+), 198 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 852cb30ae1..60718ab587 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8030,6 +8030,19 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        origin.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>sublastusedid</structfield> <type>int8</type>
+      </para>
+      <para>
+      The last used ID for tablesync workers. It acts as an unique identifier
+      for replication slots which are created by tablesync workers.
+      The last used ID needs to be persisted to make logical replication safely
+      proceed after any interruption. If sublastusedid is 0, then no table has
+      been synced yet.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
@@ -8114,6 +8127,24 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        otherwise null
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srrelslotname</structfield> <type>name</type>
+      </para>
+      <para>
+       Replication slot name that is used for synchronization of relation
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srreloriginname</structfield> <type>name</type>
+      </para>
+      <para>
+       Origin name that is used for tracking synchronization of relation
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index d07f88ce28..152fdaa310 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -108,6 +108,14 @@ GetSubscription(Oid subid, bool missing_ok)
 								   Anum_pg_subscription_suborigin);
 	sub->origin = TextDatumGetCString(datum);
 
+	/* Get last used id */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
+							tup,
+							Anum_pg_subscription_sublastusedid,
+							&isnull);
+	Assert(!isnull);
+	sub->lastusedid = DatumGetInt64(datum);
+
 	ReleaseSysCache(tup);
 
 	return sub;
@@ -199,6 +207,44 @@ DisableSubscription(Oid subid)
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update the last used replication slot ID for the given subscription.
+ */
+void
+UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid)
+{
+	Relation	rel;
+	bool		nulls[Natts_pg_subscription];
+	bool		replaces[Natts_pg_subscription];
+	Datum		values[Natts_pg_subscription];
+	HeapTuple	tup;
+
+	/* Look up the subscription in the catalog */
+	rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+	tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	/* Form a new tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	replaces[Anum_pg_subscription_sublastusedid - 1] = true;
+	values[Anum_pg_subscription_sublastusedid- 1] = Int64GetDatum(lastusedid);
+
+	/* Update the catalog */
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+	heap_freetuple(tup);
+
+	table_close(rel, NoLock);
+}
+
 /*
  * Convert text array to list of strings.
  *
@@ -228,7 +274,7 @@ textarray_to_stringlist(ArrayType *textarray)
  */
 void
 AddSubscriptionRelState(Oid subid, Oid relid, char state,
-						XLogRecPtr sublsn)
+						XLogRecPtr sublsn, char *relslotname, char *reloriginname)
 {
 	Relation	rel;
 	HeapTuple	tup;
@@ -257,6 +303,16 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
 	else
 		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -269,6 +325,60 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Internal function to modify columns for relation state update
+ */
+static void
+UpdateSubscriptionRelState_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char state,
+									XLogRecPtr sublsn)
+{
+	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
+	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+
+	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (sublsn != InvalidXLogRecPtr)
+		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
+	else
+		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication slot update
+ */
+static void
+UpdateSubscriptionRelReplicationSlot_internal(Datum *values,
+											bool *nulls,
+											bool *replaces,
+											char *relslotname)
+{
+	replaces[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication origin update
+ */
+static void
+UpdateSubscriptionRelOrigin_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char *reloriginname)
+{
+	replaces[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+}
+
 /*
  * Update the state of a subscription table.
  */
@@ -299,14 +409,56 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	memset(nulls, false, sizeof(nulls));
 	memset(replaces, false, sizeof(replaces));
 
-	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
-	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
 
-	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
-	if (sublsn != InvalidXLogRecPtr)
-		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
-	else
-		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update replication slot name, origin name and state of
+ * a subscription table in one transaction.
+ */
+void
+UpdateSubscriptionRel(Oid subid,
+					  Oid relid,
+					  char state,
+					  XLogRecPtr sublsn,
+					  char *relslotname,
+					  char *reloriginname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
+	UpdateSubscriptionRelOrigin_internal(values, nulls, replaces, reloriginname);
 
 	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
 							replaces);
@@ -318,6 +470,85 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Get origin name of subscription table.
+ *
+ * reloriginname's value has the replication origin name if the origin exists.
+ */
+void
+GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*originname;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srreloriginname, isnull);
+	if (!*isnull)
+	{
+		originname = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(reloriginname, originname, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
+/*
+ * Get replication slot name of subscription table.
+ *
+ * slotname's value has the replication slot name if the subscription has any.
+ */
+void
+GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*relrepslot;
+	bool		isnull;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srrelslotname, &isnull);
+	if (!isnull)
+	{
+		relrepslot = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(slotname, relrepslot, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
 /*
  * Get state of subscription table.
  *
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 54895ba929..9f5a295ddc 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -710,6 +710,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 		publicationListToArray(publications);
 	values[Anum_pg_subscription_suborigin - 1] =
 		CStringGetTextDatum(opts.origin);
+	values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(0);
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -773,7 +774,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 										 rv->schemaname, rv->relname);
 
 				AddSubscriptionRelState(subid, relid, table_state,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 			}
 
 			/*
@@ -864,6 +865,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
 	bool		must_use_password;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -943,7 +946,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				AddSubscriptionRelState(sub->oid, relid,
 										copy_data ? SUBREL_STATE_INIT : SUBREL_STATE_READY,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 				ereport(DEBUG1,
 						(errmsg_internal("table \"%s.%s\" added to subscription \"%s\"",
 										 rv->schemaname, rv->relname, sub->name)));
@@ -967,6 +970,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				char		state;
 				XLogRecPtr	statelsn;
+				char		slotname[NAMEDATALEN] = {0};
 
 				/*
 				 * Lock pg_subscription_rel with AccessExclusiveLock to
@@ -993,13 +997,36 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/*
+				 * Find the logical replication sync worker. If exists, store
+				 * the slot number for dropping associated replication slots
+				 * later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot_name);
+				}
+				else
+				{
+					/*
+					 * Sync of this relation might be failed in an earlier
+					 * attempt, but the replication slot might still exist.
+					 */
+					GetSubscriptionRelReplicationSlot(sub->oid, relid, slotname);
+					if (strlen(slotname) > 0)
+						sub_remove_slots = lappend(sub_remove_slots, slotname);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
 				 * tablesync origin.
 				 */
-				if (state != SUBREL_STATE_READY)
+				if (state != SUBREL_STATE_READY &&
+					state != SUBREL_STATE_SYNCDONE)
 				{
 					char		originname[NAMEDATALEN];
 
@@ -1027,31 +1054,24 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
-		 * the database operations we won't be able to rollback dropped slots.
+		 * Drop the replication slots associated with tablesync workers for
+		 * removed tables. This has to be at the end because otherwise if
+		 * there is an error while doing the database operations we won't be
+		 * able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			memcpy(syncslotname, lfirst(lc), sizeof(NAMEDATALEN));
+
+			/*
+			 * There is no certainty, maybe the slot does not exist yet. Also,
+			 * if we fail after removing some of the slots, next time, it will
+			 * again try to drop already dropped slots and fail. For these
+			 * reasons, we allow missing_ok = true for the drop.
+			 */
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1474,6 +1494,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	char	   *subname;
 	char	   *conninfo;
 	char	   *slotname;
+	int64		lastusedid;
 	List	   *subworkers;
 	ListCell   *lc;
 	char		originname[NAMEDATALEN];
@@ -1546,6 +1567,14 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	else
 		slotname = NULL;
 
+	/* Get the last used identifier by the subscription */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID, tup,
+							Anum_pg_subscription_sublastusedid, &isnull);
+	if (!isnull)
+		lastusedid = DatumGetInt64(datum);
+	else
+		lastusedid = 0;
+
 	/*
 	 * Since dropping a replication slot is not transactional, the replication
 	 * slot stays dropped even if the transaction rolls back.  So we cannot
@@ -1595,6 +1624,8 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	}
 	list_free(subworkers);
 
+	rstates = GetSubscriptionRelations(subid, true);
+
 	/*
 	 * Remove the no-longer-useful entry in the launcher's table of apply
 	 * worker start times.
@@ -1606,36 +1637,26 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	ApplyLauncherForgetWorkerStartTime(subid);
 
 	/*
-	 * Cleanup of tablesync replication origins.
-	 *
-	 * Any READY-state relations would already have dealt with clean-ups.
+	 * Cleanup of tablesync replication origins associated with the
+	 * subscription, if exists. Try to drop origins by creating all origin
+	 * names created for this subscription.
 	 *
 	 * Note that the state can't change because we have already stopped both
 	 * the apply and tablesync workers and they can't restart because of
 	 * exclusive lock on the subscription.
+	 *
+	 * XXX: This can be handled better instead of looping through all possible
 	 */
-	rstates = GetSubscriptionRelations(subid, true);
-	foreach(lc, rstates)
+	for (int64 i = 1; i <= lastusedid; i++)
 	{
-		SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-		Oid			relid = rstate->relid;
-
-		/* Only cleanup resources of tablesync workers */
-		if (!OidIsValid(relid))
-			continue;
+		char		originname_to_drop[NAMEDATALEN] = {0};
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * It is possible that the origin is not yet created for tablesync
-		 * worker so passing missing_ok = true. This can happen for the states
-		 * before SUBREL_STATE_FINISHEDCOPY.
-		 */
-		ReplicationOriginNameForLogicalRep(subid, relid, originname,
-										   sizeof(originname));
-		replorigin_drop_by_name(originname, true, false);
+		snprintf(originname_to_drop, sizeof(originname_to_drop), "pg_%u_%lld", subid, (long long) i);
+		/* missing_ok = true, since the origin might be already dropped. */
+		replorigin_drop_by_name(originname_to_drop, true, false);
 	}
 
+
 	/* Clean up dependencies */
 	deleteSharedDependencyRecordsFor(SubscriptionRelationId, subid, 0);
 
@@ -1688,39 +1709,17 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
+		List	   *slots = NULL;
 
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
 
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char	   *syncslotname = (char *) lfirst(lc);
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1743,6 +1742,71 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * Get the replication slot names associated with the subscription.
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok)
+{
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+					 " FROM pg_replication_slots"
+					 " WHERE slot_name LIKE 'pg_%i_sync_%%';",
+					 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+					errmsg("could not receive list of slots associated with the subscription %u, error: %s",
+					subid, res->err));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();
+		return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -2155,6 +2219,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -2165,18 +2230,20 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
-			char		syncslotname[NAMEDATALEN] = {0};
-
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
-				 syncslotname);
+				 worker->slot_name);
 		}
 	}
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 1d4e83c4c1..d2c70dffbc 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -440,7 +440,8 @@ pa_launch_parallel_worker(void)
 										MySubscription->name,
 										MyLogicalRepWorker->userid,
 										InvalidOid,
-										dsm_segment_handle(winfo->dsm_seg));
+										dsm_segment_handle(winfo->dsm_seg),
+										InvalidRepSlotId);
 
 	if (launched)
 	{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 4c3f72cb46..6d49e46dbb 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -304,7 +304,7 @@ logicalrep_workers_find(Oid subid, bool only_running)
  */
 bool
 logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
-						 Oid relid, dsm_handle subworker_dsm)
+						 Oid relid, dsm_handle subworker_dsm, int64 slotid)
 {
 	BackgroundWorker bgw;
 	BackgroundWorkerHandle *bgw_handle;
@@ -430,6 +430,9 @@ retry:
 	worker->launch_time = now;
 	worker->in_use = true;
 	worker->generation++;
+	worker->created_slot = false;
+	worker->rep_slot_id = slotid;
+	worker->slot_name = (char *) palloc(NAMEDATALEN);
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -1180,7 +1183,8 @@ ApplyLauncherMain(Datum main_arg)
 				ApplyLauncherSetWorkerStartTime(sub->oid, now);
 				logicalrep_worker_launch(sub->dbid, sub->oid, sub->name,
 										 sub->owner, InvalidOid,
-										 DSM_HANDLE_INVALID);
+										 DSM_HANDLE_INVALID,
+										 InvalidRepSlotId);
 			}
 			else
 			{
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 8d44ed207f..e2b535f920 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -328,40 +328,29 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 								   MyLogicalRepWorker->relid,
 								   MyLogicalRepWorker->relstate,
 								   MyLogicalRepWorker->relstate_lsn);
+		CommitTransactionCommand();
 
 		/*
-		 * End streaming so that LogRepWorkerWalRcvConn can be used to drop
-		 * the slot.
-		 */
-		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
-
-		/*
-		 * Cleanup the tablesync slot.
+		 * Cleanup the tablesync slot. If the slot name used by this worker is
+		 * different from the default slot name for the worker, this means the
+		 * current table had started to being synchronized by another worker
+		 * and replication slot. And this worker is reusing a replication slot
+		 * from a previous attempt. We do not need that replication slot
+		 * anymore.
 		 *
 		 * This has to be done after updating the state because otherwise if
 		 * there is an error while doing the database operations we won't be
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
+										MyLogicalRepWorker->rep_slot_id,
 										syncslotname,
 										sizeof(syncslotname));
 
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
-		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
-
-		CommitTransactionCommand();
-		pgstat_report_stat(false);
-
-		/*
-		 * Start a new transaction to clean up the tablesync origin tracking.
-		 * This transaction will be ended within the finish_sync_worker().
-		 * Now, even, if we fail to remove this here, the apply worker will
-		 * ensure to clean it up afterward.
+		 * We are safe to drop the replication tracking origin after this
+		 * point. Now, even, if we fail to remove this here, the apply worker
+		 * will ensure to clean it up afterward.
 		 *
 		 * We need to do this after the table state is set to SYNCDONE.
 		 * Otherwise, if an error occurs while performing the database
@@ -370,32 +359,70 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 * have been cleared before restart. So, the restarted worker will use
 		 * invalid replication progress state resulting in replay of
 		 * transactions that have already been applied.
+		 *
+		 * Firstly reset the origin session to remove the ownership of the
+		 * slot. This is needed to allow the origin to be dropped or reused
+		 * later.
 		 */
+		replorigin_session_reset();
+		replorigin_session_origin = InvalidRepOriginId;
+		replorigin_session_origin_lsn = InvalidXLogRecPtr;
+		replorigin_session_origin_timestamp = 0;
+
 		StartTransactionCommand();
+		if (MyLogicalRepWorker->slot_name && strcmp(syncslotname, MyLogicalRepWorker->slot_name) != 0)
+		{
+			/*
+			 * End streaming so that LogRepWorkerWalRcvConn can be used to
+			 * drop the slot.
+			 */
+			walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
 
-		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
+			ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+											   MyLogicalRepWorker->relid,
+											   originname,
+											   sizeof(originname));
+
+			/*
+			 * Drop replication origin
+			 *
+			 * There is a chance that the user is concurrently performing refresh
+			 * for the subscription where we remove the table state and its origin
+			 * or the apply worker would have removed this origin. So passing
+			 * missing_ok = true.
+			 */
+			replorigin_drop_by_name(originname, true, false);
+		}
 
 		/*
-		 * Resetting the origin session removes the ownership of the slot.
-		 * This is needed to allow the origin to be dropped.
+		 * We are safe to remove persisted replication slot and origin data,
+		 * since it's already in SYNCDONE state. They will not be needed
+		 * anymore.
 		 */
-		replorigin_session_reset();
-		replorigin_session_origin = InvalidRepOriginId;
-		replorigin_session_origin_lsn = InvalidXLogRecPtr;
-		replorigin_session_origin_timestamp = 0;
+		UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+							  MyLogicalRepWorker->relid,
+							  MyLogicalRepWorker->relstate,
+							  MyLogicalRepWorker->relstate_lsn,
+							  NULL,
+							  NULL);
+		ereport(DEBUG2,
+			(errmsg("process_syncing_tables_for_sync: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					"NULL", "NULL", MyLogicalRepWorker->relstate,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+		CommitTransactionCommand();
+		pgstat_report_stat(false);
 
 		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * There is a chance that the user is concurrently performing refresh
-		 * for the subscription where we remove the table state and its origin
-		 * or the apply worker would have removed this origin. So passing
-		 * missing_ok = true.
+		 * This should return the default origin name for the worker. Even if
+		 * the worker used a different origin for this table, it should be
+		 * dropped and removed from the catalog so far.
 		 */
-		replorigin_drop_by_name(originname, true, false);
+		StartTransactionCommand();
+		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+										   MyLogicalRepWorker->relid,
+										   originname,
+										   sizeof(originname));
 
 		/* Sync worker has completed synchronization of the current table. */
 		MyLogicalRepWorker->is_sync_completed = true;
@@ -492,6 +519,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 			if (current_lsn >= rstate->lsn)
 			{
 				char		originname[NAMEDATALEN];
+				bool		is_origin_null = true;
 
 				rstate->state = SUBREL_STATE_READY;
 				rstate->lsn = current_lsn;
@@ -512,18 +540,31 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				 * error while dropping we won't restart it to drop the
 				 * origin. So passing missing_ok = true.
 				 */
-				ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-												   rstate->relid,
-												   originname,
-												   sizeof(originname));
-				replorigin_drop_by_name(originname, true, false);
+				GetSubscriptionRelOrigin(MyLogicalRepWorker->subid,
+										 rstate->relid, originname,
+										 &is_origin_null);
+
+				if (!is_origin_null)
+				{
+					replorigin_drop_by_name(originname, true, false);
+				}
 
 				/*
 				 * Update the state to READY only after the origin cleanup.
 				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-										   rstate->relid, rstate->state,
-										   rstate->lsn);
+				UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+									  rstate->relid,
+									  rstate->state,
+									  rstate->lsn,
+									  NULL,
+									  NULL);
+				ereport(DEBUG2,
+					(errmsg("process_syncing_tables_for_apply: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+							"NULL", "NULL", rstate->state,
+							rstate->relid, MyLogicalRepWorker->subid)));
+
+				CommitTransactionCommand();
+				started_tx = false;
 			}
 		}
 		else
@@ -612,12 +653,25 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						TimestampDifferenceExceeds(hentry->last_start_time, now,
 												   wal_retrieve_retry_interval))
 					{
+						if (IsTransactionState())
+							CommitTransactionCommand();
+						StartTransactionCommand();
+						started_tx = true;
+
+						MySubscription->lastusedid++;
+						UpdateSubscriptionLastSlotId(MyLogicalRepWorker->subid,
+													 MySubscription->lastusedid);
+						ereport(DEBUG2,
+								(errmsg("process_syncing_tables_for_apply: incremented lastusedid to %lld for subscription %u",
+										(long long) MySubscription->lastusedid, MySubscription->oid)));
+
 						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
 												 MySubscription->oid,
 												 MySubscription->name,
 												 MyLogicalRepWorker->userid,
 												 rstate->relid,
-												 DSM_HANDLE_INVALID);
+												 DSM_HANDLE_INVALID,
+												 MySubscription->lastusedid);
 						hentry->last_start_time = now;
 					}
 				}
@@ -1240,8 +1294,8 @@ copy_table(Relation rel)
  * The name must not exceed NAMEDATALEN - 1 because of remote node constraints
  * on slot name length. We append system_identifier to avoid slot_name
  * collision with subscriptions in other clusters. With the current scheme
- * pg_%u_sync_%u_UINT64_FORMAT (3 + 10 + 6 + 10 + 20 + '\0'), the maximum
- * length of slot_name will be 50.
+ * pg_%u_sync_%lu_UINT64_FORMAT (3 + 10 + 6 + 20 + 20 + '\0'), the maximum
+ * length of slot_name will be 45.
  *
  * The returned slot name is stored in the supplied buffer (syncslotname) with
  * the given size.
@@ -1252,11 +1306,11 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
+ReplicationSlotNameForTablesync(Oid suboid, int64 slotid,
 								char *syncslotname, Size szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%lld_" UINT64_FORMAT, suboid,
+			(long long) slotid, GetSystemIdentifier());
 }
 
 /*
@@ -1282,6 +1336,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 	UserContext ucxt;
 	bool		must_use_password;
 	bool		run_as_owner;
+	char	   *prev_slotname;
 
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
@@ -1316,7 +1371,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
+									MyLogicalRepWorker->rep_slot_id,
 									slotname,
 									NAMEDATALEN);
 
@@ -1355,12 +1410,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC ||
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY);
 
+	/*
+	 * See if tablesync of the current relation has been started with another
+	 * replication slot.
+	 *
+	 * Read previous slot name from the catalog, if exists.
+	 */
+	prev_slotname = (char *) palloc(NAMEDATALEN);
+	StartTransactionCommand();
+	GetSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+									  MyLogicalRepWorker->relid,
+									  prev_slotname);
+
 	/* Assign the origin tracking record name. */
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   sizeof(originname));
 
+	CommitTransactionCommand();
+
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC)
 	{
 		/*
@@ -1374,10 +1443,53 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 		 * breakdown then it wouldn't have succeeded so trying it next time
 		 * seems like a better bet.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, slotname, true);
+		if (strlen(prev_slotname) > 0)
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);
+
+			StartTransactionCommand();
+			/* Replication origin might still exist. Try to drop */
+			replorigin_drop_by_name(originname, true, false);
+
+			/*
+			 * Remove replication slot and origin name from the relation's
+			 * catalog record
+			 */
+			UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+								  MyLogicalRepWorker->relid,
+								  MyLogicalRepWorker->relstate,
+								  MyLogicalRepWorker->relstate_lsn,
+								  NULL,
+								  NULL);
+			CommitTransactionCommand();
+			ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+						"NULL", "NULL", MyLogicalRepWorker->relstate,
+						MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+		}
 	}
 	else if (MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY)
 	{
+		/*
+		 * At this point, the table that is currently being synchronized
+		 * should have its replication slot name filled in the catalog. The
+		 * tablesync process was started with another sync worker and
+		 * replication slot. We need to continue using the same replication
+		 * slot in this worker too.
+		 */
+		if (strlen(prev_slotname) == 0)
+		{
+			elog(ERROR, "Replication slot could not be found for subscription %u, relation %u",
+				 MyLogicalRepWorker->subid,
+				 MyLogicalRepWorker->relid);
+		}
+
+		/*
+		 * Proceed with the correct replication slot. Use previously created
+		 * replication slot to sync this table.
+		 */
+		memcpy(slotname, prev_slotname, NAMEDATALEN);
+
 		/*
 		 * The COPY phase was previously done, but tablesync then crashed
 		 * before it was able to finish normally.
@@ -1397,7 +1509,9 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 
 		goto copy_table_done;
 	}
+	pfree(prev_slotname);
 
+	/* Preparing for table copy operation */
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 	MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
 	MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
@@ -1405,11 +1519,31 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   MyLogicalRepWorker->relstate,
-							   MyLogicalRepWorker->relstate_lsn);
+
+	/*
+	 * Refresh the originname in case of having non-existing origin
+	 * from previous failed sync attempts.
+	 * If that's the case, it should be removed from the catalog so far.
+	 * Then, we can continue by reusing the origin created by the current
+	 * worker instead of .
+	 */
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									MyLogicalRepWorker->relid,
+									originname,
+									sizeof(originname));
+
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  MyLogicalRepWorker->relstate,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					slotname, originname, MyLogicalRepWorker->relstate,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+
 	pgstat_report_stat(true);
 
 	StartTransactionCommand();
@@ -1437,48 +1571,95 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 						res->err)));
 	walrcv_clear_result(res);
 
+	originid = replorigin_by_name(originname, true);
+
 	/*
 	 * Create a new permanent logical decoding slot. This slot will be used
 	 * for the catchup phase after COPY is done, so tell it to use the
 	 * snapshot to make the final data consistent.
+	 *
+	 * Replication slot will only be created if either this is the first run
+	 * of the worker or we're not using a previous replication slot.
 	 */
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-
-	/*
-	 * Setup replication origin tracking. The purpose of doing this before the
-	 * copy is to avoid doing the copy again due to any error in setting up
-	 * origin tracking.
-	 */
-	originid = replorigin_by_name(originname, true);
-	if (!OidIsValid(originid))
+	if (!MyLogicalRepWorker->created_slot)
 	{
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						   slotname, false /* permanent */ , false /* two_phase */ ,
+						   CRS_USE_SNAPSHOT, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: created replication slot %s for subscription %u",
+						slotname, MyLogicalRepWorker->subid)));
+
 		/*
-		 * Origin tracking does not exist, so create it now.
-		 *
-		 * Then advance to the LSN got from walrcv_create_slot. This is WAL
-		 * logged for the purpose of recovery. Locks are to prevent the
-		 * replication origin from vanishing while advancing.
+		 * Remember that we created the slot so that we will not try to create
+		 * it again.
 		 */
-		originid = replorigin_create(originname);
-
-		LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
-		replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
-						   true /* go backward */ , true /* WAL log */ );
-		UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->created_slot = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
+		/*
+		 * Setup replication origin tracking. The purpose of doing this before
+		 * the copy is to avoid doing the copy again due to any error in
+		 * setting up origin tracking.
+		 */
+		if (!OidIsValid(originid))
+		{
+			/*
+			 * Origin tracking does not exist, so create it now.
+			 */
+			originid = replorigin_create(originname);
+		}
+		else
+		{
+			/*
+			 * At this point, there shouldn't be any existing replication
+			 * origin with the same name.
+			 */
+			ereport(ERROR,
+					(errcode(ERRCODE_DUPLICATE_OBJECT),
+					 errmsg("replication origin \"%s\" already exists",
+							originname)));
+		}
 	}
 	else
 	{
-		ereport(ERROR,
-				(errcode(ERRCODE_DUPLICATE_OBJECT),
-				 errmsg("replication origin \"%s\" already exists",
-						originname)));
+		/*
+		 * Do not create a new replication slot, reuse the existing one
+		 * instead. Use a new snapshot for the replication slot to ensure that
+		 * tablesync and apply proceses are consistent with each other.
+		 */
+		WalRcvStreamOptions options;
+		int			server_version;
+
+		server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+		options.proto.logical.proto_version =
+			server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+			server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+			LOGICALREP_PROTO_VERSION_NUM;
+		options.proto.logical.publication_names = MySubscription->publications;
+
+		walrcv_slot_snapshot(LogRepWorkerWalRcvConn, slotname, &options, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: reusing replication slot %s for relation %u in subscription %u",
+						slotname, MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
 	}
 
+	/*
+	 * Advance to the LSN got from walrcv_create_slot or walrcv_slot_snapshot.
+	 * This is WAL logged for the purpose of recovery. Locks are to prevent
+	 * the replication origin from vanishing while advancing.
+	 *
+	 * Then setup replication origin tracking.
+	 */
+	LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+	replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
+					   true /* go backward */ , true /* WAL log */ );
+	UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+
 	/*
 	 * Make sure that the copy command runs as the table owner, unless the
 	 * user has opted out of that behaviour.
@@ -1537,12 +1718,18 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 	 * Update the persisted state to indicate the COPY phase is done; make it
 	 * visible to others.
 	 */
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   SUBREL_STATE_FINISHEDCOPY,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  SUBREL_STATE_FINISHEDCOPY,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					originname, slotname, SUBREL_STATE_FINISHEDCOPY,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
 
 copy_table_done:
 
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 5420675ce8..f7a8db45ee 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -464,8 +464,16 @@ ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 {
 	if (OidIsValid(relid))
 	{
-		/* Replication origin name for tablesync workers. */
-		snprintf(originname, szoriginname, "pg_%u_%u", suboid, relid);
+		bool		is_null = true;
+
+		/*
+		 * Replication origin name for tablesync workers. First, look into the
+		 * catalog. If originname does not exist, then use the default name.
+		 */
+		GetSubscriptionRelOrigin(suboid, relid,
+								 originname, &is_null);
+		if (is_null)
+			snprintf(originname, szoriginname, "pg_%u_%lld", suboid, (long long) MyLogicalRepWorker->rep_slot_id);
 	}
 	else
 	{
@@ -3770,7 +3778,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* Tablesync workers should end streaming before exiting the main loop
+	/*
+	 * Tablesync workers should end streaming before exiting the main loop
 	 * to drop replication slot. Only end streaming here for apply workers.
 	 */
 	if (!am_tablesync_worker())
@@ -4496,6 +4505,9 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname, int worker_slot
 
 	/* allocate slot name in long-lived context */
 	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+
+	/* Keep the replication slot name used for this sync. */
+	MyLogicalRepWorker->slot_name = *myslotname;
 	pfree(syncslotname);
 }
 
@@ -4551,10 +4563,12 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 	/* Start table synchronization. */
 	start_table_sync(origin_startpos, &slotname, worker_slot);
 
+	StartTransactionCommand();
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   originname_size);
+	CommitTransactionCommand();
 
 	set_apply_error_context_origin(originname);
 
@@ -4595,11 +4609,10 @@ run_apply_worker(WalRcvStreamOptions *options,
 				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
 					errmsg("subscription has no replication slot set")));
 
-	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-									   originname, originname_size);
-
 	/* Setup replication origin tracking. */
 	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
 	originid = replorigin_by_name(originname, true);
 	if (!OidIsValid(originid))
 		originid = replorigin_create(originname);
@@ -4878,12 +4891,12 @@ TablesyncWorkerMain(Datum main_arg)
 				memcpy(rstate, lfirst(lc), sizeof(SubscriptionRelState));
 
 				/*
-				* Pick the table for the next run if it is not already picked up
-				* by another worker.
-				*
-				* Take exclusive lock to prevent any other sync worker from picking
-				* the same table.
-				*/
+				 * Pick the table for the next run if it is not already picked up
+				 * by another worker.
+				 *
+				 * Take exclusive lock to prevent any other sync worker from picking
+				 * the same table.
+				 */
 				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
 				if (rstate->state != SUBREL_STATE_SYNCDONE &&
 					!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
@@ -4915,7 +4928,32 @@ TablesyncWorkerMain(Datum main_arg)
 			}
 
 			if (!is_table_found)
+			{
+				TimeLineID	tli;
+
+				/*
+				 * It is important to give an error if we are unable to drop the
+				 * slot, otherwise, it won't be dropped till the corresponding
+				 * subscription is dropped. So passing missing_ok = false.
+				 */
+				if (MyLogicalRepWorker->created_slot)
+				{
+					walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+					ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
+				}
+
+				/*
+				 * Drop replication origin before exiting.
+				 *
+				 * There is a chance that the user is concurrently performing refresh
+				 * for the subscription where we remove the table state and its origin
+				 * or the apply worker would have removed this origin. So passing
+				 * missing_ok = true.
+				 */
+				replorigin_drop_by_name(originname, true, false);
+
 				break;
+			}
 		}
 	}
 
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 1d40eebc78..7e13f59847 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -108,6 +108,9 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
 
 	/* Only publish data originating from the specified origin */
 	text		suborigin BKI_DEFAULT(LOGICALREP_ORIGIN_ANY);
+
+	/* The last used ID to create a replication slot for tablesync */
+	int64		sublastusedid BKI_DEFAULT(0);
 #endif
 } FormData_pg_subscription;
 
@@ -144,6 +147,8 @@ typedef struct Subscription
 	List	   *publications;	/* List of publication names to subscribe to */
 	char	   *origin;			/* Only publish data originating from the
 								 * specified origin */
+	int64		lastusedid;		/* Last used unique ID to create replication
+								 * slots in tablesync */
 } Subscription;
 
 /* Disallow streaming in-progress transactions. */
@@ -164,6 +169,7 @@ typedef struct Subscription
 extern Subscription *GetSubscription(Oid subid, bool missing_ok);
 extern void FreeSubscription(Subscription *sub);
 extern void DisableSubscription(Oid subid);
+extern void UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid);
 
 extern int	CountDBSubscriptions(Oid dbid);
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 60a2bcca23..185164d75e 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -44,6 +44,12 @@ CATALOG(pg_subscription_rel,6102,SubscriptionRelRelationId)
 											 * used for synchronization
 											 * coordination, or NULL if not
 											 * valid */
+	NameData	srrelslotname BKI_FORCE_NULL;	/* name of the replication
+												 * slot for relation in
+												 * subscription */
+	NameData	srreloriginname BKI_FORCE_NULL; /* origin name for relation in
+												 * subscription */
+
 #endif
 } FormData_pg_subscription_rel;
 
@@ -81,10 +87,16 @@ typedef struct SubscriptionRelState
 } SubscriptionRelState;
 
 extern void AddSubscriptionRelState(Oid subid, Oid relid, char state,
-									XLogRecPtr sublsn);
+									XLogRecPtr sublsn, char *relslotname, char *reloriginname);
 extern void UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 									   XLogRecPtr sublsn);
+extern void UpdateSubscriptionRel(Oid subid, Oid relid, char state,
+								  XLogRecPtr sublsn, char *relslotname, char *reloriginname);
+
 extern char GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn);
+extern void GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname);
+extern void GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull);
+
 extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 
 extern bool HasSubscriptionRelations(Oid subid);
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index a8a89dc784..31b2c41893 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -237,8 +237,9 @@ extern bool InvalidateObsoleteReplicationSlots(ReplicationSlotInvalidationCause
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
 extern bool ReplicationSlotName(int index, Name name);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, Size szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, int64 slotid, char *syncslotname, Size szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List *GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index d3cabf4033..be8aecfc16 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -35,6 +35,23 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/*
+	 * Indicates if the sync worker created a replication slot for itself
+	 * in any point of its lifetime.
+	 * False means that the worker has not created a slot yet, and has been
+	 * reusing replication slots created by other workers so far.
+	 */
+	bool		created_slot;
+
+	/*
+	 * Unique identifier for replication slot to be created by tablesnync
+	 * workers, if needed.
+	 */
+	int64		rep_slot_id;
+
+	/* Replication slot name used by the worker. */
+	char	   *slot_name;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
@@ -239,7 +256,8 @@ extern LogicalRepWorker *logicalrep_worker_find(Oid subid, Oid relid,
 extern List *logicalrep_workers_find(Oid subid, bool only_running);
 extern bool logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname,
 									 Oid userid, Oid relid,
-									 dsm_handle subworker_dsm);
+									 dsm_handle subworker_dsm,
+									 int64 slotid);
 extern void logicalrep_worker_stop(Oid subid, Oid relid);
 extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
 extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
@@ -335,4 +353,7 @@ am_parallel_apply_worker(void)
 	return isParallelApplyWorker(MyLogicalRepWorker);
 }
 
+/* Invalid identifier to be used for naming replication slots */
+#define InvalidRepSlotId	0
+
 #endif							/* WORKER_INTERNAL_H */
-- 
2.27.0

v2-0006-Use-slot-name-as-application_name-again.patchapplication/octet-stream; name=v2-0006-Use-slot-name-as-application_name-again.patchDownload
From f8c891e232da418c8bca614ea5a6825a6ed55e08 Mon Sep 17 00:00:00 2001
From: Hayato Kuroda <kuroda.hayato@fujitsu.com>
Date: Wed, 28 Jun 2023 06:19:34 +0000
Subject: [PATCH v2 6/6] Use slot name as application_name again

---
 src/backend/replication/logical/tablesync.c | 15 +++------------
 1 file changed, 3 insertions(+), 12 deletions(-)

diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index e2b535f920..9728541acf 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -1382,23 +1382,14 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 	 */
 	if (LogRepWorkerWalRcvConn == NULL)
 	{
-		char application_name[NAMEDATALEN];
-
 		/*
-		 * FIXME: set appropriate application_name. Previously, the slot name
-		 * was used becasue the lifetime of the tablesync worker was same as
-		 * that, but now the tablesync worker handles many slots during the
-		 * synchronization so that it is not suitable. So what should be?
-		 * Note that if the tablesync worker starts to reuse the replication
-		 * slot during synchronization, we should use the slot name as
-		 * application_name again.
+		 * Here we use the slot name instead of the subscription name as the
+		 * application_name,
 		 */
-		snprintf(application_name, NAMEDATALEN, "pg_%u_sync_%i",
-				 MySubscription->oid, worker_slot);
 		LogRepWorkerWalRcvConn =
 			walrcv_connect(MySubscription->conninfo, true,
 						   must_use_password,
-						   application_name, &err);
+						   slotname, &err);
 	}
 
 	if (LogRepWorkerWalRcvConn == NULL)
-- 
2.27.0

#82Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Hayato Kuroda (Fujitsu) (#73)
5 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi,

Hayato Kuroda (Fujitsu) <kuroda.hayato@fujitsu.com>, 27 Haz 2023 Sal,
10:42 tarihinde şunu yazdı:

Dear Melih,

Thank you for updating the patch! I have not reviewed yet, but I wanted
to reply your comments.

This actually makes sense. I quickly try to do that without adding any
new replication message. As you would expect, it did not work.
I don't really know what's needed to make a connection to last for
more than one iteration. Need to look into this. Happy to hear any
suggestions and thoughts.

I have analyzed how we handle this. Please see attached the patch (0003) which
allows reusing connection. The patchset passed tests on my CI.
To make cfbot happy I reassigned the patch number.

In this patch, the tablesync worker does not call clean_sync_worker() at the end
of iterations, and the establishment of the connection is done only once.
The creation of memory context is also suppressed.

Regarding the walsender, streamingDone{Sending|Receiving} is now initialized
before executing StartLogicalReplication(). These flags have been used to decide
when the process exits copy mode. The default value is false, and they are set
to true when the copy mode is finished.
I think there was no use-case that the same walsender executes START_REPLICATION
replication twice so there were no codes for restoring flags. Please tell me if any other
reasons.

Thanks for the 0003 patch. But it did not work for me. Can you create
a subscription successfully with patch 0003 applied?
I get the following error: " ERROR: table copy could not start
transaction on publisher: another command is already in progress".

I think streaming needs to be ended before moving to another table. So
I changed the patch a little bit and also addressed the reviews from
recent emails. Please see the attached patch set.

I'm still keeping the reuse connection patch separate for now to see
what is needed clearly.

Thanks,
Melih

Attachments:

v2-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchapplication/octet-stream; name=v2-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchDownload
From a76e29807608fd3ca837483fd8c6c6caf6c3a407 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Mon, 5 Jun 2023 15:04:41 +0300
Subject: [PATCH 1/5] Refactor to split Apply and Tablesync Workers

Both apply and tablesync workers were using ApplyWorkerMain() as entry
point. As the name implies, ApplyWorkerMain() should be considered as
the main function for apply workers. Tablesync worker's path was hidden
and does not have enough in common to share the same main function with
apply worker.

Also; most of the code shared by both worker types are already combined
in LogicalRepApplyLoop(). There is no need to combine the rest in
ApplyWorkerMain() anymore.

This commit introduces TablesyncWorkerMain() as a new entry point for
tablesync workers and separates both type of workers from each other.
This aims to increase code readability and help to maintain logical
replication workers separately.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/postmaster/bgworker.c             |   3 +
 .../replication/logical/applyparallelworker.c |   2 +-
 src/backend/replication/logical/launcher.c    |  32 +-
 src/backend/replication/logical/tablesync.c   |   2 +-
 src/backend/replication/logical/worker.c      | 388 +++++++++++-------
 src/include/replication/logicalworker.h       |   1 +
 src/include/replication/worker_internal.h     |   4 +-
 7 files changed, 270 insertions(+), 162 deletions(-)

diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 5b4bd71694..505e38376c 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -131,6 +131,9 @@ static const struct
 	},
 	{
 		"ParallelApplyWorkerMain", ParallelApplyWorkerMain
+	},
+	{
+		"TablesyncWorkerMain", TablesyncWorkerMain
 	}
 };
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 6fb96148f4..1d4e83c4c1 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -942,7 +942,7 @@ ParallelApplyWorkerMain(Datum main_arg)
 	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
 		MyLogicalRepWorker->reply_time = 0;
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 542af7d863..e231fa7f95 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -459,24 +459,30 @@ retry:
 	snprintf(bgw.bgw_library_name, MAXPGPATH, "postgres");
 
 	if (is_parallel_apply_worker)
+	{
 		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ParallelApplyWorkerMain");
-	else
-		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
-
-	if (OidIsValid(relid))
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication worker for subscription %u sync %u", subid, relid);
-	else if (is_parallel_apply_worker)
+				 "logical replication parallel apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
+	}
+	else if (OidIsValid(relid))
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "TablesyncWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication parallel apply worker for subscription %u", subid);
+				 "logical replication tablesync worker for subscription %u sync %u",
+				 subid,
+				 relid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication tablesync worker");
+	}
 	else
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication apply worker for subscription %u", subid);
-
-	if (is_parallel_apply_worker)
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
-	else
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication worker");
+				 "logical replication apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication apply worker");
+	}
 
 	bgw.bgw_restart_time = BGW_NEVER_RESTART;
 	bgw.bgw_notify_pid = MyProcPid;
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 6d461654ab..8125bbd170 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -131,7 +131,7 @@ static StringInfo copybuf = NULL;
 /*
  * Exit routine for synchronization worker.
  */
-static void
+void
 pg_attribute_noreturn()
 finish_sync_worker(void)
 {
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 0ee764d68f..96ddad356b 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -392,6 +392,9 @@ static void stream_open_file(Oid subid, TransactionId xid,
 static void stream_write_change(char action, StringInfo s);
 static void stream_open_and_write_change(TransactionId xid, char action, StringInfo s);
 static void stream_close_file(void);
+static void set_stream_options(WalRcvStreamOptions *options,
+							   char *slotname,
+							   XLogRecPtr *origin_startpos);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
@@ -4330,6 +4333,72 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+ /* set_stream_options
+  * 	Set logical replication streaming options.
+  *
+  * This function sets streaming options including replication slot name and
+  * origin start position. Workers need these options for logical replication.
+  */
+static void
+set_stream_options(WalRcvStreamOptions *options,
+				   char *slotname,
+				   XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+		options->proto.logical.twophase = true;
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4433,7 +4502,8 @@ start_apply(XLogRecPtr origin_startpos)
 			 * idle state.
 			 */
 			AbortOutOfAnyTransaction();
-			pgstat_report_subscription_error(MySubscription->oid, !am_tablesync_worker());
+			pgstat_report_subscription_error(MySubscription->oid,
+											 !am_tablesync_worker());
 
 			PG_RE_THROW();
 		}
@@ -4442,13 +4512,133 @@ start_apply(XLogRecPtr origin_startpos)
 }
 
 /*
- * Common initialization for leader apply worker and parallel apply worker.
+ * Runs the tablesync worker.
+ * It starts syncing tables. After a successful sync, sets streaming options
+ * and starts streaming to catchup.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Start table synchronization. */
+	start_table_sync(origin_startpos, &slotname);
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   originname_size);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	/* Start applying changes to catchup. */
+	start_apply(*origin_startpos);
+}
+
+/*
+ * Runs the leader apply worker.
+ * It sets up replication origin, streaming options and then starts streaming.
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
+{
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+	bool		must_use_password;
+
+	slotname = MySubscription->slotname;
+
+	/*
+	 * This shouldn't happen if the subscription is enabled, but guard
+	 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+	 * crash if slot is NULL.)
+	 */
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+
+	/* Is the use of a password mandatory? */
+	must_use_password = MySubscription->passwordrequired &&
+		!superuser_arg(MySubscription->owner);
+
+	/* Note that the superuser_arg call can access the DB */
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											must_use_password,
+											MySubscription->name, &err);
+
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+	 * We don't really use the output identify_system for anything but it
+	 * does some initializations on the upstream so let's still call it.
+	 */
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+	{
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
+	}
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+
+	/* Run the main loop. */
+	start_apply(*origin_startpos);
+}
+
+/*
+ * Common initialization for logical replication workers; leader apply worker,
+ * parallel apply worker and tablesync worker.
  *
  * Initialize the database connection, in-memory subscription and necessary
  * config options.
  */
 void
-InitializeApplyWorker(void)
+InitializeLogRepWorker(void)
 {
 	MemoryContext oldctx;
 
@@ -4512,7 +4702,8 @@ InitializeApplyWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has started",
+				(errmsg("%s for subscription \"%s\", table \"%s\" has started",
+						get_worker_name(),
 						MySubscription->name,
 						get_rel_name(MyLogicalRepWorker->relid))));
 	else
@@ -4533,7 +4724,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	InitializingApplyWorker = true;
 
@@ -4557,7 +4747,7 @@ ApplyWorkerMain(Datum main_arg)
 	/* Load the libpq-specific functions */
 	load_file("libpqwalreceiver", false);
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
@@ -4565,71 +4755,6 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-		bool		must_use_password;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-
-		/* Is the use of a password mandatory? */
-		must_use_password = MySubscription->passwordrequired &&
-			!superuser_arg(MySubscription->owner);
-
-		/* Note that the superuser_arg call can access the DB */
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												must_use_password,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
 	 * the subscription relation state.
@@ -4638,92 +4763,63 @@ ApplyWorkerMain(Datum main_arg)
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
+	/* This is leader apply worker */
+	run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
+	proc_exit(0);
+}
 
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
+/* Logical Replication Tablesync worker entry point */
+void
+TablesyncWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *myslotname = NULL;
+	WalRcvStreamOptions options;
+
+	/* Attach to slot */
+	logicalrep_worker_attach(worker_slot);
+
+	/* Setup signal handling */
+	pqsignal(SIGHUP, SignalHandlerForConfigReload);
+	pqsignal(SIGTERM, die);
+	BackgroundWorkerUnblockSignals();
 
 	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 * We don't currently need any ResourceOwner in a walreceiver process, but
+	 * if we did, we could call CreateAuxProcessResourceOwner here.
 	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
 
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
+	/* Initialise stats to a sanish value */
+	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
+		MyLogicalRepWorker->reply_time = GetCurrentTimestamp();
 
-	if (!am_tablesync_worker())
-	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
-		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+	/* Load the libpq-specific functions */
+	load_file("libpqwalreceiver", false);
 
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
-		}
-		else
-		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-		}
+	InitializeLogRepWorker();
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-								 MySubscription->name,
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-								 "?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
+	/* Connect to the origin and start the replication. */
+	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
+		 MySubscription->conninfo);
 
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+	/*
+	 * Setup callback for syscache so that we know when something changes in
+	 * the subscription relation state.
+	 */
+	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
+								  invalidate_syncing_table_states,
+								  (Datum) 0);
 
-	proc_exit(0);
+	run_tablesync_worker(&options,
+						 myslotname,
+						 originname,
+						 sizeof(originname),
+						 &origin_startpos);
+
+	finish_sync_worker();
 }
 
 /*
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 39588da79f..bbd71d0b42 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -18,6 +18,7 @@ extern PGDLLIMPORT volatile sig_atomic_t ParallelApplyMessagePending;
 
 extern void ApplyWorkerMain(Datum main_arg);
 extern void ParallelApplyWorkerMain(Datum main_arg);
+extern void TablesyncWorkerMain(Datum main_arg);
 
 extern bool IsLogicalWorker(void);
 extern bool IsLogicalParallelApplyWorker(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 343e781896..7aba034774 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -265,7 +265,7 @@ extern void maybe_reread_subscription(void);
 
 extern void stream_cleanup_files(Oid subid, TransactionId xid);
 
-extern void InitializeApplyWorker(void);
+extern void InitializeLogRepWorker(void);
 
 extern void store_flush_position(XLogRecPtr remote_lsn, XLogRecPtr local_lsn);
 
@@ -307,6 +307,8 @@ extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
 
 #define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)
 
+extern void finish_sync_worker(void);
+
 static inline bool
 am_tablesync_worker(void)
 {
-- 
2.25.1

v4-0002-Reuse-Tablesync-Workers.patchapplication/octet-stream; name=v4-0002-Reuse-Tablesync-Workers.patchDownload
From d482022b40e0a5ce1b74fd0e320cb5b45da2f671 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:04:46 +0300
Subject: [PATCH 2/5] Reuse Tablesync Workers

This commit allows reusing tablesync workers for syncing more than one
table sequentially during their lifetime, instead of exiting after
only syncing one table.

Before this commit, tablesync workers were capable of syncing only one
table. For each table, a new sync worker was launched and that worker would
exit when done processing the table.

Now, tablesync workers are not limited to processing only one
table. When done, they can move to processing another table in
the same subscription.

If there is a table that needs to be synced, an available tablesync
worker picks up that table and syncs it. Each tablesync worker
continues to pick new tables to sync until there are no tables left
requiring synchronization. If there was no available worker to
process the table, then a new tablesync worker will be launched,
provided the number of tablesync workers for the subscription does not
exceed max_sync_workers_per_subscription.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/replication/logical/launcher.c  |   1 +
 src/backend/replication/logical/tablesync.c |  50 ++++++--
 src/backend/replication/logical/worker.c    | 121 ++++++++++++++++++--
 src/include/replication/worker_internal.h   |   7 ++
 4 files changed, 158 insertions(+), 21 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e231fa7f95..72e5ef8a78 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -440,6 +440,7 @@ retry:
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
+	worker->is_sync_completed = false;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 8125bbd170..605c5bd4ec 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -129,15 +129,14 @@ static bool FetchTableStates(bool *started_tx);
 static StringInfo copybuf = NULL;
 
 /*
- * Exit routine for synchronization worker.
+ * Prepares the synchronization worker for reuse or exit.
  */
 void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+clean_sync_worker(void)
 {
 	/*
-	 * Commit any outstanding transaction. This is the usual case, unless
-	 * there was nothing to do for the table.
+	 * Commit any outstanding transaction. This is the usual case, unless there
+	 * was nothing to do for the table.
 	 */
 	if (IsTransactionState())
 	{
@@ -145,19 +144,38 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
+	/*
+	 * Disconnect from publisher. Otherwise reused sync workers causes
+	 * exceeding max_wal_senders
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		walrcv_disconnect(LogRepWorkerWalRcvConn);
+		LogRepWorkerWalRcvConn = NULL;
+	}
+
+	/* Find the leader apply worker and signal it. */
+	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
+}
+
+/*
+ * Exit routine for synchronization worker.
+ */
+void
+pg_attribute_noreturn()
+finish_sync_worker(void)
+{
+	clean_sync_worker();
+
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
 	StartTransactionCommand();
 	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
+			(errmsg("logical replication table synchronization worker for subscription \"%s\" has finished",
+					MySubscription->name)));
 	CommitTransactionCommand();
 
-	/* Find the leader apply worker and signal it. */
-	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
-
 	/* Stop gracefully */
 	proc_exit(0);
 }
@@ -379,7 +397,15 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 */
 		replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
+		/* Sync worker has completed synchronization of the current table. */
+		MyLogicalRepWorker->is_sync_completed = true;
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", relation \"%s\" with relid %u has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
+		CommitTransactionCommand();
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 96ddad356b..eae561db05 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3626,6 +3626,20 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can call process_syncing_tables_for_sync.
+				 *
+				 * process_syncing_tables_for_sync decides whether the sync of
+				 * the current table is completed. If it is completed,
+				 * streaming must be already ended. So, we can break the loop.
+				 */
+				if (MyLogicalRepWorker->is_sync_completed)
+				{
+					endofstream = true;
+					break;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -3645,6 +3659,15 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+
+			/*
+			 * If is_sync_completed is true, this means that the tablesync
+			 * worker is done with synchronization. Streaming has already been
+			 * ended by process_syncing_tables_for_sync. We should move to the
+			 * next table if needed, or exit.
+			 */
+			if (MyLogicalRepWorker->is_sync_completed)
+				endofstream = true;
 		}
 
 		/* Cleanup the memory. */
@@ -3747,8 +3770,12 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * Tablesync workers should end streaming before exiting the main loop to
+	 * drop replication slot. Only end streaming here for apply workers.
+	 */
+	if (!am_tablesync_worker())
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
 }
 
 /*
@@ -4523,6 +4550,8 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 					 int originname_size,
 					 XLogRecPtr *origin_startpos)
 {
+	MyLogicalRepWorker->is_sync_completed = false;
+
 	/* Start table synchronization. */
 	start_table_sync(origin_startpos, &slotname);
 
@@ -4702,10 +4731,11 @@ InitializeLogRepWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("%s for subscription \"%s\", table \"%s\" has started",
+				(errmsg("%s for subscription \"%s\", table \"%s\" with relid %u has started",
 						get_worker_name(),
 						MySubscription->name,
-						get_rel_name(MyLogicalRepWorker->relid))));
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
 	else
 		ereport(LOG,
 		/* translator: first %s is the name of logical replication worker */
@@ -4813,11 +4843,84 @@ TablesyncWorkerMain(Datum main_arg)
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	run_tablesync_worker(&options,
-						 myslotname,
-						 originname,
-						 sizeof(originname),
-						 &origin_startpos);
+	/*
+	 * The loop where worker does its job. It loops until there is no relation
+	 * left to sync.
+	 */
+	for (;;)
+	{
+		List	   *rstates;
+		ListCell   *lc;
+		bool 	is_table_found = false;
+
+		run_tablesync_worker(&options,
+							 myslotname,
+							 originname,
+							 sizeof(originname),
+							 &origin_startpos);
+
+		if (IsTransactionState())
+			CommitTransactionCommand();
+
+		if (MyLogicalRepWorker->is_sync_completed)
+		{
+			/* This transaction will be committed by clean_sync_worker. */
+			StartTransactionCommand();
+
+			/*
+			 * Check if there is any table whose relation state is still INIT.
+			 * If a table in INIT state is found, the worker will not be
+			 * finished, it will be reused instead.
+			 */
+			rstates = GetSubscriptionRelations(MySubscription->oid, true);
+
+			foreach(lc, rstates)
+			{
+				SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
+
+				if (rstate->state == SUBREL_STATE_SYNCDONE)
+					continue;
+
+				/*
+				 * Take exclusive lock to prevent any other sync worker from
+				 * picking the same table.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
+
+				/*
+				 * Pick the table for the next run if it is not already picked
+				 * up by another worker.
+				 */
+				if (!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+				{
+					/* Update worker state for the next table */
+					MyLogicalRepWorker->relid = rstate->relid;
+					MyLogicalRepWorker->relstate = rstate->state;
+					MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+					LWLockRelease(LogicalRepWorkerLock);
+
+					/* Found a table for next iteration */
+					is_table_found = true;
+					clean_sync_worker();
+
+					StartTransactionCommand();
+					ereport(LOG,
+							(errmsg("%s for subscription \"%s\" has moved to sync table \"%s\" with relid %u.",
+									get_worker_name(),
+									MySubscription->name,
+									get_rel_name(MyLogicalRepWorker->relid),
+									MyLogicalRepWorker->relid)));
+					CommitTransactionCommand();
+
+					break;
+				}
+				LWLockRelease(LogicalRepWorkerLock);
+			}
+
+			if (!is_table_found)
+				break;
+		}
+	}
 
 	finish_sync_worker();
 }
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 7aba034774..1e9f8e6e72 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -56,6 +56,12 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/*
+	 * Indicates whether tablesync worker has completed sycning its assigned
+	 * table. If true, no need to continue with that table.
+	 */
+	bool		is_sync_completed;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction or
@@ -308,6 +314,7 @@ extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
 #define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)
 
 extern void finish_sync_worker(void);
+extern void clean_sync_worker(void);
 
 static inline bool
 am_tablesync_worker(void)
-- 
2.25.1

v3-0003-reuse-connection-when-tablesync-workers-change-the-t.patchapplication/octet-stream; name=v3-0003-reuse-connection-when-tablesync-workers-change-the-t.patchDownload
From ae042ea7aabacfa6a97960d4e5e3c8810bffd74e Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:13:52 +0300
Subject: [PATCH 3/5] reuse connection when tablesync workers change the target

---
 src/backend/replication/logical/tablesync.c | 53 ++++++++++++++-------
 src/backend/replication/logical/worker.c    | 40 +++++++++++-----
 src/backend/replication/walsender.c         |  6 +++
 src/include/replication/worker_internal.h   |  3 +-
 4 files changed, 71 insertions(+), 31 deletions(-)

diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 605c5bd4ec..f042d9ae00 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -144,16 +144,6 @@ clean_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
-	/*
-	 * Disconnect from publisher. Otherwise reused sync workers causes
-	 * exceeding max_wal_senders
-	 */
-	if (LogRepWorkerWalRcvConn != NULL)
-	{
-		walrcv_disconnect(LogRepWorkerWalRcvConn);
-		LogRepWorkerWalRcvConn = NULL;
-	}
-
 	/* Find the leader apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
 }
@@ -167,6 +157,16 @@ finish_sync_worker(void)
 {
 	clean_sync_worker();
 
+	/*
+	 * Disconnect from publisher. Otherwise reused sync workers causes
+	 * exceeding max_wal_senders.
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		walrcv_disconnect(LogRepWorkerWalRcvConn);
+		LogRepWorkerWalRcvConn = NULL;
+	}
+
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
@@ -1268,7 +1268,7 @@ ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
  * The returned slot name is palloc'ed in current memory context.
  */
 char *
-LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
+LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 {
 	char	   *slotname;
 	char	   *err;
@@ -1321,14 +1321,31 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 									NAMEDATALEN);
 
 	/*
-	 * Here we use the slot name instead of the subscription name as the
-	 * application_name, so that it is different from the leader apply worker,
-	 * so that synchronous replication can distinguish them.
+	 * Connect to publisher if not yet. The application_name must be also
+	 * different from the leader apply worker because synchronous replication
+	 * must distinguish them.
 	 */
-	LogRepWorkerWalRcvConn =
-		walrcv_connect(MySubscription->conninfo, true,
-					   must_use_password,
-					   slotname, &err);
+	if (LogRepWorkerWalRcvConn == NULL)
+	{
+		char application_name[NAMEDATALEN];
+
+		/*
+		 * FIXME: set appropriate application_name. Previously, the slot name
+		 * was used becasue the lifetime of the tablesync worker was same as
+		 * that, but now the tablesync worker handles many slots during the
+		 * synchronization so that it is not suitable. So what should be?
+		 * Note that if the tablesync worker starts to reuse the replication
+		 * slot during synchronization, we should use the slot name as
+		 * application_name again.
+		 */
+		snprintf(application_name, NAMEDATALEN, "pg_%u_sync_%i",
+				 MySubscription->oid, worker_slot);
+		LogRepWorkerWalRcvConn =
+			walrcv_connect(MySubscription->conninfo, true,
+						   must_use_password,
+						   application_name, &err);
+	}
+
 	if (LogRepWorkerWalRcvConn == NULL)
 		ereport(ERROR,
 				(errcode(ERRCODE_CONNECTION_FAILURE),
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index eae561db05..537cd33a30 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3500,19 +3500,21 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 	/*
 	 * Init the ApplyMessageContext which we clean up after each replication
-	 * protocol message.
+	 * protocol message, if needed.
 	 */
-	ApplyMessageContext = AllocSetContextCreate(ApplyContext,
-												"ApplyMessageContext",
-												ALLOCSET_DEFAULT_SIZES);
+	if (!ApplyMessageContext)
+		ApplyMessageContext = AllocSetContextCreate(ApplyContext,
+													"ApplyMessageContext",
+													ALLOCSET_DEFAULT_SIZES);
 
 	/*
 	 * This memory context is used for per-stream data when the streaming mode
 	 * is enabled. This context is reset on each stream stop.
 	 */
-	LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
-													"LogicalStreamingContext",
-													ALLOCSET_DEFAULT_SIZES);
+	if (!LogicalStreamingContext)
+		LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
+														"LogicalStreamingContext",
+														ALLOCSET_DEFAULT_SIZES);
 
 	/* mark as idle, before starting to loop */
 	pgstat_report_activity(STATE_IDLE, NULL);
@@ -4468,7 +4470,9 @@ TwoPhaseTransactionGid(Oid subid, TransactionId xid, char *gid, int szgid)
  * are not repeatable.
  */
 static void
-start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+start_table_sync(XLogRecPtr *origin_startpos,
+				 char **myslotname,
+				 int worker_slot)
 {
 	char	   *syncslotname = NULL;
 
@@ -4477,7 +4481,7 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 	PG_TRY();
 	{
 		/* Call initial sync. */
-		syncslotname = LogicalRepSyncTableStart(origin_startpos);
+		syncslotname = LogicalRepSyncTableStart(origin_startpos, worker_slot);
 	}
 	PG_CATCH();
 	{
@@ -4548,12 +4552,23 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 					 char *slotname,
 					 char *originname,
 					 int originname_size,
-					 XLogRecPtr *origin_startpos)
+					 XLogRecPtr *origin_startpos,
+					 int worker_slot)
 {
 	MyLogicalRepWorker->is_sync_completed = false;
 
+	/*
+	 * If it's already connected to the publisher, end streaming before using
+	 * the same connection for another iteration
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		TimeLineID tli;
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	}
+
 	/* Start table synchronization. */
-	start_table_sync(origin_startpos, &slotname);
+	start_table_sync(origin_startpos, &slotname, worker_slot);
 
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
@@ -4857,7 +4872,8 @@ TablesyncWorkerMain(Datum main_arg)
 							 myslotname,
 							 originname,
 							 sizeof(originname),
-							 &origin_startpos);
+							 &origin_startpos,
+							 worker_slot);
 
 		if (IsTransactionState())
 			CommitTransactionCommand();
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index d3a136b6f5..429d00f2f0 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1828,6 +1828,12 @@ exec_replication_command(const char *cmd_string)
 				set_ps_display(cmdtag);
 				PreventInTransactionBlock(true, cmdtag);
 
+				/*
+				 * Initialize the flag again because this streaming may be
+				 * second time.
+				 */
+				streamingDoneSending = streamingDoneReceiving = false;
+
 				if (cmd->kind == REPLICATION_KIND_PHYSICAL)
 					StartReplication(cmd);
 				else
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 1e9f8e6e72..af6fd339f7 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -249,7 +249,8 @@ extern int	logicalrep_sync_worker_count(Oid subid);
 
 extern void ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 											   char *originname, Size szoriginname);
-extern char *LogicalRepSyncTableStart(XLogRecPtr *origin_startpos);
+extern char *LogicalRepSyncTableStart(XLogRecPtr *origin_startpos,
+									  int worker_slot);
 
 extern bool AllTablesyncsReady(void);
 extern void UpdateTwoPhaseState(Oid suboid, char new_state);
-- 
2.25.1

v12-0004-Add-replication-protocol-cmd-to-create-a-snapshot.patchapplication/octet-stream; name=v12-0004-Add-replication-protocol-cmd-to-create-a-snapshot.patchDownload
From 55a72eee60c14d18d84c93f4ec0dc5b70648406d Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 13 Oct 2022 17:05:45 +0300
Subject: [PATCH 4/5] Add replication protocol cmd to create a snapshot

Introduced CREATE_REPLICATION_SNAPSHOT to be able to create and use a
snapshot without creating a new replication slot, but by using an
existing slot.

CREATE_REPLICATION_SNAPSHOT simply does what CREATE_REPLICATION_SLOT does
without creating a new replication slot.

CREATE_REPLICATION_SNAPSHOT command imports the snapshot into the current
transaction and returns consistent_point. The changes earlier than the
consistent_point will be applied by importing the snapshot. All changes
later than the consistent_point will be available to be consumed from
the replication slot.

This is useful for reusing replication slots in logical replication.
Otherwise, tablesync workers cannot start from a consistent point to copy
a relation and then apply changes by consuming from replication slot.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/protocol.sgml                    |  31 ++++++
 .../libpqwalreceiver/libpqwalreceiver.c       |  69 +++++++++++-
 src/backend/replication/logical/logical.c     |  40 ++++++-
 .../replication/logical/logicalfuncs.c        |   1 +
 src/backend/replication/repl_gram.y           |  18 ++-
 src/backend/replication/repl_scanner.l        |   2 +
 src/backend/replication/slotfuncs.c           |   1 +
 src/backend/replication/walsender.c           | 104 +++++++++++++++++-
 src/include/nodes/replnodes.h                 |  11 ++
 src/include/replication/logical.h             |   1 +
 src/include/replication/walreceiver.h         |  13 +++
 src/tools/pgindent/typedefs.list              |   2 +
 12 files changed, 289 insertions(+), 4 deletions(-)

diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index b11d9a6ba3..f7d5acee93 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2595,6 +2595,37 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
      </listitem>
     </varlistentry>
 
+    <varlistentry id="protocol-replication-replication-slot-snapshot">
+     <term><literal>CREATE_REPLICATION_SNAPSHOT</literal> <replaceable class="parameter">slot_name</replaceable> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
+      <indexterm><primary>CREATE_REPLICATION_SNAPSHOT</primary></indexterm>
+     </term>
+     <listitem>
+      <para>
+       Creates a snapshot including all the changes from the replication slot until
+       the point at which the replication slot becomes consistent. Then the snapshot
+       is used in the current transaction. This command is currently only supported
+       for logical replication slots.
+      </para>
+
+      <para>
+       In response to this command, the server will return a one-row result set,
+       containing the following field:
+       <variablelist>
+        <varlistentry>
+         <term><literal>consistent_point</literal> (<type>text</type>)</term>
+         <listitem>
+          <para>
+           The WAL location at which the slot became consistent.  This is the
+           earliest location from which streaming can start on this replication
+           slot.
+          </para>
+         </listitem>
+        </varlistentry>
+       </variablelist>
+      </para>
+     </listitem>
+    </varlistentry>
+
     <varlistentry id="protocol-replication-base-backup" xreflabel="BASE_BACKUP">
      <term><literal>BASE_BACKUP</literal> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
       <indexterm><primary>BASE_BACKUP</primary></indexterm>
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index dc9c5c82d9..17358bf552 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -82,6 +82,8 @@ static WalRcvExecResult *libpqrcv_exec(WalReceiverConn *conn,
 									   const int nRetTypes,
 									   const Oid *retTypes);
 static void libpqrcv_disconnect(WalReceiverConn *conn);
+static void libpqrcv_slot_snapshot(WalReceiverConn *conn, char *slotname,
+								   const WalRcvStreamOptions *options, XLogRecPtr *lsn);
 
 static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_connect = libpqrcv_connect,
@@ -98,7 +100,8 @@ static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_create_slot = libpqrcv_create_slot,
 	.walrcv_get_backend_pid = libpqrcv_get_backend_pid,
 	.walrcv_exec = libpqrcv_exec,
-	.walrcv_disconnect = libpqrcv_disconnect
+	.walrcv_disconnect = libpqrcv_disconnect,
+	.walrcv_slot_snapshot = libpqrcv_slot_snapshot
 };
 
 /* Prototypes for private functions */
@@ -982,6 +985,70 @@ libpqrcv_create_slot(WalReceiverConn *conn, const char *slotname,
 	return snapshot;
 }
 
+/*
+ * Create a new snapshot by using an existing replication slot.
+ */
+static void
+libpqrcv_slot_snapshot(WalReceiverConn *conn,
+					   char *slotname,
+					   const WalRcvStreamOptions *options,
+					   XLogRecPtr *lsn)
+{
+	StringInfoData cmd;
+	PGresult   *res;
+	char	   *pubnames_str;
+	List	   *pubnames;
+	char	   *pubnames_literal;
+
+	initStringInfo(&cmd);
+
+	/* Build the command. */
+	appendStringInfo(&cmd, "CREATE_REPLICATION_SNAPSHOT \"%s\"", slotname);
+	appendStringInfoString(&cmd, " (");
+	appendStringInfo(&cmd, " proto_version '%u'",
+					 options->proto.logical.proto_version);
+
+	/* Add publication names. */
+	pubnames = options->proto.logical.publication_names;
+	pubnames_str = stringlist_to_identifierstr(conn->streamConn, pubnames);
+	if (!pubnames_str)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("Could not create a snapshot by replication slot %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	pubnames_literal = PQescapeLiteral(conn->streamConn, pubnames_str,
+									   strlen(pubnames_str));
+	if (!pubnames_literal)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("Could not create a snapshot by replication slot %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	appendStringInfo(&cmd, ", publication_names %s", pubnames_literal);
+	PQfreemem(pubnames_literal);
+	pfree(pubnames_str);
+
+	appendStringInfoString(&cmd, " )");
+
+	/* Execute the command. */
+	res = libpqrcv_PQexec(conn->streamConn, cmd.data);
+	pfree(cmd.data);
+
+	if (PQresultStatus(res) != PGRES_TUPLES_OK)
+	{
+		PQclear(res);
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg("Could not create a snapshot by replication slot \"%s\": %s",
+						slotname, pchomp(PQerrorMessage(conn->streamConn)))));
+	}
+
+	if (lsn)
+		*lsn = DatumGetLSN(DirectFunctionCall1Coll(pg_lsn_in, InvalidOid,
+												   CStringGetDatum(PQgetvalue(res, 0, 0))));
+
+	PQclear(res);
+}
+
 /*
  * Return PID of remote backend process.
  */
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 41243d0187..3c11f8e1dd 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -476,6 +476,10 @@ CreateInitDecodingContext(const char *plugin,
  * fast_forward
  *		bypass the generation of logical changes.
  *
+ * need_full_snapshot
+ * 		if true, must obtain a snapshot able to read all tables;
+ *  	if false, one that can read only catalogs is acceptable.
+ *
  * xl_routine
  *		XLogReaderRoutine used by underlying xlogreader
  *
@@ -494,6 +498,7 @@ LogicalDecodingContext *
 CreateDecodingContext(XLogRecPtr start_lsn,
 					  List *output_plugin_options,
 					  bool fast_forward,
+					  bool need_full_snapshot,
 					  XLogReaderRoutine *xl_routine,
 					  LogicalOutputPluginWriterPrepareWrite prepare_write,
 					  LogicalOutputPluginWriterWrite do_write,
@@ -502,6 +507,7 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 	LogicalDecodingContext *ctx;
 	ReplicationSlot *slot;
 	MemoryContext old_context;
+	TransactionId xmin_horizon = InvalidTransactionId;
 
 	/* shorter lines... */
 	slot = MyReplicationSlot;
@@ -571,8 +577,40 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 		start_lsn = slot->data.confirmed_flush;
 	}
 
+
+	/*
+	 * We need to determine a safe xmin horizon to start decoding from if we
+	 * want to create a snapshot too. Otherwise we would end up with a
+	 * snapshot that cannot be imported since xmin value from the snapshot may
+	 * be less than the oldest safe xmin. To avoid this call
+	 * GetOldestSafeDecodingTransactionId() to return a safe xmin value, which
+	 * can be used while exporting/importing the snapshot.
+	 *
+	 * So we have to acquire the ProcArrayLock to prevent computation of new
+	 * xmin horizons by other backends, get the safe decoding xid, and inform
+	 * the slot machinery about the new limit. Once that's done the
+	 * ProcArrayLock can be released as the slot machinery now is protecting
+	 * against vacuum.
+	 */
+	if (need_full_snapshot)
+	{
+		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+		xmin_horizon = GetOldestSafeDecodingTransactionId(!need_full_snapshot);
+
+		SpinLockAcquire(&slot->mutex);
+		slot->effective_catalog_xmin = xmin_horizon;
+		slot->data.catalog_xmin = xmin_horizon;
+		slot->effective_xmin = xmin_horizon;
+		SpinLockRelease(&slot->mutex);
+
+		ReplicationSlotsComputeRequiredXmin(true);
+
+		LWLockRelease(ProcArrayLock);
+	}
+
 	ctx = StartupDecodingContext(output_plugin_options,
-								 start_lsn, InvalidTransactionId, false,
+								 start_lsn, xmin_horizon, need_full_snapshot,
 								 fast_forward, xl_routine, prepare_write,
 								 do_write, update_progress);
 
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index 55a24c02c9..85c5cdb633 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -208,6 +208,7 @@ pg_logical_slot_get_changes_guts(FunctionCallInfo fcinfo, bool confirm, bool bin
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									options,
 									false,
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/repl_gram.y b/src/backend/replication/repl_gram.y
index 0c874e33cf..957a5cc653 100644
--- a/src/backend/replication/repl_gram.y
+++ b/src/backend/replication/repl_gram.y
@@ -65,6 +65,7 @@ Node *replication_parse_result;
 %token K_CREATE_REPLICATION_SLOT
 %token K_DROP_REPLICATION_SLOT
 %token K_TIMELINE_HISTORY
+%token K_CREATE_REPLICATION_SNAPSHOT
 %token K_WAIT
 %token K_TIMELINE
 %token K_PHYSICAL
@@ -80,7 +81,7 @@ Node *replication_parse_result;
 %type <node>	command
 %type <node>	base_backup start_replication start_logical_replication
 				create_replication_slot drop_replication_slot identify_system
-				read_replication_slot timeline_history show
+				read_replication_slot timeline_history show create_replication_snapshot
 %type <list>	generic_option_list
 %type <defelt>	generic_option
 %type <uintval>	opt_timeline
@@ -114,6 +115,7 @@ command:
 			| read_replication_slot
 			| timeline_history
 			| show
+			| create_replication_snapshot
 			;
 
 /*
@@ -307,6 +309,19 @@ timeline_history:
 				}
 			;
 
+/*
+ * CREATE_REPLICATION_SNAPSHOT %s options
+ */
+create_replication_snapshot:
+			K_CREATE_REPLICATION_SNAPSHOT var_name plugin_options
+				{
+					CreateReplicationSnapshotCmd *n = makeNode(CreateReplicationSnapshotCmd);
+					n->slotname = $2;
+					n->options = $3;
+					$$ = (Node *) n;
+				}
+			;
+
 opt_physical:
 			K_PHYSICAL
 			| /* EMPTY */
@@ -400,6 +415,7 @@ ident_or_keyword:
 			| K_CREATE_REPLICATION_SLOT	{ $$ = "create_replication_slot"; }
 			| K_DROP_REPLICATION_SLOT		{ $$ = "drop_replication_slot"; }
 			| K_TIMELINE_HISTORY			{ $$ = "timeline_history"; }
+			| K_CREATE_REPLICATION_SNAPSHOT	{ $$ = "create_replication_snapshot"; }
 			| K_WAIT						{ $$ = "wait"; }
 			| K_TIMELINE					{ $$ = "timeline"; }
 			| K_PHYSICAL					{ $$ = "physical"; }
diff --git a/src/backend/replication/repl_scanner.l b/src/backend/replication/repl_scanner.l
index cb467ca46f..5ba2e9c54b 100644
--- a/src/backend/replication/repl_scanner.l
+++ b/src/backend/replication/repl_scanner.l
@@ -126,6 +126,7 @@ START_REPLICATION	{ return K_START_REPLICATION; }
 CREATE_REPLICATION_SLOT		{ return K_CREATE_REPLICATION_SLOT; }
 DROP_REPLICATION_SLOT		{ return K_DROP_REPLICATION_SLOT; }
 TIMELINE_HISTORY	{ return K_TIMELINE_HISTORY; }
+CREATE_REPLICATION_SNAPSHOT	{ return K_CREATE_REPLICATION_SNAPSHOT; }
 PHYSICAL			{ return K_PHYSICAL; }
 RESERVE_WAL			{ return K_RESERVE_WAL; }
 LOGICAL				{ return K_LOGICAL; }
@@ -303,6 +304,7 @@ replication_scanner_is_replication_command(void)
 		case K_DROP_REPLICATION_SLOT:
 		case K_READ_REPLICATION_SLOT:
 		case K_TIMELINE_HISTORY:
+		case K_CREATE_REPLICATION_SNAPSHOT:
 		case K_SHOW:
 			/* Yes; push back the first token so we can parse later. */
 			repl_pushed_back_token = first_token;
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 6035cf4816..c5b2d5b61f 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -486,6 +486,7 @@ pg_logical_replication_slot_advance(XLogRecPtr moveto)
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									NIL,
 									true,	/* fast_forward */
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 429d00f2f0..974ab5ca35 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -238,6 +238,7 @@ static void CreateReplicationSlot(CreateReplicationSlotCmd *cmd);
 static void DropReplicationSlot(DropReplicationSlotCmd *cmd);
 static void StartReplication(StartReplicationCmd *cmd);
 static void StartLogicalReplication(StartReplicationCmd *cmd);
+static void CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd);
 static void ProcessStandbyMessage(void);
 static void ProcessStandbyReplyMessage(void);
 static void ProcessStandbyHSFeedbackMessage(void);
@@ -1284,7 +1285,7 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	 * are reported early.
 	 */
 	logical_decoding_ctx =
-		CreateDecodingContext(cmd->startpoint, cmd->options, false,
+		CreateDecodingContext(cmd->startpoint, cmd->options, false, false,
 							  XL_ROUTINE(.page_read = logical_read_xlog_page,
 										 .segment_open = WalSndSegmentOpen,
 										 .segment_close = wal_segment_close),
@@ -1336,6 +1337,98 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	EndCommand(&qc, DestRemote, false);
 }
 
+/*
+ * Create a snapshot from an existing replication slot.
+ */
+static void
+CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd)
+{
+	Snapshot	snap;
+	LogicalDecodingContext *ctx;
+	char		xloc[MAXFNAMELEN];
+	DestReceiver *dest;
+	TupOutputState *tstate;
+	TupleDesc	tupdesc;
+	Datum		values[1];
+	bool		nulls[1] = {0};
+
+	Assert(!MyReplicationSlot);
+
+	CheckLogicalDecodingRequirements();
+
+	if (!IsTransactionBlock())
+		ereport(ERROR,
+				(errmsg("%s must be called inside a transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (XactIsoLevel != XACT_REPEATABLE_READ)
+		ereport(ERROR,
+				(errmsg("%s must be called in REPEATABLE READ isolation mode transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (!XactReadOnly)
+		ereport(ERROR,
+				(errmsg("%s must be called in a read only transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (FirstSnapshotSet)
+		ereport(ERROR,
+				(errmsg("%s must be called before any query",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (IsSubTransaction())
+		ereport(ERROR,
+				(errmsg("%s must not be called in a subtransaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	ReplicationSlotAcquire(cmd->slotname, false);
+
+	ctx = CreateDecodingContext(MyReplicationSlot->data.restart_lsn,
+								cmd->options,
+								false,
+								true,
+								XL_ROUTINE(.page_read = logical_read_xlog_page,
+										   .segment_open = WalSndSegmentOpen,
+										   .segment_close = wal_segment_close),
+								WalSndPrepareWrite, WalSndWriteData,
+								WalSndUpdateProgress);
+
+	/*
+	 * Signal that we don't need the timeout mechanism. We're just creating
+	 * the snapshot with the replication slot and don't yet accept feedback
+	 * messages or send keepalives. As we possibly need to wait for further
+	 * WAL the walsender would otherwise possibly be killed too soon.
+	 */
+	last_reply_timestamp = 0;
+
+	/* build initial snapshot, might take a while */
+	DecodingContextFindStartpoint(ctx);
+
+	snap = SnapBuildInitialSnapshot(ctx->snapshot_builder);
+	RestoreTransactionSnapshot(snap, MyProc);
+
+	/* Don't need the decoding context anymore */
+	FreeDecodingContext(ctx);
+
+	/* Create a tuple to send consistent WAL location */
+	snprintf(xloc, sizeof(xloc), "%X/%X",
+			 LSN_FORMAT_ARGS(MyReplicationSlot->data.confirmed_flush));
+
+	dest = CreateDestReceiver(DestRemoteSimple);
+	tupdesc = CreateTemplateTupleDesc(1);
+	TupleDescInitBuiltinEntry(tupdesc, (AttrNumber) 1, "consistent_point",
+							  TEXTOID, -1, 0);
+	tstate = begin_tup_output_tupdesc(dest, tupdesc, &TTSOpsVirtual);
+
+	/* consistent wal location */
+	values[0] = CStringGetTextDatum(xloc);
+
+	do_tup_output(tstate, values, nulls);
+	end_tup_output(tstate);
+
+	ReplicationSlotRelease();
+}
+
 /*
  * LogicalDecodingContext 'prepare_write' callback.
  *
@@ -1870,6 +1963,15 @@ exec_replication_command(const char *cmd_string)
 			}
 			break;
 
+		case T_CreateReplicationSnapshotCmd:
+			{
+				cmdtag = "CREATE_REPLICATION_SNAPSHOT";
+				set_ps_display(cmdtag);
+				CreateReplicationSnapshot((CreateReplicationSnapshotCmd *) cmd_node);
+				EndReplicationCommand(cmdtag);
+				break;
+			}
+
 		default:
 			elog(ERROR, "unrecognized replication command node tag: %u",
 				 cmd_node->type);
diff --git a/src/include/nodes/replnodes.h b/src/include/nodes/replnodes.h
index 4321ba8f86..154ab74f33 100644
--- a/src/include/nodes/replnodes.h
+++ b/src/include/nodes/replnodes.h
@@ -108,4 +108,15 @@ typedef struct TimeLineHistoryCmd
 	TimeLineID	timeline;
 } TimeLineHistoryCmd;
 
+/* ----------------------
+ *		CREATE_REPLICATION_SNAPSHOT command
+ * ----------------------
+ */
+typedef struct CreateReplicationSnapshotCmd
+{
+	NodeTag		type;
+	char	   *slotname;
+	List	   *options;
+} CreateReplicationSnapshotCmd;
+
 #endif							/* REPLNODES_H */
diff --git a/src/include/replication/logical.h b/src/include/replication/logical.h
index 5f49554ea0..6535786a0e 100644
--- a/src/include/replication/logical.h
+++ b/src/include/replication/logical.h
@@ -125,6 +125,7 @@ extern LogicalDecodingContext *CreateInitDecodingContext(const char *plugin,
 extern LogicalDecodingContext *CreateDecodingContext(XLogRecPtr start_lsn,
 													 List *output_plugin_options,
 													 bool fast_forward,
+													 bool need_full_snapshot,
 													 XLogReaderRoutine *xl_routine,
 													 LogicalOutputPluginWriterPrepareWrite prepare_write,
 													 LogicalOutputPluginWriterWrite do_write,
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 281626fa6f..f01e5f23a7 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -386,6 +386,16 @@ typedef WalRcvExecResult *(*walrcv_exec_fn) (WalReceiverConn *conn,
  */
 typedef void (*walrcv_disconnect_fn) (WalReceiverConn *conn);
 
+/*
+ * walrcv_slot_snapshot_fn
+ *
+ * Create a snapshot by an existing replication slot
+ */
+typedef void (*walrcv_slot_snapshot_fn) (WalReceiverConn *conn,
+										 char *slotname,
+										 const WalRcvStreamOptions *options,
+										 XLogRecPtr *lsn);
+
 typedef struct WalReceiverFunctionsType
 {
 	walrcv_connect_fn walrcv_connect;
@@ -403,6 +413,7 @@ typedef struct WalReceiverFunctionsType
 	walrcv_get_backend_pid_fn walrcv_get_backend_pid;
 	walrcv_exec_fn walrcv_exec;
 	walrcv_disconnect_fn walrcv_disconnect;
+	walrcv_slot_snapshot_fn walrcv_slot_snapshot;
 } WalReceiverFunctionsType;
 
 extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
@@ -437,6 +448,8 @@ extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
 	WalReceiverFunctions->walrcv_exec(conn, exec, nRetTypes, retTypes)
 #define walrcv_disconnect(conn) \
 	WalReceiverFunctions->walrcv_disconnect(conn)
+#define walrcv_slot_snapshot(conn, slotname, options, lsn) \
+	WalReceiverFunctions->walrcv_slot_snapshot(conn, slotname, options, lsn)
 
 static inline void
 walrcv_clear_result(WalRcvExecResult *walres)
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index e941fb6c82..cc023b10fa 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2348,6 +2348,7 @@ ReplicationSlotInvalidationCause
 ReplicationSlotOnDisk
 ReplicationSlotPersistency
 ReplicationSlotPersistentData
+CreateReplicationSnapshotCmd
 ReplicationState
 ReplicationStateCtl
 ReplicationStateOnDisk
@@ -3855,6 +3856,7 @@ walrcv_receive_fn
 walrcv_send_fn
 walrcv_server_version_fn
 walrcv_startstreaming_fn
+walrcv_slot_snapshot_fn
 wchar2mb_with_len_converter
 wchar_t
 win32_deadchild_waitinfo
-- 
2.25.1

v15-0005-Reuse-Replication-Slot-and-Origin-in-Tablesync.patchapplication/octet-stream; name=v15-0005-Reuse-Replication-Slot-and-Origin-in-Tablesync.patchDownload
From e429474581810c668af7f03d16a17f112a4e295f Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH 5/5] Reuse Replication Slot and Origin in Tablesync

This commit allows reusing replication slots and origins during tablesync.

Earlier, a tablesync worker was creating a new replication slot and origin
each time it syncs a new table. With this patch, replication
slots/origins can be reusable for tablesync.

This reduces the overhead of creating/dropping replication slots and origins
and improves tablesync speed significantly especially for empty or small tables.

If the state of the current table is INIT or DATASYNC, tablesync worker needs a
repliation slot/origin. If the worker has not created slot and origin in
its previous runs, it will create those first. Otherwise the worker reuses
slot and origin created by the same worker in previous iterations earlier.
Tables in FINISHEDCOPY are expected to have a replication slot and origin.
Slot and origin names for such tables are persisted in
pg_subscription_rel catalog. Tablesync worker can fetch them and proceed
with existing slot and origin of FINISHEDCOPY tables and does not need to
create new ones.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/catalogs.sgml                    |  31 ++
 src/backend/catalog/pg_subscription.c         | 247 +++++++++++-
 src/backend/commands/subscriptioncmds.c       | 229 +++++++----
 .../replication/logical/applyparallelworker.c |   3 +-
 src/backend/replication/logical/launcher.c    |   8 +-
 src/backend/replication/logical/tablesync.c   | 373 +++++++++++++-----
 src/backend/replication/logical/worker.c      |  47 ++-
 src/include/catalog/pg_subscription.h         |   6 +
 src/include/catalog/pg_subscription_rel.h     |  14 +-
 src/include/replication/slot.h                |   3 +-
 src/include/replication/worker_internal.h     |  23 +-
 11 files changed, 793 insertions(+), 191 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 852cb30ae1..60718ab587 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8030,6 +8030,19 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        origin.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>sublastusedid</structfield> <type>int8</type>
+      </para>
+      <para>
+      The last used ID for tablesync workers. It acts as an unique identifier
+      for replication slots which are created by tablesync workers.
+      The last used ID needs to be persisted to make logical replication safely
+      proceed after any interruption. If sublastusedid is 0, then no table has
+      been synced yet.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
@@ -8114,6 +8127,24 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        otherwise null
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srrelslotname</structfield> <type>name</type>
+      </para>
+      <para>
+       Replication slot name that is used for synchronization of relation
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srreloriginname</structfield> <type>name</type>
+      </para>
+      <para>
+       Origin name that is used for tracking synchronization of relation
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index d07f88ce28..152fdaa310 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -108,6 +108,14 @@ GetSubscription(Oid subid, bool missing_ok)
 								   Anum_pg_subscription_suborigin);
 	sub->origin = TextDatumGetCString(datum);
 
+	/* Get last used id */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
+							tup,
+							Anum_pg_subscription_sublastusedid,
+							&isnull);
+	Assert(!isnull);
+	sub->lastusedid = DatumGetInt64(datum);
+
 	ReleaseSysCache(tup);
 
 	return sub;
@@ -199,6 +207,44 @@ DisableSubscription(Oid subid)
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update the last used replication slot ID for the given subscription.
+ */
+void
+UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid)
+{
+	Relation	rel;
+	bool		nulls[Natts_pg_subscription];
+	bool		replaces[Natts_pg_subscription];
+	Datum		values[Natts_pg_subscription];
+	HeapTuple	tup;
+
+	/* Look up the subscription in the catalog */
+	rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+	tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	/* Form a new tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	replaces[Anum_pg_subscription_sublastusedid - 1] = true;
+	values[Anum_pg_subscription_sublastusedid- 1] = Int64GetDatum(lastusedid);
+
+	/* Update the catalog */
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+	heap_freetuple(tup);
+
+	table_close(rel, NoLock);
+}
+
 /*
  * Convert text array to list of strings.
  *
@@ -228,7 +274,7 @@ textarray_to_stringlist(ArrayType *textarray)
  */
 void
 AddSubscriptionRelState(Oid subid, Oid relid, char state,
-						XLogRecPtr sublsn)
+						XLogRecPtr sublsn, char *relslotname, char *reloriginname)
 {
 	Relation	rel;
 	HeapTuple	tup;
@@ -257,6 +303,16 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
 	else
 		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -269,6 +325,60 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Internal function to modify columns for relation state update
+ */
+static void
+UpdateSubscriptionRelState_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char state,
+									XLogRecPtr sublsn)
+{
+	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
+	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+
+	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (sublsn != InvalidXLogRecPtr)
+		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
+	else
+		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication slot update
+ */
+static void
+UpdateSubscriptionRelReplicationSlot_internal(Datum *values,
+											bool *nulls,
+											bool *replaces,
+											char *relslotname)
+{
+	replaces[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication origin update
+ */
+static void
+UpdateSubscriptionRelOrigin_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char *reloriginname)
+{
+	replaces[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+}
+
 /*
  * Update the state of a subscription table.
  */
@@ -299,14 +409,56 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	memset(nulls, false, sizeof(nulls));
 	memset(replaces, false, sizeof(replaces));
 
-	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
-	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
 
-	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
-	if (sublsn != InvalidXLogRecPtr)
-		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
-	else
-		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update replication slot name, origin name and state of
+ * a subscription table in one transaction.
+ */
+void
+UpdateSubscriptionRel(Oid subid,
+					  Oid relid,
+					  char state,
+					  XLogRecPtr sublsn,
+					  char *relslotname,
+					  char *reloriginname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
+	UpdateSubscriptionRelOrigin_internal(values, nulls, replaces, reloriginname);
 
 	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
 							replaces);
@@ -318,6 +470,85 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Get origin name of subscription table.
+ *
+ * reloriginname's value has the replication origin name if the origin exists.
+ */
+void
+GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*originname;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srreloriginname, isnull);
+	if (!*isnull)
+	{
+		originname = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(reloriginname, originname, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
+/*
+ * Get replication slot name of subscription table.
+ *
+ * slotname's value has the replication slot name if the subscription has any.
+ */
+void
+GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*relrepslot;
+	bool		isnull;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srrelslotname, &isnull);
+	if (!isnull)
+	{
+		relrepslot = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(slotname, relrepslot, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
 /*
  * Get state of subscription table.
  *
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 54895ba929..9f5a295ddc 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -710,6 +710,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 		publicationListToArray(publications);
 	values[Anum_pg_subscription_suborigin - 1] =
 		CStringGetTextDatum(opts.origin);
+	values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(0);
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -773,7 +774,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 										 rv->schemaname, rv->relname);
 
 				AddSubscriptionRelState(subid, relid, table_state,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 			}
 
 			/*
@@ -864,6 +865,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
 	bool		must_use_password;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -943,7 +946,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				AddSubscriptionRelState(sub->oid, relid,
 										copy_data ? SUBREL_STATE_INIT : SUBREL_STATE_READY,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 				ereport(DEBUG1,
 						(errmsg_internal("table \"%s.%s\" added to subscription \"%s\"",
 										 rv->schemaname, rv->relname, sub->name)));
@@ -967,6 +970,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				char		state;
 				XLogRecPtr	statelsn;
+				char		slotname[NAMEDATALEN] = {0};
 
 				/*
 				 * Lock pg_subscription_rel with AccessExclusiveLock to
@@ -993,13 +997,36 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/*
+				 * Find the logical replication sync worker. If exists, store
+				 * the slot number for dropping associated replication slots
+				 * later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot_name);
+				}
+				else
+				{
+					/*
+					 * Sync of this relation might be failed in an earlier
+					 * attempt, but the replication slot might still exist.
+					 */
+					GetSubscriptionRelReplicationSlot(sub->oid, relid, slotname);
+					if (strlen(slotname) > 0)
+						sub_remove_slots = lappend(sub_remove_slots, slotname);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
 				 * tablesync origin.
 				 */
-				if (state != SUBREL_STATE_READY)
+				if (state != SUBREL_STATE_READY &&
+					state != SUBREL_STATE_SYNCDONE)
 				{
 					char		originname[NAMEDATALEN];
 
@@ -1027,31 +1054,24 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
-		 * the database operations we won't be able to rollback dropped slots.
+		 * Drop the replication slots associated with tablesync workers for
+		 * removed tables. This has to be at the end because otherwise if
+		 * there is an error while doing the database operations we won't be
+		 * able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			memcpy(syncslotname, lfirst(lc), sizeof(NAMEDATALEN));
+
+			/*
+			 * There is no certainty, maybe the slot does not exist yet. Also,
+			 * if we fail after removing some of the slots, next time, it will
+			 * again try to drop already dropped slots and fail. For these
+			 * reasons, we allow missing_ok = true for the drop.
+			 */
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1474,6 +1494,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	char	   *subname;
 	char	   *conninfo;
 	char	   *slotname;
+	int64		lastusedid;
 	List	   *subworkers;
 	ListCell   *lc;
 	char		originname[NAMEDATALEN];
@@ -1546,6 +1567,14 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	else
 		slotname = NULL;
 
+	/* Get the last used identifier by the subscription */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID, tup,
+							Anum_pg_subscription_sublastusedid, &isnull);
+	if (!isnull)
+		lastusedid = DatumGetInt64(datum);
+	else
+		lastusedid = 0;
+
 	/*
 	 * Since dropping a replication slot is not transactional, the replication
 	 * slot stays dropped even if the transaction rolls back.  So we cannot
@@ -1595,6 +1624,8 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	}
 	list_free(subworkers);
 
+	rstates = GetSubscriptionRelations(subid, true);
+
 	/*
 	 * Remove the no-longer-useful entry in the launcher's table of apply
 	 * worker start times.
@@ -1606,36 +1637,26 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	ApplyLauncherForgetWorkerStartTime(subid);
 
 	/*
-	 * Cleanup of tablesync replication origins.
-	 *
-	 * Any READY-state relations would already have dealt with clean-ups.
+	 * Cleanup of tablesync replication origins associated with the
+	 * subscription, if exists. Try to drop origins by creating all origin
+	 * names created for this subscription.
 	 *
 	 * Note that the state can't change because we have already stopped both
 	 * the apply and tablesync workers and they can't restart because of
 	 * exclusive lock on the subscription.
+	 *
+	 * XXX: This can be handled better instead of looping through all possible
 	 */
-	rstates = GetSubscriptionRelations(subid, true);
-	foreach(lc, rstates)
+	for (int64 i = 1; i <= lastusedid; i++)
 	{
-		SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-		Oid			relid = rstate->relid;
-
-		/* Only cleanup resources of tablesync workers */
-		if (!OidIsValid(relid))
-			continue;
+		char		originname_to_drop[NAMEDATALEN] = {0};
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * It is possible that the origin is not yet created for tablesync
-		 * worker so passing missing_ok = true. This can happen for the states
-		 * before SUBREL_STATE_FINISHEDCOPY.
-		 */
-		ReplicationOriginNameForLogicalRep(subid, relid, originname,
-										   sizeof(originname));
-		replorigin_drop_by_name(originname, true, false);
+		snprintf(originname_to_drop, sizeof(originname_to_drop), "pg_%u_%lld", subid, (long long) i);
+		/* missing_ok = true, since the origin might be already dropped. */
+		replorigin_drop_by_name(originname_to_drop, true, false);
 	}
 
+
 	/* Clean up dependencies */
 	deleteSharedDependencyRecordsFor(SubscriptionRelationId, subid, 0);
 
@@ -1688,39 +1709,17 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
+		List	   *slots = NULL;
 
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
 
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char	   *syncslotname = (char *) lfirst(lc);
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1743,6 +1742,71 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * Get the replication slot names associated with the subscription.
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok)
+{
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+					 " FROM pg_replication_slots"
+					 " WHERE slot_name LIKE 'pg_%i_sync_%%';",
+					 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+					errmsg("could not receive list of slots associated with the subscription %u, error: %s",
+					subid, res->err));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();
+		return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -2155,6 +2219,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -2165,18 +2230,20 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
-			char		syncslotname[NAMEDATALEN] = {0};
-
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
-				 syncslotname);
+				 worker->slot_name);
 		}
 	}
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 1d4e83c4c1..d2c70dffbc 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -440,7 +440,8 @@ pa_launch_parallel_worker(void)
 										MySubscription->name,
 										MyLogicalRepWorker->userid,
 										InvalidOid,
-										dsm_segment_handle(winfo->dsm_seg));
+										dsm_segment_handle(winfo->dsm_seg),
+										InvalidRepSlotId);
 
 	if (launched)
 	{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 72e5ef8a78..c84193319c 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -304,7 +304,7 @@ logicalrep_workers_find(Oid subid, bool only_running)
  */
 bool
 logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
-						 Oid relid, dsm_handle subworker_dsm)
+						 Oid relid, dsm_handle subworker_dsm, int64 slotid)
 {
 	BackgroundWorker bgw;
 	BackgroundWorkerHandle *bgw_handle;
@@ -430,6 +430,9 @@ retry:
 	worker->launch_time = now;
 	worker->in_use = true;
 	worker->generation++;
+	worker->created_slot = false;
+	worker->rep_slot_id = slotid;
+	worker->slot_name = (char *) palloc(NAMEDATALEN);
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -1183,7 +1186,8 @@ ApplyLauncherMain(Datum main_arg)
 				ApplyLauncherSetWorkerStartTime(sub->oid, now);
 				logicalrep_worker_launch(sub->dbid, sub->oid, sub->name,
 										 sub->owner, InvalidOid,
-										 DSM_HANDLE_INVALID);
+										 DSM_HANDLE_INVALID,
+										 InvalidRepSlotId);
 			}
 			else
 			{
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index f042d9ae00..830fa4696c 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -328,40 +328,29 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 								   MyLogicalRepWorker->relid,
 								   MyLogicalRepWorker->relstate,
 								   MyLogicalRepWorker->relstate_lsn);
+		CommitTransactionCommand();
 
 		/*
-		 * End streaming so that LogRepWorkerWalRcvConn can be used to drop
-		 * the slot.
-		 */
-		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
-
-		/*
-		 * Cleanup the tablesync slot.
+		 * Cleanup the tablesync slot. If the slot name used by this worker is
+		 * different from the default slot name for the worker, this means the
+		 * current table had started to being synchronized by another worker
+		 * and replication slot. And this worker is reusing a replication slot
+		 * from a previous attempt. We do not need that replication slot
+		 * anymore.
 		 *
 		 * This has to be done after updating the state because otherwise if
 		 * there is an error while doing the database operations we won't be
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
+										MyLogicalRepWorker->rep_slot_id,
 										syncslotname,
 										sizeof(syncslotname));
 
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
-		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
-
-		CommitTransactionCommand();
-		pgstat_report_stat(false);
-
-		/*
-		 * Start a new transaction to clean up the tablesync origin tracking.
-		 * This transaction will be ended within the finish_sync_worker().
-		 * Now, even, if we fail to remove this here, the apply worker will
-		 * ensure to clean it up afterward.
+		 * We are safe to drop the replication tracking origin after this
+		 * point. Now, even, if we fail to remove this here, the apply worker
+		 * will ensure to clean it up afterward.
 		 *
 		 * We need to do this after the table state is set to SYNCDONE.
 		 * Otherwise, if an error occurs while performing the database
@@ -370,32 +359,73 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 * have been cleared before restart. So, the restarted worker will use
 		 * invalid replication progress state resulting in replay of
 		 * transactions that have already been applied.
+		 *
+		 * Firstly reset the origin session to remove the ownership of the
+		 * slot. This is needed to allow the origin to be dropped or reused
+		 * later.
 		 */
+		replorigin_session_reset();
+		replorigin_session_origin = InvalidRepOriginId;
+		replorigin_session_origin_lsn = InvalidXLogRecPtr;
+		replorigin_session_origin_timestamp = 0;
+
 		StartTransactionCommand();
+		if (MyLogicalRepWorker->slot_name && strcmp(syncslotname, MyLogicalRepWorker->slot_name) != 0)
+		{
+			/*
+			 * End streaming so that LogRepWorkerWalRcvConn can be used to
+			 * drop the slot.
+			 */
+			walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
 
-		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
+			ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+											   MyLogicalRepWorker->relid,
+											   originname,
+											   sizeof(originname));
+
+			/*
+			 * Drop replication origin
+			 *
+			 * There is a chance that the user is concurrently performing refresh
+			 * for the subscription where we remove the table state and its origin
+			 * or the apply worker would have removed this origin. So passing
+			 * missing_ok = true.
+			 */
+			replorigin_drop_by_name(originname, true, false);
+		}
 
 		/*
-		 * Resetting the origin session removes the ownership of the slot.
-		 * This is needed to allow the origin to be dropped.
+		 * We are safe to remove persisted replication slot and origin data,
+		 * since it's already in SYNCDONE state. They will not be needed
+		 * anymore.
 		 */
-		replorigin_session_reset();
-		replorigin_session_origin = InvalidRepOriginId;
-		replorigin_session_origin_lsn = InvalidXLogRecPtr;
-		replorigin_session_origin_timestamp = 0;
+		UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+							  MyLogicalRepWorker->relid,
+							  MyLogicalRepWorker->relstate,
+							  MyLogicalRepWorker->relstate_lsn,
+							  NULL,
+							  NULL);
+		ereport(DEBUG2,
+				(errmsg("process_syncing_tables_for_sync: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+						"NULL",
+						"NULL",
+						MyLogicalRepWorker->relstate,
+						MyLogicalRepWorker->relid,
+						MyLogicalRepWorker->subid)));
+		CommitTransactionCommand();
+		pgstat_report_stat(false);
 
 		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * There is a chance that the user is concurrently performing refresh
-		 * for the subscription where we remove the table state and its origin
-		 * or the apply worker would have removed this origin. So passing
-		 * missing_ok = true.
+		 * This should return the default origin name for the worker. Even if
+		 * the worker used a different origin for this table, it should be
+		 * dropped and removed from the catalog so far.
 		 */
-		replorigin_drop_by_name(originname, true, false);
+		StartTransactionCommand();
+		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+										   MyLogicalRepWorker->relid,
+										   originname,
+										   sizeof(originname));
 
 		/* Sync worker has completed synchronization of the current table. */
 		MyLogicalRepWorker->is_sync_completed = true;
@@ -492,6 +522,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 			if (current_lsn >= rstate->lsn)
 			{
 				char		originname[NAMEDATALEN];
+				bool		is_origin_null = true;
 
 				rstate->state = SUBREL_STATE_READY;
 				rstate->lsn = current_lsn;
@@ -512,18 +543,31 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				 * error while dropping we won't restart it to drop the
 				 * origin. So passing missing_ok = true.
 				 */
-				ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-												   rstate->relid,
-												   originname,
-												   sizeof(originname));
-				replorigin_drop_by_name(originname, true, false);
+				GetSubscriptionRelOrigin(MyLogicalRepWorker->subid,
+										 rstate->relid, originname,
+										 &is_origin_null);
+
+				if (!is_origin_null)
+				{
+					replorigin_drop_by_name(originname, true, false);
+				}
 
 				/*
 				 * Update the state to READY only after the origin cleanup.
 				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-										   rstate->relid, rstate->state,
-										   rstate->lsn);
+				UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+									  rstate->relid,
+									  rstate->state,
+									  rstate->lsn,
+									  NULL,
+									  NULL);
+				ereport(DEBUG2,
+					(errmsg("process_syncing_tables_for_apply: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+							"NULL", "NULL", rstate->state,
+							rstate->relid, MyLogicalRepWorker->subid)));
+
+				CommitTransactionCommand();
+				started_tx = false;
 			}
 		}
 		else
@@ -612,12 +656,25 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						TimestampDifferenceExceeds(hentry->last_start_time, now,
 												   wal_retrieve_retry_interval))
 					{
+						if (IsTransactionState())
+							CommitTransactionCommand();
+						StartTransactionCommand();
+						started_tx = true;
+
+						MySubscription->lastusedid++;
+						UpdateSubscriptionLastSlotId(MyLogicalRepWorker->subid,
+													 MySubscription->lastusedid);
+						ereport(DEBUG2,
+								(errmsg("process_syncing_tables_for_apply: incremented lastusedid to %lld for subscription %u",
+										(long long) MySubscription->lastusedid, MySubscription->oid)));
+
 						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
 												 MySubscription->oid,
 												 MySubscription->name,
 												 MyLogicalRepWorker->userid,
 												 rstate->relid,
-												 DSM_HANDLE_INVALID);
+												 DSM_HANDLE_INVALID,
+												 MySubscription->lastusedid);
 						hentry->last_start_time = now;
 					}
 				}
@@ -1240,8 +1297,8 @@ copy_table(Relation rel)
  * The name must not exceed NAMEDATALEN - 1 because of remote node constraints
  * on slot name length. We append system_identifier to avoid slot_name
  * collision with subscriptions in other clusters. With the current scheme
- * pg_%u_sync_%u_UINT64_FORMAT (3 + 10 + 6 + 10 + 20 + '\0'), the maximum
- * length of slot_name will be 50.
+ * pg_%u_sync_%lu_UINT64_FORMAT (3 + 10 + 6 + 20 + 20 + '\0'), the maximum
+ * length of slot_name will be 45.
  *
  * The returned slot name is stored in the supplied buffer (syncslotname) with
  * the given size.
@@ -1252,11 +1309,11 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
+ReplicationSlotNameForTablesync(Oid suboid, int64 slotid,
 								char *syncslotname, Size szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%lld_" UINT64_FORMAT, suboid,
+			(long long) slotid, GetSystemIdentifier());
 }
 
 /*
@@ -1282,6 +1339,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 	UserContext ucxt;
 	bool		must_use_password;
 	bool		run_as_owner;
+	char	   *prev_slotname;
 
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
@@ -1316,7 +1374,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
+									MyLogicalRepWorker->rep_slot_id,
 									slotname,
 									NAMEDATALEN);
 
@@ -1355,12 +1413,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC ||
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY);
 
+	/*
+	 * See if tablesync of the current relation has been started with another
+	 * replication slot.
+	 *
+	 * Read previous slot name from the catalog, if exists.
+	 */
+	prev_slotname = (char *) palloc(NAMEDATALEN);
+	StartTransactionCommand();
+	GetSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+									  MyLogicalRepWorker->relid,
+									  prev_slotname);
+
 	/* Assign the origin tracking record name. */
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   sizeof(originname));
 
+	CommitTransactionCommand();
+
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC)
 	{
 		/*
@@ -1374,10 +1446,53 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 		 * breakdown then it wouldn't have succeeded so trying it next time
 		 * seems like a better bet.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, slotname, true);
+		if (strlen(prev_slotname) > 0)
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);
+
+			StartTransactionCommand();
+			/* Replication origin might still exist. Try to drop */
+			replorigin_drop_by_name(originname, true, false);
+
+			/*
+			 * Remove replication slot and origin name from the relation's
+			 * catalog record
+			 */
+			UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+								  MyLogicalRepWorker->relid,
+								  MyLogicalRepWorker->relstate,
+								  MyLogicalRepWorker->relstate_lsn,
+								  NULL,
+								  NULL);
+			CommitTransactionCommand();
+			ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+						"NULL", "NULL", MyLogicalRepWorker->relstate,
+						MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+		}
 	}
 	else if (MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY)
 	{
+		/*
+		 * At this point, the table that is currently being synchronized
+		 * should have its replication slot name filled in the catalog. The
+		 * tablesync process was started with another sync worker and
+		 * replication slot. We need to continue using the same replication
+		 * slot in this worker too.
+		 */
+		if (strlen(prev_slotname) == 0)
+		{
+			elog(ERROR, "Replication slot could not be found for subscription %u, relation %u",
+				 MyLogicalRepWorker->subid,
+				 MyLogicalRepWorker->relid);
+		}
+
+		/*
+		 * Proceed with the correct replication slot. Use previously created
+		 * replication slot to sync this table.
+		 */
+		memcpy(slotname, prev_slotname, NAMEDATALEN);
+
 		/*
 		 * The COPY phase was previously done, but tablesync then crashed
 		 * before it was able to finish normally.
@@ -1397,7 +1512,9 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 
 		goto copy_table_done;
 	}
+	pfree(prev_slotname);
 
+	/* Preparing for table copy operation */
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 	MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
 	MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
@@ -1405,11 +1522,31 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   MyLogicalRepWorker->relstate,
-							   MyLogicalRepWorker->relstate_lsn);
+
+	/*
+	 * Refresh the originname in case of having non-existing origin
+	 * from previous failed sync attempts.
+	 * If that's the case, it should be removed from the catalog so far.
+	 * Then, we can continue by reusing the origin created by the current
+	 * worker instead of .
+	 */
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									MyLogicalRepWorker->relid,
+									originname,
+									sizeof(originname));
+
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  MyLogicalRepWorker->relstate,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					slotname, originname, MyLogicalRepWorker->relstate,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+
 	pgstat_report_stat(true);
 
 	StartTransactionCommand();
@@ -1437,48 +1574,96 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 						res->err)));
 	walrcv_clear_result(res);
 
+	originid = replorigin_by_name(originname, true);
+
 	/*
 	 * Create a new permanent logical decoding slot. This slot will be used
 	 * for the catchup phase after COPY is done, so tell it to use the
 	 * snapshot to make the final data consistent.
+	 *
+	 * Replication slot will only be created if either this is the first run
+	 * of the worker or we're not using a previous replication slot.
 	 */
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-
-	/*
-	 * Setup replication origin tracking. The purpose of doing this before the
-	 * copy is to avoid doing the copy again due to any error in setting up
-	 * origin tracking.
-	 */
-	originid = replorigin_by_name(originname, true);
-	if (!OidIsValid(originid))
+	if (!MyLogicalRepWorker->created_slot)
 	{
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						   slotname, false /* permanent */ , false /* two_phase */ ,
+						   CRS_USE_SNAPSHOT, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: created replication slot %s for subscription %u",
+						slotname, MyLogicalRepWorker->subid)));
+
 		/*
-		 * Origin tracking does not exist, so create it now.
-		 *
-		 * Then advance to the LSN got from walrcv_create_slot. This is WAL
-		 * logged for the purpose of recovery. Locks are to prevent the
-		 * replication origin from vanishing while advancing.
+		 * Remember that we created the slot so that we will not try to create
+		 * it again.
 		 */
-		originid = replorigin_create(originname);
-
-		LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
-		replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
-						   true /* go backward */ , true /* WAL log */ );
-		UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->created_slot = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
+		/*
+		 * Setup replication origin tracking. The purpose of doing this before
+		 * the copy is to avoid doing the copy again due to any error in
+		 * setting up origin tracking.
+		 */
+		if (!OidIsValid(originid))
+		{
+			/*
+			 * Origin tracking does not exist, so create it now.
+			 */
+			originid = replorigin_create(originname);
+		}
+		else
+		{
+			/*
+			 * At this point, there shouldn't be any existing replication
+			 * origin with the same name.
+			 */
+			ereport(ERROR,
+					(errcode(ERRCODE_DUPLICATE_OBJECT),
+					 errmsg("replication origin \"%s\" already exists",
+							originname)));
+		}
 	}
 	else
 	{
-		ereport(ERROR,
-				(errcode(ERRCODE_DUPLICATE_OBJECT),
-				 errmsg("replication origin \"%s\" already exists",
-						originname)));
+		/*
+		 * Do not create a new replication slot, reuse the existing one
+		 * instead. Use a new snapshot for the replication slot to ensure that
+		 * tablesync and apply proceses are consistent with each other.
+		 */
+		WalRcvStreamOptions options;
+		int			server_version;
+
+		server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+		options.proto.logical.proto_version =
+			server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+			server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+			LOGICALREP_PROTO_VERSION_NUM;
+		options.proto.logical.publication_names = MySubscription->publications;
+
+		walrcv_slot_snapshot(LogRepWorkerWalRcvConn, slotname, &options, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: reusing replication slot %s for relation %u in subscription %u",
+						slotname, MyLogicalRepWorker->relid,
+						MyLogicalRepWorker->subid)));
 	}
 
+	/*
+	 * Advance to the LSN got from walrcv_create_slot or walrcv_slot_snapshot.
+	 * This is WAL logged for the purpose of recovery. Locks are to prevent
+	 * the replication origin from vanishing while advancing.
+	 *
+	 * Then setup replication origin tracking.
+	 */
+	LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+	replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
+					   true /* go backward */ , true /* WAL log */ );
+	UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+
 	/*
 	 * Make sure that the copy command runs as the table owner, unless the
 	 * user has opted out of that behaviour.
@@ -1537,12 +1722,18 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 	 * Update the persisted state to indicate the COPY phase is done; make it
 	 * visible to others.
 	 */
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   SUBREL_STATE_FINISHEDCOPY,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  SUBREL_STATE_FINISHEDCOPY,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					originname, slotname, SUBREL_STATE_FINISHEDCOPY,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
 
 copy_table_done:
 
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 537cd33a30..8966e02b77 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -466,8 +466,16 @@ ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 {
 	if (OidIsValid(relid))
 	{
-		/* Replication origin name for tablesync workers. */
-		snprintf(originname, szoriginname, "pg_%u_%u", suboid, relid);
+		bool		is_null = true;
+
+		/*
+		 * Replication origin name for tablesync workers. First, look into the
+		 * catalog. If originname does not exist, then use the default name.
+		 */
+		GetSubscriptionRelOrigin(suboid, relid,
+								 originname, &is_null);
+		if (is_null)
+			snprintf(originname, szoriginname, "pg_%u_%lld", suboid, (long long) MyLogicalRepWorker->rep_slot_id);
 	}
 	else
 	{
@@ -4504,6 +4512,9 @@ start_table_sync(XLogRecPtr *origin_startpos,
 
 	/* allocate slot name in long-lived context */
 	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+
+	/* Keep the replication slot name used for this sync. */
+	MyLogicalRepWorker->slot_name = *myslotname;
 	pfree(syncslotname);
 }
 
@@ -4570,10 +4581,12 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 	/* Start table synchronization. */
 	start_table_sync(origin_startpos, &slotname, worker_slot);
 
+	StartTransactionCommand();
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   originname_size);
+	CommitTransactionCommand();
 
 	set_apply_error_context_origin(originname);
 
@@ -4613,11 +4626,10 @@ run_apply_worker(WalRcvStreamOptions *options,
 				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
 					errmsg("subscription has no replication slot set")));
 
-	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-									   originname, originname_size);
-
 	/* Setup replication origin tracking. */
 	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
 	originid = replorigin_by_name(originname, true);
 	if (!OidIsValid(originid))
 		originid = replorigin_create(originname);
@@ -4934,7 +4946,32 @@ TablesyncWorkerMain(Datum main_arg)
 			}
 
 			if (!is_table_found)
+			{
+				TimeLineID	tli;
+
+				/*
+				 * It is important to give an error if we are unable to drop the
+				 * slot, otherwise, it won't be dropped till the corresponding
+				 * subscription is dropped. So passing missing_ok = false.
+				 */
+				if (MyLogicalRepWorker->created_slot)
+				{
+					walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+					ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
+				}
+
+				/*
+				 * Drop replication origin before exiting.
+				 *
+				 * There is a chance that the user is concurrently performing refresh
+				 * for the subscription where we remove the table state and its origin
+				 * or the apply worker would have removed this origin. So passing
+				 * missing_ok = true.
+				 */
+				replorigin_drop_by_name(originname, true, false);
+
 				break;
+			}
 		}
 	}
 
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 1d40eebc78..7e13f59847 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -108,6 +108,9 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
 
 	/* Only publish data originating from the specified origin */
 	text		suborigin BKI_DEFAULT(LOGICALREP_ORIGIN_ANY);
+
+	/* The last used ID to create a replication slot for tablesync */
+	int64		sublastusedid BKI_DEFAULT(0);
 #endif
 } FormData_pg_subscription;
 
@@ -144,6 +147,8 @@ typedef struct Subscription
 	List	   *publications;	/* List of publication names to subscribe to */
 	char	   *origin;			/* Only publish data originating from the
 								 * specified origin */
+	int64		lastusedid;		/* Last used unique ID to create replication
+								 * slots in tablesync */
 } Subscription;
 
 /* Disallow streaming in-progress transactions. */
@@ -164,6 +169,7 @@ typedef struct Subscription
 extern Subscription *GetSubscription(Oid subid, bool missing_ok);
 extern void FreeSubscription(Subscription *sub);
 extern void DisableSubscription(Oid subid);
+extern void UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid);
 
 extern int	CountDBSubscriptions(Oid dbid);
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 60a2bcca23..185164d75e 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -44,6 +44,12 @@ CATALOG(pg_subscription_rel,6102,SubscriptionRelRelationId)
 											 * used for synchronization
 											 * coordination, or NULL if not
 											 * valid */
+	NameData	srrelslotname BKI_FORCE_NULL;	/* name of the replication
+												 * slot for relation in
+												 * subscription */
+	NameData	srreloriginname BKI_FORCE_NULL; /* origin name for relation in
+												 * subscription */
+
 #endif
 } FormData_pg_subscription_rel;
 
@@ -81,10 +87,16 @@ typedef struct SubscriptionRelState
 } SubscriptionRelState;
 
 extern void AddSubscriptionRelState(Oid subid, Oid relid, char state,
-									XLogRecPtr sublsn);
+									XLogRecPtr sublsn, char *relslotname, char *reloriginname);
 extern void UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 									   XLogRecPtr sublsn);
+extern void UpdateSubscriptionRel(Oid subid, Oid relid, char state,
+								  XLogRecPtr sublsn, char *relslotname, char *reloriginname);
+
 extern char GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn);
+extern void GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname);
+extern void GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull);
+
 extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 
 extern bool HasSubscriptionRelations(Oid subid);
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index a8a89dc784..31b2c41893 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -237,8 +237,9 @@ extern bool InvalidateObsoleteReplicationSlots(ReplicationSlotInvalidationCause
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
 extern bool ReplicationSlotName(int index, Name name);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, Size szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, int64 slotid, char *syncslotname, Size szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List *GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index af6fd339f7..97c7d2aa50 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -35,6 +35,23 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/*
+	 * Indicates if the sync worker created a replication slot for itself
+	 * in any point of its lifetime.
+	 * False means that the worker has not created a slot yet, and has been
+	 * reusing replication slots created by other workers so far.
+	 */
+	bool		created_slot;
+
+	/*
+	 * Unique identifier for replication slot to be created by tablesnync
+	 * workers, if needed.
+	 */
+	int64		rep_slot_id;
+
+	/* Replication slot name used by the worker. */
+	char	   *slot_name;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
@@ -239,7 +256,8 @@ extern LogicalRepWorker *logicalrep_worker_find(Oid subid, Oid relid,
 extern List *logicalrep_workers_find(Oid subid, bool only_running);
 extern bool logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname,
 									 Oid userid, Oid relid,
-									 dsm_handle subworker_dsm);
+									 dsm_handle subworker_dsm,
+									 int64 slotid);
 extern void logicalrep_worker_stop(Oid subid, Oid relid);
 extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
 extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
@@ -336,4 +354,7 @@ am_parallel_apply_worker(void)
 	return isParallelApplyWorker(MyLogicalRepWorker);
 }
 
+/* Invalid identifier to be used for naming replication slots */
+#define InvalidRepSlotId	0
+
 #endif							/* WORKER_INTERNAL_H */
-- 
2.25.1

#83Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Hayato Kuroda (Fujitsu) (#81)
1 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hayato Kuroda (Fujitsu) <kuroda.hayato@fujitsu.com>, 4 Tem 2023 Sal,
08:42 tarihinde şunu yazdı:

But in the later patch the tablesync worker tries to reuse the slot during the
synchronization, so in this case the application_name should be same as

slotname.

Fair enough. I am slightly afraid that if we can't show the benefits
with later patches then we may need to drop them but at this stage I
feel we need to investigate why those are not helping?

Agreed. Now I'm planning to do performance testing independently. We can discuss
based on that or Melih's one.

Here I attached what I use for performance testing of this patch.

I only benchmarked the patch set with reusing connections very roughly
so far. But seems like it improves quite significantly. For example,
it took 611 ms to sync 100 empty tables, it was 1782 ms without
reusing connections.
First 3 patches from the set actually bring a good amount of
improvement, but not sure about the later patches yet.

Amit Kapila <amit.kapila16@gmail.com>, 3 Tem 2023 Pzt, 08:59 tarihinde
şunu yazdı:

On thinking about this, I think the primary benefit we were expecting
by saving network round trips for slot drop/create but now that we
anyway need an extra round trip to establish a snapshot, so such a
benefit was not visible. This is just a theory so we should validate
it. The another idea as discussed before [1] could be to try copying
multiple tables in a single transaction. Now, keeping a transaction
open for a longer time could have side-effects on the publisher node.
So, we probably need to ensure that we don't perform multiple large
syncs and even for smaller tables (and later sequences) perform it
only for some threshold number of tables which we can figure out by
some tests. Also, the other safety-check could be that anytime we need
to perform streaming (sync with apply worker), we won't copy more
tables in same transaction.

Thoughts?

Yeah, maybe going to the publisher for creating a slot or only a
snapshot does not really make enough difference. I was hoping that
creating only snapshot by an existing replication slot would help the
performance. I guess I was either wrong or am missing something in the
implementation.

The tricky bit with keeping a long transaction to copy multiple tables
is deciding how many tables one transaction can copy.

Thanks,
--
Melih Mutlu
Microsoft

Attachments:

log_rep_testing.txttext/plain; charset=US-ASCII; name=log_rep_testing.txtDownload
--- on publisher
SELECT 'CREATE TABLE manytables_'||i||'(i int);' FROM generate_series(1, 100) g(i) \gexec
SELECT pg_create_logical_replication_slot('mysub_slot', 'pgoutput');

--- on subscriber
SELECT 'CREATE TABLE manytables_'||i||'(i int);' FROM generate_series(1, 100) g(i) \gexec

CREATE OR REPLACE PROCEDURE log_rep_test(max INTEGER) AS $$
DECLARE
    counter INTEGER := 1;
    total_duration INTERVAL := '0';
    avg_duration FLOAT := 0.0;
    start_time TIMESTAMP;
    end_time TIMESTAMP;
BEGIN
    WHILE counter <= max LOOP
        
        EXECUTE 'DROP SUBSCRIPTION IF EXISTS mysub;';

        start_time := clock_timestamp();
        EXECUTE 'CREATE SUBSCRIPTION mysub CONNECTION ''dbname=postgres port=5432'' PUBLICATION mypub WITH (create_slot=false, slot_name=''mysub_slot'');';
        COMMIT;

        WHILE EXISTS (SELECT 1 FROM pg_subscription_rel WHERE srsubstate != 'r') LOOP
            COMMIT;
        END LOOP;

        end_time := clock_timestamp();


        EXECUTE 'ALTER SUBSCRIPTION mysub DISABLE;';
        EXECUTE 'ALTER SUBSCRIPTION mysub SET (slot_name = none);';

        
        total_duration := total_duration + (end_time - start_time);
        
        counter := counter + 1;
    END LOOP;
    
    IF max > 0 THEN
        avg_duration := EXTRACT(EPOCH FROM total_duration) / max * 1000;
    END IF;
    
    RAISE NOTICE '%', avg_duration;
END;
$$ LANGUAGE plpgsql;


call log_rep_test(5);
#84Amit Kapila
amit.kapila16@gmail.com
In reply to: Melih Mutlu (#83)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Wed, Jul 5, 2023 at 1:48 AM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hayato Kuroda (Fujitsu) <kuroda.hayato@fujitsu.com>, 4 Tem 2023 Sal,
08:42 tarihinde şunu yazdı:

But in the later patch the tablesync worker tries to reuse the slot during the
synchronization, so in this case the application_name should be same as

slotname.

Fair enough. I am slightly afraid that if we can't show the benefits
with later patches then we may need to drop them but at this stage I
feel we need to investigate why those are not helping?

Agreed. Now I'm planning to do performance testing independently. We can discuss
based on that or Melih's one.

Here I attached what I use for performance testing of this patch.

I only benchmarked the patch set with reusing connections very roughly
so far. But seems like it improves quite significantly. For example,
it took 611 ms to sync 100 empty tables, it was 1782 ms without
reusing connections.
First 3 patches from the set actually bring a good amount of
improvement, but not sure about the later patches yet.

I suggest then we should focus first on those 3, get them committed
and then look at the remaining.

Amit Kapila <amit.kapila16@gmail.com>, 3 Tem 2023 Pzt, 08:59 tarihinde
şunu yazdı:

On thinking about this, I think the primary benefit we were expecting
by saving network round trips for slot drop/create but now that we
anyway need an extra round trip to establish a snapshot, so such a
benefit was not visible. This is just a theory so we should validate
it. The another idea as discussed before [1] could be to try copying
multiple tables in a single transaction. Now, keeping a transaction
open for a longer time could have side-effects on the publisher node.
So, we probably need to ensure that we don't perform multiple large
syncs and even for smaller tables (and later sequences) perform it
only for some threshold number of tables which we can figure out by
some tests. Also, the other safety-check could be that anytime we need
to perform streaming (sync with apply worker), we won't copy more
tables in same transaction.

Thoughts?

Yeah, maybe going to the publisher for creating a slot or only a
snapshot does not really make enough difference. I was hoping that
creating only snapshot by an existing replication slot would help the
performance. I guess I was either wrong or am missing something in the
implementation.

The tricky bit with keeping a long transaction to copy multiple tables
is deciding how many tables one transaction can copy.

Yeah, I was thinking that we should not allow copying some threshold
data in one transaction. After every copy, we will check the size of
the table and add it to the previously copied table size in the same
transaction. Once the size crosses a certain threshold, we will end
the transaction. This may not be a very good scheme but I think it
this helps then it would be much simpler than creating-only-snapshot
approach.

--
With Regards,
Amit Kapila.

#85Hayato Kuroda (Fujitsu)
kuroda.hayato@fujitsu.com
In reply to: Melih Mutlu (#82)
7 attachment(s)
RE: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Dear Melih,

Thanks for the 0003 patch. But it did not work for me. Can you create
a subscription successfully with patch 0003 applied?
I get the following error: " ERROR: table copy could not start
transaction on publisher: another command is already in progress".

You got the ERROR when all the patches (0001-0005) were applied, right?
I have focused on 0001 and 0002 only, so I missed something.
If it was not correct, please attach the logfile and test script what you did.

As you might know, the error is output when the worker executs walrcv_endstreaming()
before doing walrcv_startstreaming().

I think streaming needs to be ended before moving to another table. So
I changed the patch a little bit

Your modification seemed not correct. I applied only first three patches (0001-0003), and
executed attached script. Then I got following error on subscriber (attached as N2.log):

ERROR: could not send end-of-streaming message to primary: no COPY in progress

IIUC the tablesync worker has been already stopped streaming without your modification.
Please see process_syncing_tables_for_sync():

```
if (MyLogicalRepWorker->relstate == SUBREL_STATE_CATCHUP &&
current_lsn >= MyLogicalRepWorker->relstate_lsn)
{
TimeLineID tli;
char syncslotname[NAMEDATALEN] = {0};
char originname[NAMEDATALEN] = {0};

MyLogicalRepWorker->relstate = SUBREL_STATE_SYNCDONE;
...
/*
* End streaming so that LogRepWorkerWalRcvConn can be used to drop
* the slot.
*/
walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
```

This means that following changes should not be in the 0003, should be at 0005.
PSA fixed patches.

```
+	/*
+	 * If it's already connected to the publisher, end streaming before using
+	 * the same connection for another iteration
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		TimeLineID tli;
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	}
```

Besides, cfbot could not apply your patch set [1]http://cfbot.cputube.org/patch_43_3784.log. According to the log, the
bot tried to apply 0004 and 0005 first and got error. IIUC you should assign
same version number within the same mail, like v16-0001, v16-0002,....

[1]: http://cfbot.cputube.org/patch_43_3784.log

Best Regards,
Hayato Kuroda
FUJITSU LIMITED

Attachments:

N2.logapplication/octet-stream; name=N2.logDownload
test.shapplication/octet-stream; name=test.shDownload
v16-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchapplication/octet-stream; name=v16-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchDownload
From 6fc111ce99572adddca734810470b7de02a54882 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Mon, 5 Jun 2023 15:04:41 +0300
Subject: [PATCH v16 1/5] Refactor to split Apply and Tablesync Workers

Both apply and tablesync workers were using ApplyWorkerMain() as entry
point. As the name implies, ApplyWorkerMain() should be considered as
the main function for apply workers. Tablesync worker's path was hidden
and does not have enough in common to share the same main function with
apply worker.

Also; most of the code shared by both worker types are already combined
in LogicalRepApplyLoop(). There is no need to combine the rest in
ApplyWorkerMain() anymore.

This commit introduces TablesyncWorkerMain() as a new entry point for
tablesync workers and separates both type of workers from each other.
This aims to increase code readability and help to maintain logical
replication workers separately.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/postmaster/bgworker.c             |   3 +
 .../replication/logical/applyparallelworker.c |   2 +-
 src/backend/replication/logical/launcher.c    |  32 +-
 src/backend/replication/logical/tablesync.c   |   2 +-
 src/backend/replication/logical/worker.c      | 388 +++++++++++-------
 src/include/replication/logicalworker.h       |   1 +
 src/include/replication/worker_internal.h     |   4 +-
 7 files changed, 270 insertions(+), 162 deletions(-)

diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 5b4bd71694..505e38376c 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -131,6 +131,9 @@ static const struct
 	},
 	{
 		"ParallelApplyWorkerMain", ParallelApplyWorkerMain
+	},
+	{
+		"TablesyncWorkerMain", TablesyncWorkerMain
 	}
 };
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 6fb96148f4..1d4e83c4c1 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -942,7 +942,7 @@ ParallelApplyWorkerMain(Datum main_arg)
 	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
 		MyLogicalRepWorker->reply_time = 0;
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 542af7d863..e231fa7f95 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -459,24 +459,30 @@ retry:
 	snprintf(bgw.bgw_library_name, MAXPGPATH, "postgres");
 
 	if (is_parallel_apply_worker)
+	{
 		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ParallelApplyWorkerMain");
-	else
-		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
-
-	if (OidIsValid(relid))
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication worker for subscription %u sync %u", subid, relid);
-	else if (is_parallel_apply_worker)
+				 "logical replication parallel apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
+	}
+	else if (OidIsValid(relid))
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "TablesyncWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication parallel apply worker for subscription %u", subid);
+				 "logical replication tablesync worker for subscription %u sync %u",
+				 subid,
+				 relid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication tablesync worker");
+	}
 	else
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication apply worker for subscription %u", subid);
-
-	if (is_parallel_apply_worker)
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
-	else
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication worker");
+				 "logical replication apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication apply worker");
+	}
 
 	bgw.bgw_restart_time = BGW_NEVER_RESTART;
 	bgw.bgw_notify_pid = MyProcPid;
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 6d461654ab..8125bbd170 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -131,7 +131,7 @@ static StringInfo copybuf = NULL;
 /*
  * Exit routine for synchronization worker.
  */
-static void
+void
 pg_attribute_noreturn()
 finish_sync_worker(void)
 {
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 0ee764d68f..96ddad356b 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -392,6 +392,9 @@ static void stream_open_file(Oid subid, TransactionId xid,
 static void stream_write_change(char action, StringInfo s);
 static void stream_open_and_write_change(TransactionId xid, char action, StringInfo s);
 static void stream_close_file(void);
+static void set_stream_options(WalRcvStreamOptions *options,
+							   char *slotname,
+							   XLogRecPtr *origin_startpos);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
@@ -4330,6 +4333,72 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+ /* set_stream_options
+  * 	Set logical replication streaming options.
+  *
+  * This function sets streaming options including replication slot name and
+  * origin start position. Workers need these options for logical replication.
+  */
+static void
+set_stream_options(WalRcvStreamOptions *options,
+				   char *slotname,
+				   XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+		options->proto.logical.twophase = true;
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4433,7 +4502,8 @@ start_apply(XLogRecPtr origin_startpos)
 			 * idle state.
 			 */
 			AbortOutOfAnyTransaction();
-			pgstat_report_subscription_error(MySubscription->oid, !am_tablesync_worker());
+			pgstat_report_subscription_error(MySubscription->oid,
+											 !am_tablesync_worker());
 
 			PG_RE_THROW();
 		}
@@ -4442,13 +4512,133 @@ start_apply(XLogRecPtr origin_startpos)
 }
 
 /*
- * Common initialization for leader apply worker and parallel apply worker.
+ * Runs the tablesync worker.
+ * It starts syncing tables. After a successful sync, sets streaming options
+ * and starts streaming to catchup.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Start table synchronization. */
+	start_table_sync(origin_startpos, &slotname);
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   originname_size);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	/* Start applying changes to catchup. */
+	start_apply(*origin_startpos);
+}
+
+/*
+ * Runs the leader apply worker.
+ * It sets up replication origin, streaming options and then starts streaming.
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
+{
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+	bool		must_use_password;
+
+	slotname = MySubscription->slotname;
+
+	/*
+	 * This shouldn't happen if the subscription is enabled, but guard
+	 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+	 * crash if slot is NULL.)
+	 */
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+
+	/* Is the use of a password mandatory? */
+	must_use_password = MySubscription->passwordrequired &&
+		!superuser_arg(MySubscription->owner);
+
+	/* Note that the superuser_arg call can access the DB */
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											must_use_password,
+											MySubscription->name, &err);
+
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+	 * We don't really use the output identify_system for anything but it
+	 * does some initializations on the upstream so let's still call it.
+	 */
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+	{
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
+	}
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+
+	/* Run the main loop. */
+	start_apply(*origin_startpos);
+}
+
+/*
+ * Common initialization for logical replication workers; leader apply worker,
+ * parallel apply worker and tablesync worker.
  *
  * Initialize the database connection, in-memory subscription and necessary
  * config options.
  */
 void
-InitializeApplyWorker(void)
+InitializeLogRepWorker(void)
 {
 	MemoryContext oldctx;
 
@@ -4512,7 +4702,8 @@ InitializeApplyWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has started",
+				(errmsg("%s for subscription \"%s\", table \"%s\" has started",
+						get_worker_name(),
 						MySubscription->name,
 						get_rel_name(MyLogicalRepWorker->relid))));
 	else
@@ -4533,7 +4724,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	InitializingApplyWorker = true;
 
@@ -4557,7 +4747,7 @@ ApplyWorkerMain(Datum main_arg)
 	/* Load the libpq-specific functions */
 	load_file("libpqwalreceiver", false);
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
@@ -4565,71 +4755,6 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-		bool		must_use_password;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-
-		/* Is the use of a password mandatory? */
-		must_use_password = MySubscription->passwordrequired &&
-			!superuser_arg(MySubscription->owner);
-
-		/* Note that the superuser_arg call can access the DB */
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												must_use_password,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
 	 * the subscription relation state.
@@ -4638,92 +4763,63 @@ ApplyWorkerMain(Datum main_arg)
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
+	/* This is leader apply worker */
+	run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
+	proc_exit(0);
+}
 
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
+/* Logical Replication Tablesync worker entry point */
+void
+TablesyncWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *myslotname = NULL;
+	WalRcvStreamOptions options;
+
+	/* Attach to slot */
+	logicalrep_worker_attach(worker_slot);
+
+	/* Setup signal handling */
+	pqsignal(SIGHUP, SignalHandlerForConfigReload);
+	pqsignal(SIGTERM, die);
+	BackgroundWorkerUnblockSignals();
 
 	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 * We don't currently need any ResourceOwner in a walreceiver process, but
+	 * if we did, we could call CreateAuxProcessResourceOwner here.
 	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
 
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
+	/* Initialise stats to a sanish value */
+	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
+		MyLogicalRepWorker->reply_time = GetCurrentTimestamp();
 
-	if (!am_tablesync_worker())
-	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
-		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+	/* Load the libpq-specific functions */
+	load_file("libpqwalreceiver", false);
 
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
-		}
-		else
-		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-		}
+	InitializeLogRepWorker();
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-								 MySubscription->name,
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-								 "?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
+	/* Connect to the origin and start the replication. */
+	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
+		 MySubscription->conninfo);
 
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+	/*
+	 * Setup callback for syscache so that we know when something changes in
+	 * the subscription relation state.
+	 */
+	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
+								  invalidate_syncing_table_states,
+								  (Datum) 0);
 
-	proc_exit(0);
+	run_tablesync_worker(&options,
+						 myslotname,
+						 originname,
+						 sizeof(originname),
+						 &origin_startpos);
+
+	finish_sync_worker();
 }
 
 /*
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 39588da79f..bbd71d0b42 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -18,6 +18,7 @@ extern PGDLLIMPORT volatile sig_atomic_t ParallelApplyMessagePending;
 
 extern void ApplyWorkerMain(Datum main_arg);
 extern void ParallelApplyWorkerMain(Datum main_arg);
+extern void TablesyncWorkerMain(Datum main_arg);
 
 extern bool IsLogicalWorker(void);
 extern bool IsLogicalParallelApplyWorker(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 343e781896..7aba034774 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -265,7 +265,7 @@ extern void maybe_reread_subscription(void);
 
 extern void stream_cleanup_files(Oid subid, TransactionId xid);
 
-extern void InitializeApplyWorker(void);
+extern void InitializeLogRepWorker(void);
 
 extern void store_flush_position(XLogRecPtr remote_lsn, XLogRecPtr local_lsn);
 
@@ -307,6 +307,8 @@ extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
 
 #define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)
 
+extern void finish_sync_worker(void);
+
 static inline bool
 am_tablesync_worker(void)
 {
-- 
2.27.0

v16-0002-Reuse-Tablesync-Workers.patchapplication/octet-stream; name=v16-0002-Reuse-Tablesync-Workers.patchDownload
From db3e8e2d7aadea79126c5816bce8b06dc82f33c2 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:04:46 +0300
Subject: [PATCH v16 2/5] Reuse Tablesync Workers

This commit allows reusing tablesync workers for syncing more than one
table sequentially during their lifetime, instead of exiting after
only syncing one table.

Before this commit, tablesync workers were capable of syncing only one
table. For each table, a new sync worker was launched and that worker would
exit when done processing the table.

Now, tablesync workers are not limited to processing only one
table. When done, they can move to processing another table in
the same subscription.

If there is a table that needs to be synced, an available tablesync
worker picks up that table and syncs it. Each tablesync worker
continues to pick new tables to sync until there are no tables left
requiring synchronization. If there was no available worker to
process the table, then a new tablesync worker will be launched,
provided the number of tablesync workers for the subscription does not
exceed max_sync_workers_per_subscription.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/replication/logical/launcher.c  |   1 +
 src/backend/replication/logical/tablesync.c |  50 ++++++--
 src/backend/replication/logical/worker.c    | 121 ++++++++++++++++++--
 src/include/replication/worker_internal.h   |   7 ++
 4 files changed, 158 insertions(+), 21 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e231fa7f95..72e5ef8a78 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -440,6 +440,7 @@ retry:
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
+	worker->is_sync_completed = false;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 8125bbd170..605c5bd4ec 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -129,15 +129,14 @@ static bool FetchTableStates(bool *started_tx);
 static StringInfo copybuf = NULL;
 
 /*
- * Exit routine for synchronization worker.
+ * Prepares the synchronization worker for reuse or exit.
  */
 void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+clean_sync_worker(void)
 {
 	/*
-	 * Commit any outstanding transaction. This is the usual case, unless
-	 * there was nothing to do for the table.
+	 * Commit any outstanding transaction. This is the usual case, unless there
+	 * was nothing to do for the table.
 	 */
 	if (IsTransactionState())
 	{
@@ -145,19 +144,38 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
+	/*
+	 * Disconnect from publisher. Otherwise reused sync workers causes
+	 * exceeding max_wal_senders
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		walrcv_disconnect(LogRepWorkerWalRcvConn);
+		LogRepWorkerWalRcvConn = NULL;
+	}
+
+	/* Find the leader apply worker and signal it. */
+	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
+}
+
+/*
+ * Exit routine for synchronization worker.
+ */
+void
+pg_attribute_noreturn()
+finish_sync_worker(void)
+{
+	clean_sync_worker();
+
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
 	StartTransactionCommand();
 	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
+			(errmsg("logical replication table synchronization worker for subscription \"%s\" has finished",
+					MySubscription->name)));
 	CommitTransactionCommand();
 
-	/* Find the leader apply worker and signal it. */
-	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
-
 	/* Stop gracefully */
 	proc_exit(0);
 }
@@ -379,7 +397,15 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 */
 		replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
+		/* Sync worker has completed synchronization of the current table. */
+		MyLogicalRepWorker->is_sync_completed = true;
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", relation \"%s\" with relid %u has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
+		CommitTransactionCommand();
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 96ddad356b..eae561db05 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3626,6 +3626,20 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can call process_syncing_tables_for_sync.
+				 *
+				 * process_syncing_tables_for_sync decides whether the sync of
+				 * the current table is completed. If it is completed,
+				 * streaming must be already ended. So, we can break the loop.
+				 */
+				if (MyLogicalRepWorker->is_sync_completed)
+				{
+					endofstream = true;
+					break;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -3645,6 +3659,15 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+
+			/*
+			 * If is_sync_completed is true, this means that the tablesync
+			 * worker is done with synchronization. Streaming has already been
+			 * ended by process_syncing_tables_for_sync. We should move to the
+			 * next table if needed, or exit.
+			 */
+			if (MyLogicalRepWorker->is_sync_completed)
+				endofstream = true;
 		}
 
 		/* Cleanup the memory. */
@@ -3747,8 +3770,12 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * Tablesync workers should end streaming before exiting the main loop to
+	 * drop replication slot. Only end streaming here for apply workers.
+	 */
+	if (!am_tablesync_worker())
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
 }
 
 /*
@@ -4523,6 +4550,8 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 					 int originname_size,
 					 XLogRecPtr *origin_startpos)
 {
+	MyLogicalRepWorker->is_sync_completed = false;
+
 	/* Start table synchronization. */
 	start_table_sync(origin_startpos, &slotname);
 
@@ -4702,10 +4731,11 @@ InitializeLogRepWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("%s for subscription \"%s\", table \"%s\" has started",
+				(errmsg("%s for subscription \"%s\", table \"%s\" with relid %u has started",
 						get_worker_name(),
 						MySubscription->name,
-						get_rel_name(MyLogicalRepWorker->relid))));
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
 	else
 		ereport(LOG,
 		/* translator: first %s is the name of logical replication worker */
@@ -4813,11 +4843,84 @@ TablesyncWorkerMain(Datum main_arg)
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	run_tablesync_worker(&options,
-						 myslotname,
-						 originname,
-						 sizeof(originname),
-						 &origin_startpos);
+	/*
+	 * The loop where worker does its job. It loops until there is no relation
+	 * left to sync.
+	 */
+	for (;;)
+	{
+		List	   *rstates;
+		ListCell   *lc;
+		bool 	is_table_found = false;
+
+		run_tablesync_worker(&options,
+							 myslotname,
+							 originname,
+							 sizeof(originname),
+							 &origin_startpos);
+
+		if (IsTransactionState())
+			CommitTransactionCommand();
+
+		if (MyLogicalRepWorker->is_sync_completed)
+		{
+			/* This transaction will be committed by clean_sync_worker. */
+			StartTransactionCommand();
+
+			/*
+			 * Check if there is any table whose relation state is still INIT.
+			 * If a table in INIT state is found, the worker will not be
+			 * finished, it will be reused instead.
+			 */
+			rstates = GetSubscriptionRelations(MySubscription->oid, true);
+
+			foreach(lc, rstates)
+			{
+				SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
+
+				if (rstate->state == SUBREL_STATE_SYNCDONE)
+					continue;
+
+				/*
+				 * Take exclusive lock to prevent any other sync worker from
+				 * picking the same table.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
+
+				/*
+				 * Pick the table for the next run if it is not already picked
+				 * up by another worker.
+				 */
+				if (!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+				{
+					/* Update worker state for the next table */
+					MyLogicalRepWorker->relid = rstate->relid;
+					MyLogicalRepWorker->relstate = rstate->state;
+					MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+					LWLockRelease(LogicalRepWorkerLock);
+
+					/* Found a table for next iteration */
+					is_table_found = true;
+					clean_sync_worker();
+
+					StartTransactionCommand();
+					ereport(LOG,
+							(errmsg("%s for subscription \"%s\" has moved to sync table \"%s\" with relid %u.",
+									get_worker_name(),
+									MySubscription->name,
+									get_rel_name(MyLogicalRepWorker->relid),
+									MyLogicalRepWorker->relid)));
+					CommitTransactionCommand();
+
+					break;
+				}
+				LWLockRelease(LogicalRepWorkerLock);
+			}
+
+			if (!is_table_found)
+				break;
+		}
+	}
 
 	finish_sync_worker();
 }
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 7aba034774..1e9f8e6e72 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -56,6 +56,12 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/*
+	 * Indicates whether tablesync worker has completed sycning its assigned
+	 * table. If true, no need to continue with that table.
+	 */
+	bool		is_sync_completed;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction or
@@ -308,6 +314,7 @@ extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
 #define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)
 
 extern void finish_sync_worker(void);
+extern void clean_sync_worker(void);
 
 static inline bool
 am_tablesync_worker(void)
-- 
2.27.0

v16-0003-reuse-connection-when-tablesync-workers-change-t.patchapplication/octet-stream; name=v16-0003-reuse-connection-when-tablesync-workers-change-t.patchDownload
From b0f3066e5d592c603aee8f89ff4171f2b2a8e7f6 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:13:52 +0300
Subject: [PATCH v16 3/5] reuse connection when tablesync workers change the
 target

---
 src/backend/replication/logical/tablesync.c | 53 ++++++++++++++-------
 src/backend/replication/logical/worker.c    | 30 +++++++-----
 src/backend/replication/walsender.c         |  6 +++
 src/include/replication/worker_internal.h   |  3 +-
 4 files changed, 61 insertions(+), 31 deletions(-)

diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 605c5bd4ec..f042d9ae00 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -144,16 +144,6 @@ clean_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
-	/*
-	 * Disconnect from publisher. Otherwise reused sync workers causes
-	 * exceeding max_wal_senders
-	 */
-	if (LogRepWorkerWalRcvConn != NULL)
-	{
-		walrcv_disconnect(LogRepWorkerWalRcvConn);
-		LogRepWorkerWalRcvConn = NULL;
-	}
-
 	/* Find the leader apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
 }
@@ -167,6 +157,16 @@ finish_sync_worker(void)
 {
 	clean_sync_worker();
 
+	/*
+	 * Disconnect from publisher. Otherwise reused sync workers causes
+	 * exceeding max_wal_senders.
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		walrcv_disconnect(LogRepWorkerWalRcvConn);
+		LogRepWorkerWalRcvConn = NULL;
+	}
+
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
@@ -1268,7 +1268,7 @@ ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
  * The returned slot name is palloc'ed in current memory context.
  */
 char *
-LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
+LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 {
 	char	   *slotname;
 	char	   *err;
@@ -1321,14 +1321,31 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 									NAMEDATALEN);
 
 	/*
-	 * Here we use the slot name instead of the subscription name as the
-	 * application_name, so that it is different from the leader apply worker,
-	 * so that synchronous replication can distinguish them.
+	 * Connect to publisher if not yet. The application_name must be also
+	 * different from the leader apply worker because synchronous replication
+	 * must distinguish them.
 	 */
-	LogRepWorkerWalRcvConn =
-		walrcv_connect(MySubscription->conninfo, true,
-					   must_use_password,
-					   slotname, &err);
+	if (LogRepWorkerWalRcvConn == NULL)
+	{
+		char application_name[NAMEDATALEN];
+
+		/*
+		 * FIXME: set appropriate application_name. Previously, the slot name
+		 * was used becasue the lifetime of the tablesync worker was same as
+		 * that, but now the tablesync worker handles many slots during the
+		 * synchronization so that it is not suitable. So what should be?
+		 * Note that if the tablesync worker starts to reuse the replication
+		 * slot during synchronization, we should use the slot name as
+		 * application_name again.
+		 */
+		snprintf(application_name, NAMEDATALEN, "pg_%u_sync_%i",
+				 MySubscription->oid, worker_slot);
+		LogRepWorkerWalRcvConn =
+			walrcv_connect(MySubscription->conninfo, true,
+						   must_use_password,
+						   application_name, &err);
+	}
+
 	if (LogRepWorkerWalRcvConn == NULL)
 		ereport(ERROR,
 				(errcode(ERRCODE_CONNECTION_FAILURE),
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index eae561db05..ca663445c1 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3500,19 +3500,21 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 	/*
 	 * Init the ApplyMessageContext which we clean up after each replication
-	 * protocol message.
+	 * protocol message, if needed.
 	 */
-	ApplyMessageContext = AllocSetContextCreate(ApplyContext,
-												"ApplyMessageContext",
-												ALLOCSET_DEFAULT_SIZES);
+	if (!ApplyMessageContext)
+		ApplyMessageContext = AllocSetContextCreate(ApplyContext,
+													"ApplyMessageContext",
+													ALLOCSET_DEFAULT_SIZES);
 
 	/*
 	 * This memory context is used for per-stream data when the streaming mode
 	 * is enabled. This context is reset on each stream stop.
 	 */
-	LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
-													"LogicalStreamingContext",
-													ALLOCSET_DEFAULT_SIZES);
+	if (!LogicalStreamingContext)
+		LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
+														"LogicalStreamingContext",
+														ALLOCSET_DEFAULT_SIZES);
 
 	/* mark as idle, before starting to loop */
 	pgstat_report_activity(STATE_IDLE, NULL);
@@ -4468,7 +4470,9 @@ TwoPhaseTransactionGid(Oid subid, TransactionId xid, char *gid, int szgid)
  * are not repeatable.
  */
 static void
-start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+start_table_sync(XLogRecPtr *origin_startpos,
+				 char **myslotname,
+				 int worker_slot)
 {
 	char	   *syncslotname = NULL;
 
@@ -4477,7 +4481,7 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 	PG_TRY();
 	{
 		/* Call initial sync. */
-		syncslotname = LogicalRepSyncTableStart(origin_startpos);
+		syncslotname = LogicalRepSyncTableStart(origin_startpos, worker_slot);
 	}
 	PG_CATCH();
 	{
@@ -4548,12 +4552,13 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 					 char *slotname,
 					 char *originname,
 					 int originname_size,
-					 XLogRecPtr *origin_startpos)
+					 XLogRecPtr *origin_startpos,
+					 int worker_slot)
 {
 	MyLogicalRepWorker->is_sync_completed = false;
 
 	/* Start table synchronization. */
-	start_table_sync(origin_startpos, &slotname);
+	start_table_sync(origin_startpos, &slotname, worker_slot);
 
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
@@ -4857,7 +4862,8 @@ TablesyncWorkerMain(Datum main_arg)
 							 myslotname,
 							 originname,
 							 sizeof(originname),
-							 &origin_startpos);
+							 &origin_startpos,
+							 worker_slot);
 
 		if (IsTransactionState())
 			CommitTransactionCommand();
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index cedadb0036..06adcf6f01 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1827,6 +1827,12 @@ exec_replication_command(const char *cmd_string)
 				set_ps_display(cmdtag);
 				PreventInTransactionBlock(true, cmdtag);
 
+				/*
+				 * Initialize the flag again because this streaming may be
+				 * second time.
+				 */
+				streamingDoneSending = streamingDoneReceiving = false;
+
 				if (cmd->kind == REPLICATION_KIND_PHYSICAL)
 					StartReplication(cmd);
 				else
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 1e9f8e6e72..af6fd339f7 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -249,7 +249,8 @@ extern int	logicalrep_sync_worker_count(Oid subid);
 
 extern void ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 											   char *originname, Size szoriginname);
-extern char *LogicalRepSyncTableStart(XLogRecPtr *origin_startpos);
+extern char *LogicalRepSyncTableStart(XLogRecPtr *origin_startpos,
+									  int worker_slot);
 
 extern bool AllTablesyncsReady(void);
 extern void UpdateTwoPhaseState(Oid suboid, char new_state);
-- 
2.27.0

v16-0004-Add-replication-protocol-cmd-to-create-a-snapsho.patchapplication/octet-stream; name=v16-0004-Add-replication-protocol-cmd-to-create-a-snapsho.patchDownload
From 8cb6298a45137f853a07470f56c07296d676692a Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 13 Oct 2022 17:05:45 +0300
Subject: [PATCH v16 4/5] Add replication protocol cmd to create a snapshot

Introduced CREATE_REPLICATION_SNAPSHOT to be able to create and use a
snapshot without creating a new replication slot, but by using an
existing slot.

CREATE_REPLICATION_SNAPSHOT simply does what CREATE_REPLICATION_SLOT does
without creating a new replication slot.

CREATE_REPLICATION_SNAPSHOT command imports the snapshot into the current
transaction and returns consistent_point. The changes earlier than the
consistent_point will be applied by importing the snapshot. All changes
later than the consistent_point will be available to be consumed from
the replication slot.

This is useful for reusing replication slots in logical replication.
Otherwise, tablesync workers cannot start from a consistent point to copy
a relation and then apply changes by consuming from replication slot.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/protocol.sgml                    |  31 ++++++
 .../libpqwalreceiver/libpqwalreceiver.c       |  69 +++++++++++-
 src/backend/replication/logical/logical.c     |  40 ++++++-
 .../replication/logical/logicalfuncs.c        |   1 +
 src/backend/replication/repl_gram.y           |  18 ++-
 src/backend/replication/repl_scanner.l        |   2 +
 src/backend/replication/slotfuncs.c           |   1 +
 src/backend/replication/walsender.c           | 104 +++++++++++++++++-
 src/include/nodes/replnodes.h                 |  11 ++
 src/include/replication/logical.h             |   1 +
 src/include/replication/walreceiver.h         |  13 +++
 src/tools/pgindent/typedefs.list              |   2 +
 12 files changed, 289 insertions(+), 4 deletions(-)

diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index b11d9a6ba3..f7d5acee93 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2595,6 +2595,37 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
      </listitem>
     </varlistentry>
 
+    <varlistentry id="protocol-replication-replication-slot-snapshot">
+     <term><literal>CREATE_REPLICATION_SNAPSHOT</literal> <replaceable class="parameter">slot_name</replaceable> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
+      <indexterm><primary>CREATE_REPLICATION_SNAPSHOT</primary></indexterm>
+     </term>
+     <listitem>
+      <para>
+       Creates a snapshot including all the changes from the replication slot until
+       the point at which the replication slot becomes consistent. Then the snapshot
+       is used in the current transaction. This command is currently only supported
+       for logical replication slots.
+      </para>
+
+      <para>
+       In response to this command, the server will return a one-row result set,
+       containing the following field:
+       <variablelist>
+        <varlistentry>
+         <term><literal>consistent_point</literal> (<type>text</type>)</term>
+         <listitem>
+          <para>
+           The WAL location at which the slot became consistent.  This is the
+           earliest location from which streaming can start on this replication
+           slot.
+          </para>
+         </listitem>
+        </varlistentry>
+       </variablelist>
+      </para>
+     </listitem>
+    </varlistentry>
+
     <varlistentry id="protocol-replication-base-backup" xreflabel="BASE_BACKUP">
      <term><literal>BASE_BACKUP</literal> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
       <indexterm><primary>BASE_BACKUP</primary></indexterm>
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index dc9c5c82d9..17358bf552 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -82,6 +82,8 @@ static WalRcvExecResult *libpqrcv_exec(WalReceiverConn *conn,
 									   const int nRetTypes,
 									   const Oid *retTypes);
 static void libpqrcv_disconnect(WalReceiverConn *conn);
+static void libpqrcv_slot_snapshot(WalReceiverConn *conn, char *slotname,
+								   const WalRcvStreamOptions *options, XLogRecPtr *lsn);
 
 static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_connect = libpqrcv_connect,
@@ -98,7 +100,8 @@ static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_create_slot = libpqrcv_create_slot,
 	.walrcv_get_backend_pid = libpqrcv_get_backend_pid,
 	.walrcv_exec = libpqrcv_exec,
-	.walrcv_disconnect = libpqrcv_disconnect
+	.walrcv_disconnect = libpqrcv_disconnect,
+	.walrcv_slot_snapshot = libpqrcv_slot_snapshot
 };
 
 /* Prototypes for private functions */
@@ -982,6 +985,70 @@ libpqrcv_create_slot(WalReceiverConn *conn, const char *slotname,
 	return snapshot;
 }
 
+/*
+ * Create a new snapshot by using an existing replication slot.
+ */
+static void
+libpqrcv_slot_snapshot(WalReceiverConn *conn,
+					   char *slotname,
+					   const WalRcvStreamOptions *options,
+					   XLogRecPtr *lsn)
+{
+	StringInfoData cmd;
+	PGresult   *res;
+	char	   *pubnames_str;
+	List	   *pubnames;
+	char	   *pubnames_literal;
+
+	initStringInfo(&cmd);
+
+	/* Build the command. */
+	appendStringInfo(&cmd, "CREATE_REPLICATION_SNAPSHOT \"%s\"", slotname);
+	appendStringInfoString(&cmd, " (");
+	appendStringInfo(&cmd, " proto_version '%u'",
+					 options->proto.logical.proto_version);
+
+	/* Add publication names. */
+	pubnames = options->proto.logical.publication_names;
+	pubnames_str = stringlist_to_identifierstr(conn->streamConn, pubnames);
+	if (!pubnames_str)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("Could not create a snapshot by replication slot %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	pubnames_literal = PQescapeLiteral(conn->streamConn, pubnames_str,
+									   strlen(pubnames_str));
+	if (!pubnames_literal)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("Could not create a snapshot by replication slot %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	appendStringInfo(&cmd, ", publication_names %s", pubnames_literal);
+	PQfreemem(pubnames_literal);
+	pfree(pubnames_str);
+
+	appendStringInfoString(&cmd, " )");
+
+	/* Execute the command. */
+	res = libpqrcv_PQexec(conn->streamConn, cmd.data);
+	pfree(cmd.data);
+
+	if (PQresultStatus(res) != PGRES_TUPLES_OK)
+	{
+		PQclear(res);
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg("Could not create a snapshot by replication slot \"%s\": %s",
+						slotname, pchomp(PQerrorMessage(conn->streamConn)))));
+	}
+
+	if (lsn)
+		*lsn = DatumGetLSN(DirectFunctionCall1Coll(pg_lsn_in, InvalidOid,
+												   CStringGetDatum(PQgetvalue(res, 0, 0))));
+
+	PQclear(res);
+}
+
 /*
  * Return PID of remote backend process.
  */
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 41243d0187..3c11f8e1dd 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -476,6 +476,10 @@ CreateInitDecodingContext(const char *plugin,
  * fast_forward
  *		bypass the generation of logical changes.
  *
+ * need_full_snapshot
+ * 		if true, must obtain a snapshot able to read all tables;
+ *  	if false, one that can read only catalogs is acceptable.
+ *
  * xl_routine
  *		XLogReaderRoutine used by underlying xlogreader
  *
@@ -494,6 +498,7 @@ LogicalDecodingContext *
 CreateDecodingContext(XLogRecPtr start_lsn,
 					  List *output_plugin_options,
 					  bool fast_forward,
+					  bool need_full_snapshot,
 					  XLogReaderRoutine *xl_routine,
 					  LogicalOutputPluginWriterPrepareWrite prepare_write,
 					  LogicalOutputPluginWriterWrite do_write,
@@ -502,6 +507,7 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 	LogicalDecodingContext *ctx;
 	ReplicationSlot *slot;
 	MemoryContext old_context;
+	TransactionId xmin_horizon = InvalidTransactionId;
 
 	/* shorter lines... */
 	slot = MyReplicationSlot;
@@ -571,8 +577,40 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 		start_lsn = slot->data.confirmed_flush;
 	}
 
+
+	/*
+	 * We need to determine a safe xmin horizon to start decoding from if we
+	 * want to create a snapshot too. Otherwise we would end up with a
+	 * snapshot that cannot be imported since xmin value from the snapshot may
+	 * be less than the oldest safe xmin. To avoid this call
+	 * GetOldestSafeDecodingTransactionId() to return a safe xmin value, which
+	 * can be used while exporting/importing the snapshot.
+	 *
+	 * So we have to acquire the ProcArrayLock to prevent computation of new
+	 * xmin horizons by other backends, get the safe decoding xid, and inform
+	 * the slot machinery about the new limit. Once that's done the
+	 * ProcArrayLock can be released as the slot machinery now is protecting
+	 * against vacuum.
+	 */
+	if (need_full_snapshot)
+	{
+		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+		xmin_horizon = GetOldestSafeDecodingTransactionId(!need_full_snapshot);
+
+		SpinLockAcquire(&slot->mutex);
+		slot->effective_catalog_xmin = xmin_horizon;
+		slot->data.catalog_xmin = xmin_horizon;
+		slot->effective_xmin = xmin_horizon;
+		SpinLockRelease(&slot->mutex);
+
+		ReplicationSlotsComputeRequiredXmin(true);
+
+		LWLockRelease(ProcArrayLock);
+	}
+
 	ctx = StartupDecodingContext(output_plugin_options,
-								 start_lsn, InvalidTransactionId, false,
+								 start_lsn, xmin_horizon, need_full_snapshot,
 								 fast_forward, xl_routine, prepare_write,
 								 do_write, update_progress);
 
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index 55a24c02c9..85c5cdb633 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -208,6 +208,7 @@ pg_logical_slot_get_changes_guts(FunctionCallInfo fcinfo, bool confirm, bool bin
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									options,
 									false,
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/repl_gram.y b/src/backend/replication/repl_gram.y
index 0c874e33cf..957a5cc653 100644
--- a/src/backend/replication/repl_gram.y
+++ b/src/backend/replication/repl_gram.y
@@ -65,6 +65,7 @@ Node *replication_parse_result;
 %token K_CREATE_REPLICATION_SLOT
 %token K_DROP_REPLICATION_SLOT
 %token K_TIMELINE_HISTORY
+%token K_CREATE_REPLICATION_SNAPSHOT
 %token K_WAIT
 %token K_TIMELINE
 %token K_PHYSICAL
@@ -80,7 +81,7 @@ Node *replication_parse_result;
 %type <node>	command
 %type <node>	base_backup start_replication start_logical_replication
 				create_replication_slot drop_replication_slot identify_system
-				read_replication_slot timeline_history show
+				read_replication_slot timeline_history show create_replication_snapshot
 %type <list>	generic_option_list
 %type <defelt>	generic_option
 %type <uintval>	opt_timeline
@@ -114,6 +115,7 @@ command:
 			| read_replication_slot
 			| timeline_history
 			| show
+			| create_replication_snapshot
 			;
 
 /*
@@ -307,6 +309,19 @@ timeline_history:
 				}
 			;
 
+/*
+ * CREATE_REPLICATION_SNAPSHOT %s options
+ */
+create_replication_snapshot:
+			K_CREATE_REPLICATION_SNAPSHOT var_name plugin_options
+				{
+					CreateReplicationSnapshotCmd *n = makeNode(CreateReplicationSnapshotCmd);
+					n->slotname = $2;
+					n->options = $3;
+					$$ = (Node *) n;
+				}
+			;
+
 opt_physical:
 			K_PHYSICAL
 			| /* EMPTY */
@@ -400,6 +415,7 @@ ident_or_keyword:
 			| K_CREATE_REPLICATION_SLOT	{ $$ = "create_replication_slot"; }
 			| K_DROP_REPLICATION_SLOT		{ $$ = "drop_replication_slot"; }
 			| K_TIMELINE_HISTORY			{ $$ = "timeline_history"; }
+			| K_CREATE_REPLICATION_SNAPSHOT	{ $$ = "create_replication_snapshot"; }
 			| K_WAIT						{ $$ = "wait"; }
 			| K_TIMELINE					{ $$ = "timeline"; }
 			| K_PHYSICAL					{ $$ = "physical"; }
diff --git a/src/backend/replication/repl_scanner.l b/src/backend/replication/repl_scanner.l
index cb467ca46f..5ba2e9c54b 100644
--- a/src/backend/replication/repl_scanner.l
+++ b/src/backend/replication/repl_scanner.l
@@ -126,6 +126,7 @@ START_REPLICATION	{ return K_START_REPLICATION; }
 CREATE_REPLICATION_SLOT		{ return K_CREATE_REPLICATION_SLOT; }
 DROP_REPLICATION_SLOT		{ return K_DROP_REPLICATION_SLOT; }
 TIMELINE_HISTORY	{ return K_TIMELINE_HISTORY; }
+CREATE_REPLICATION_SNAPSHOT	{ return K_CREATE_REPLICATION_SNAPSHOT; }
 PHYSICAL			{ return K_PHYSICAL; }
 RESERVE_WAL			{ return K_RESERVE_WAL; }
 LOGICAL				{ return K_LOGICAL; }
@@ -303,6 +304,7 @@ replication_scanner_is_replication_command(void)
 		case K_DROP_REPLICATION_SLOT:
 		case K_READ_REPLICATION_SLOT:
 		case K_TIMELINE_HISTORY:
+		case K_CREATE_REPLICATION_SNAPSHOT:
 		case K_SHOW:
 			/* Yes; push back the first token so we can parse later. */
 			repl_pushed_back_token = first_token;
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 6035cf4816..c5b2d5b61f 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -486,6 +486,7 @@ pg_logical_replication_slot_advance(XLogRecPtr moveto)
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									NIL,
 									true,	/* fast_forward */
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 06adcf6f01..75ef245380 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -238,6 +238,7 @@ static void CreateReplicationSlot(CreateReplicationSlotCmd *cmd);
 static void DropReplicationSlot(DropReplicationSlotCmd *cmd);
 static void StartReplication(StartReplicationCmd *cmd);
 static void StartLogicalReplication(StartReplicationCmd *cmd);
+static void CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd);
 static void ProcessStandbyMessage(void);
 static void ProcessStandbyReplyMessage(void);
 static void ProcessStandbyHSFeedbackMessage(void);
@@ -1283,7 +1284,7 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	 * are reported early.
 	 */
 	logical_decoding_ctx =
-		CreateDecodingContext(cmd->startpoint, cmd->options, false,
+		CreateDecodingContext(cmd->startpoint, cmd->options, false, false,
 							  XL_ROUTINE(.page_read = logical_read_xlog_page,
 										 .segment_open = WalSndSegmentOpen,
 										 .segment_close = wal_segment_close),
@@ -1335,6 +1336,98 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	EndCommand(&qc, DestRemote, false);
 }
 
+/*
+ * Create a snapshot from an existing replication slot.
+ */
+static void
+CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd)
+{
+	Snapshot	snap;
+	LogicalDecodingContext *ctx;
+	char		xloc[MAXFNAMELEN];
+	DestReceiver *dest;
+	TupOutputState *tstate;
+	TupleDesc	tupdesc;
+	Datum		values[1];
+	bool		nulls[1] = {0};
+
+	Assert(!MyReplicationSlot);
+
+	CheckLogicalDecodingRequirements();
+
+	if (!IsTransactionBlock())
+		ereport(ERROR,
+				(errmsg("%s must be called inside a transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (XactIsoLevel != XACT_REPEATABLE_READ)
+		ereport(ERROR,
+				(errmsg("%s must be called in REPEATABLE READ isolation mode transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (!XactReadOnly)
+		ereport(ERROR,
+				(errmsg("%s must be called in a read only transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (FirstSnapshotSet)
+		ereport(ERROR,
+				(errmsg("%s must be called before any query",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (IsSubTransaction())
+		ereport(ERROR,
+				(errmsg("%s must not be called in a subtransaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	ReplicationSlotAcquire(cmd->slotname, false);
+
+	ctx = CreateDecodingContext(MyReplicationSlot->data.restart_lsn,
+								cmd->options,
+								false,
+								true,
+								XL_ROUTINE(.page_read = logical_read_xlog_page,
+										   .segment_open = WalSndSegmentOpen,
+										   .segment_close = wal_segment_close),
+								WalSndPrepareWrite, WalSndWriteData,
+								WalSndUpdateProgress);
+
+	/*
+	 * Signal that we don't need the timeout mechanism. We're just creating
+	 * the snapshot with the replication slot and don't yet accept feedback
+	 * messages or send keepalives. As we possibly need to wait for further
+	 * WAL the walsender would otherwise possibly be killed too soon.
+	 */
+	last_reply_timestamp = 0;
+
+	/* build initial snapshot, might take a while */
+	DecodingContextFindStartpoint(ctx);
+
+	snap = SnapBuildInitialSnapshot(ctx->snapshot_builder);
+	RestoreTransactionSnapshot(snap, MyProc);
+
+	/* Don't need the decoding context anymore */
+	FreeDecodingContext(ctx);
+
+	/* Create a tuple to send consistent WAL location */
+	snprintf(xloc, sizeof(xloc), "%X/%X",
+			 LSN_FORMAT_ARGS(MyReplicationSlot->data.confirmed_flush));
+
+	dest = CreateDestReceiver(DestRemoteSimple);
+	tupdesc = CreateTemplateTupleDesc(1);
+	TupleDescInitBuiltinEntry(tupdesc, (AttrNumber) 1, "consistent_point",
+							  TEXTOID, -1, 0);
+	tstate = begin_tup_output_tupdesc(dest, tupdesc, &TTSOpsVirtual);
+
+	/* consistent wal location */
+	values[0] = CStringGetTextDatum(xloc);
+
+	do_tup_output(tstate, values, nulls);
+	end_tup_output(tstate);
+
+	ReplicationSlotRelease();
+}
+
 /*
  * LogicalDecodingContext 'prepare_write' callback.
  *
@@ -1869,6 +1962,15 @@ exec_replication_command(const char *cmd_string)
 			}
 			break;
 
+		case T_CreateReplicationSnapshotCmd:
+			{
+				cmdtag = "CREATE_REPLICATION_SNAPSHOT";
+				set_ps_display(cmdtag);
+				CreateReplicationSnapshot((CreateReplicationSnapshotCmd *) cmd_node);
+				EndReplicationCommand(cmdtag);
+				break;
+			}
+
 		default:
 			elog(ERROR, "unrecognized replication command node tag: %u",
 				 cmd_node->type);
diff --git a/src/include/nodes/replnodes.h b/src/include/nodes/replnodes.h
index 4321ba8f86..154ab74f33 100644
--- a/src/include/nodes/replnodes.h
+++ b/src/include/nodes/replnodes.h
@@ -108,4 +108,15 @@ typedef struct TimeLineHistoryCmd
 	TimeLineID	timeline;
 } TimeLineHistoryCmd;
 
+/* ----------------------
+ *		CREATE_REPLICATION_SNAPSHOT command
+ * ----------------------
+ */
+typedef struct CreateReplicationSnapshotCmd
+{
+	NodeTag		type;
+	char	   *slotname;
+	List	   *options;
+} CreateReplicationSnapshotCmd;
+
 #endif							/* REPLNODES_H */
diff --git a/src/include/replication/logical.h b/src/include/replication/logical.h
index 5f49554ea0..6535786a0e 100644
--- a/src/include/replication/logical.h
+++ b/src/include/replication/logical.h
@@ -125,6 +125,7 @@ extern LogicalDecodingContext *CreateInitDecodingContext(const char *plugin,
 extern LogicalDecodingContext *CreateDecodingContext(XLogRecPtr start_lsn,
 													 List *output_plugin_options,
 													 bool fast_forward,
+													 bool need_full_snapshot,
 													 XLogReaderRoutine *xl_routine,
 													 LogicalOutputPluginWriterPrepareWrite prepare_write,
 													 LogicalOutputPluginWriterWrite do_write,
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 281626fa6f..f01e5f23a7 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -386,6 +386,16 @@ typedef WalRcvExecResult *(*walrcv_exec_fn) (WalReceiverConn *conn,
  */
 typedef void (*walrcv_disconnect_fn) (WalReceiverConn *conn);
 
+/*
+ * walrcv_slot_snapshot_fn
+ *
+ * Create a snapshot by an existing replication slot
+ */
+typedef void (*walrcv_slot_snapshot_fn) (WalReceiverConn *conn,
+										 char *slotname,
+										 const WalRcvStreamOptions *options,
+										 XLogRecPtr *lsn);
+
 typedef struct WalReceiverFunctionsType
 {
 	walrcv_connect_fn walrcv_connect;
@@ -403,6 +413,7 @@ typedef struct WalReceiverFunctionsType
 	walrcv_get_backend_pid_fn walrcv_get_backend_pid;
 	walrcv_exec_fn walrcv_exec;
 	walrcv_disconnect_fn walrcv_disconnect;
+	walrcv_slot_snapshot_fn walrcv_slot_snapshot;
 } WalReceiverFunctionsType;
 
 extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
@@ -437,6 +448,8 @@ extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
 	WalReceiverFunctions->walrcv_exec(conn, exec, nRetTypes, retTypes)
 #define walrcv_disconnect(conn) \
 	WalReceiverFunctions->walrcv_disconnect(conn)
+#define walrcv_slot_snapshot(conn, slotname, options, lsn) \
+	WalReceiverFunctions->walrcv_slot_snapshot(conn, slotname, options, lsn)
 
 static inline void
 walrcv_clear_result(WalRcvExecResult *walres)
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index e941fb6c82..cc023b10fa 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2348,6 +2348,7 @@ ReplicationSlotInvalidationCause
 ReplicationSlotOnDisk
 ReplicationSlotPersistency
 ReplicationSlotPersistentData
+CreateReplicationSnapshotCmd
 ReplicationState
 ReplicationStateCtl
 ReplicationStateOnDisk
@@ -3855,6 +3856,7 @@ walrcv_receive_fn
 walrcv_send_fn
 walrcv_server_version_fn
 walrcv_startstreaming_fn
+walrcv_slot_snapshot_fn
 wchar2mb_with_len_converter
 wchar_t
 win32_deadchild_waitinfo
-- 
2.27.0

v16-0005-Reuse-Replication-Slot-and-Origin-in-Tablesync.patchapplication/octet-stream; name=v16-0005-Reuse-Replication-Slot-and-Origin-in-Tablesync.patchDownload
From ba307b7714a7da7d89bae9a0670e9ab2aed38632 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH v16 5/5] Reuse Replication Slot and Origin in Tablesync

This commit allows reusing replication slots and origins during tablesync.

Earlier, a tablesync worker was creating a new replication slot and origin
each time it syncs a new table. With this patch, replication
slots/origins can be reusable for tablesync.

This reduces the overhead of creating/dropping replication slots and origins
and improves tablesync speed significantly especially for empty or small tables.

If the state of the current table is INIT or DATASYNC, tablesync worker needs a
repliation slot/origin. If the worker has not created slot and origin in
its previous runs, it will create those first. Otherwise the worker reuses
slot and origin created by the same worker in previous iterations earlier.
Tables in FINISHEDCOPY are expected to have a replication slot and origin.
Slot and origin names for such tables are persisted in
pg_subscription_rel catalog. Tablesync worker can fetch them and proceed
with existing slot and origin of FINISHEDCOPY tables and does not need to
create new ones.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/catalogs.sgml                    |  31 ++
 src/backend/catalog/pg_subscription.c         | 247 +++++++++++-
 src/backend/commands/subscriptioncmds.c       | 229 +++++++----
 .../replication/logical/applyparallelworker.c |   3 +-
 src/backend/replication/logical/launcher.c    |   8 +-
 src/backend/replication/logical/tablesync.c   | 373 +++++++++++++-----
 src/backend/replication/logical/worker.c      |  57 ++-
 src/include/catalog/pg_subscription.h         |   6 +
 src/include/catalog/pg_subscription_rel.h     |  14 +-
 src/include/replication/slot.h                |   3 +-
 src/include/replication/worker_internal.h     |  23 +-
 11 files changed, 803 insertions(+), 191 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 852cb30ae1..60718ab587 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8030,6 +8030,19 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        origin.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>sublastusedid</structfield> <type>int8</type>
+      </para>
+      <para>
+      The last used ID for tablesync workers. It acts as an unique identifier
+      for replication slots which are created by tablesync workers.
+      The last used ID needs to be persisted to make logical replication safely
+      proceed after any interruption. If sublastusedid is 0, then no table has
+      been synced yet.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
@@ -8114,6 +8127,24 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        otherwise null
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srrelslotname</structfield> <type>name</type>
+      </para>
+      <para>
+       Replication slot name that is used for synchronization of relation
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srreloriginname</structfield> <type>name</type>
+      </para>
+      <para>
+       Origin name that is used for tracking synchronization of relation
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index d07f88ce28..152fdaa310 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -108,6 +108,14 @@ GetSubscription(Oid subid, bool missing_ok)
 								   Anum_pg_subscription_suborigin);
 	sub->origin = TextDatumGetCString(datum);
 
+	/* Get last used id */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
+							tup,
+							Anum_pg_subscription_sublastusedid,
+							&isnull);
+	Assert(!isnull);
+	sub->lastusedid = DatumGetInt64(datum);
+
 	ReleaseSysCache(tup);
 
 	return sub;
@@ -199,6 +207,44 @@ DisableSubscription(Oid subid)
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update the last used replication slot ID for the given subscription.
+ */
+void
+UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid)
+{
+	Relation	rel;
+	bool		nulls[Natts_pg_subscription];
+	bool		replaces[Natts_pg_subscription];
+	Datum		values[Natts_pg_subscription];
+	HeapTuple	tup;
+
+	/* Look up the subscription in the catalog */
+	rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+	tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	/* Form a new tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	replaces[Anum_pg_subscription_sublastusedid - 1] = true;
+	values[Anum_pg_subscription_sublastusedid- 1] = Int64GetDatum(lastusedid);
+
+	/* Update the catalog */
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+	heap_freetuple(tup);
+
+	table_close(rel, NoLock);
+}
+
 /*
  * Convert text array to list of strings.
  *
@@ -228,7 +274,7 @@ textarray_to_stringlist(ArrayType *textarray)
  */
 void
 AddSubscriptionRelState(Oid subid, Oid relid, char state,
-						XLogRecPtr sublsn)
+						XLogRecPtr sublsn, char *relslotname, char *reloriginname)
 {
 	Relation	rel;
 	HeapTuple	tup;
@@ -257,6 +303,16 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
 	else
 		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -269,6 +325,60 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Internal function to modify columns for relation state update
+ */
+static void
+UpdateSubscriptionRelState_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char state,
+									XLogRecPtr sublsn)
+{
+	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
+	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+
+	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (sublsn != InvalidXLogRecPtr)
+		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
+	else
+		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication slot update
+ */
+static void
+UpdateSubscriptionRelReplicationSlot_internal(Datum *values,
+											bool *nulls,
+											bool *replaces,
+											char *relslotname)
+{
+	replaces[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication origin update
+ */
+static void
+UpdateSubscriptionRelOrigin_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char *reloriginname)
+{
+	replaces[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+}
+
 /*
  * Update the state of a subscription table.
  */
@@ -299,14 +409,56 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	memset(nulls, false, sizeof(nulls));
 	memset(replaces, false, sizeof(replaces));
 
-	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
-	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
 
-	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
-	if (sublsn != InvalidXLogRecPtr)
-		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
-	else
-		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update replication slot name, origin name and state of
+ * a subscription table in one transaction.
+ */
+void
+UpdateSubscriptionRel(Oid subid,
+					  Oid relid,
+					  char state,
+					  XLogRecPtr sublsn,
+					  char *relslotname,
+					  char *reloriginname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
+	UpdateSubscriptionRelOrigin_internal(values, nulls, replaces, reloriginname);
 
 	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
 							replaces);
@@ -318,6 +470,85 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Get origin name of subscription table.
+ *
+ * reloriginname's value has the replication origin name if the origin exists.
+ */
+void
+GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*originname;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srreloriginname, isnull);
+	if (!*isnull)
+	{
+		originname = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(reloriginname, originname, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
+/*
+ * Get replication slot name of subscription table.
+ *
+ * slotname's value has the replication slot name if the subscription has any.
+ */
+void
+GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*relrepslot;
+	bool		isnull;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srrelslotname, &isnull);
+	if (!isnull)
+	{
+		relrepslot = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(slotname, relrepslot, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
 /*
  * Get state of subscription table.
  *
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index d4e798baeb..4f5b3c572c 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -710,6 +710,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 		publicationListToArray(publications);
 	values[Anum_pg_subscription_suborigin - 1] =
 		CStringGetTextDatum(opts.origin);
+	values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(0);
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -773,7 +774,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 										 rv->schemaname, rv->relname);
 
 				AddSubscriptionRelState(subid, relid, table_state,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 			}
 
 			/*
@@ -864,6 +865,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
 	bool		must_use_password;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -943,7 +946,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				AddSubscriptionRelState(sub->oid, relid,
 										copy_data ? SUBREL_STATE_INIT : SUBREL_STATE_READY,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 				ereport(DEBUG1,
 						(errmsg_internal("table \"%s.%s\" added to subscription \"%s\"",
 										 rv->schemaname, rv->relname, sub->name)));
@@ -967,6 +970,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				char		state;
 				XLogRecPtr	statelsn;
+				char		slotname[NAMEDATALEN] = {0};
 
 				/*
 				 * Lock pg_subscription_rel with AccessExclusiveLock to
@@ -993,13 +997,36 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/*
+				 * Find the logical replication sync worker. If exists, store
+				 * the slot number for dropping associated replication slots
+				 * later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot_name);
+				}
+				else
+				{
+					/*
+					 * Sync of this relation might be failed in an earlier
+					 * attempt, but the replication slot might still exist.
+					 */
+					GetSubscriptionRelReplicationSlot(sub->oid, relid, slotname);
+					if (strlen(slotname) > 0)
+						sub_remove_slots = lappend(sub_remove_slots, slotname);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
 				 * tablesync origin.
 				 */
-				if (state != SUBREL_STATE_READY)
+				if (state != SUBREL_STATE_READY &&
+					state != SUBREL_STATE_SYNCDONE)
 				{
 					char		originname[NAMEDATALEN];
 
@@ -1027,31 +1054,24 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
-		 * the database operations we won't be able to rollback dropped slots.
+		 * Drop the replication slots associated with tablesync workers for
+		 * removed tables. This has to be at the end because otherwise if
+		 * there is an error while doing the database operations we won't be
+		 * able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			memcpy(syncslotname, lfirst(lc), sizeof(NAMEDATALEN));
+
+			/*
+			 * There is no certainty, maybe the slot does not exist yet. Also,
+			 * if we fail after removing some of the slots, next time, it will
+			 * again try to drop already dropped slots and fail. For these
+			 * reasons, we allow missing_ok = true for the drop.
+			 */
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1474,6 +1494,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	char	   *subname;
 	char	   *conninfo;
 	char	   *slotname;
+	int64		lastusedid;
 	List	   *subworkers;
 	ListCell   *lc;
 	char		originname[NAMEDATALEN];
@@ -1546,6 +1567,14 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	else
 		slotname = NULL;
 
+	/* Get the last used identifier by the subscription */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID, tup,
+							Anum_pg_subscription_sublastusedid, &isnull);
+	if (!isnull)
+		lastusedid = DatumGetInt64(datum);
+	else
+		lastusedid = 0;
+
 	/*
 	 * Since dropping a replication slot is not transactional, the replication
 	 * slot stays dropped even if the transaction rolls back.  So we cannot
@@ -1595,6 +1624,8 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	}
 	list_free(subworkers);
 
+	rstates = GetSubscriptionRelations(subid, true);
+
 	/*
 	 * Remove the no-longer-useful entry in the launcher's table of apply
 	 * worker start times.
@@ -1606,36 +1637,26 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	ApplyLauncherForgetWorkerStartTime(subid);
 
 	/*
-	 * Cleanup of tablesync replication origins.
-	 *
-	 * Any READY-state relations would already have dealt with clean-ups.
+	 * Cleanup of tablesync replication origins associated with the
+	 * subscription, if exists. Try to drop origins by creating all origin
+	 * names created for this subscription.
 	 *
 	 * Note that the state can't change because we have already stopped both
 	 * the apply and tablesync workers and they can't restart because of
 	 * exclusive lock on the subscription.
+	 *
+	 * XXX: This can be handled better instead of looping through all possible
 	 */
-	rstates = GetSubscriptionRelations(subid, true);
-	foreach(lc, rstates)
+	for (int64 i = 1; i <= lastusedid; i++)
 	{
-		SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-		Oid			relid = rstate->relid;
-
-		/* Only cleanup resources of tablesync workers */
-		if (!OidIsValid(relid))
-			continue;
+		char		originname_to_drop[NAMEDATALEN] = {0};
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * It is possible that the origin is not yet created for tablesync
-		 * worker so passing missing_ok = true. This can happen for the states
-		 * before SUBREL_STATE_FINISHEDCOPY.
-		 */
-		ReplicationOriginNameForLogicalRep(subid, relid, originname,
-										   sizeof(originname));
-		replorigin_drop_by_name(originname, true, false);
+		snprintf(originname_to_drop, sizeof(originname_to_drop), "pg_%u_%lld", subid, (long long) i);
+		/* missing_ok = true, since the origin might be already dropped. */
+		replorigin_drop_by_name(originname_to_drop, true, false);
 	}
 
+
 	/* Clean up dependencies */
 	deleteSharedDependencyRecordsFor(SubscriptionRelationId, subid, 0);
 
@@ -1694,39 +1715,17 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
+		List	   *slots = NULL;
 
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
 
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char	   *syncslotname = (char *) lfirst(lc);
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1743,6 +1742,71 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * Get the replication slot names associated with the subscription.
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok)
+{
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+					 " FROM pg_replication_slots"
+					 " WHERE slot_name LIKE 'pg_%i_sync_%%';",
+					 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+					errmsg("could not receive list of slots associated with the subscription %u, error: %s",
+					subid, res->err));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();
+		return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -2155,6 +2219,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -2165,18 +2230,20 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
-			char		syncslotname[NAMEDATALEN] = {0};
-
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
-				 syncslotname);
+				 worker->slot_name);
 		}
 	}
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 1d4e83c4c1..d2c70dffbc 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -440,7 +440,8 @@ pa_launch_parallel_worker(void)
 										MySubscription->name,
 										MyLogicalRepWorker->userid,
 										InvalidOid,
-										dsm_segment_handle(winfo->dsm_seg));
+										dsm_segment_handle(winfo->dsm_seg),
+										InvalidRepSlotId);
 
 	if (launched)
 	{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 72e5ef8a78..c84193319c 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -304,7 +304,7 @@ logicalrep_workers_find(Oid subid, bool only_running)
  */
 bool
 logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
-						 Oid relid, dsm_handle subworker_dsm)
+						 Oid relid, dsm_handle subworker_dsm, int64 slotid)
 {
 	BackgroundWorker bgw;
 	BackgroundWorkerHandle *bgw_handle;
@@ -430,6 +430,9 @@ retry:
 	worker->launch_time = now;
 	worker->in_use = true;
 	worker->generation++;
+	worker->created_slot = false;
+	worker->rep_slot_id = slotid;
+	worker->slot_name = (char *) palloc(NAMEDATALEN);
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -1183,7 +1186,8 @@ ApplyLauncherMain(Datum main_arg)
 				ApplyLauncherSetWorkerStartTime(sub->oid, now);
 				logicalrep_worker_launch(sub->dbid, sub->oid, sub->name,
 										 sub->owner, InvalidOid,
-										 DSM_HANDLE_INVALID);
+										 DSM_HANDLE_INVALID,
+										 InvalidRepSlotId);
 			}
 			else
 			{
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index f042d9ae00..830fa4696c 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -328,40 +328,29 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 								   MyLogicalRepWorker->relid,
 								   MyLogicalRepWorker->relstate,
 								   MyLogicalRepWorker->relstate_lsn);
+		CommitTransactionCommand();
 
 		/*
-		 * End streaming so that LogRepWorkerWalRcvConn can be used to drop
-		 * the slot.
-		 */
-		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
-
-		/*
-		 * Cleanup the tablesync slot.
+		 * Cleanup the tablesync slot. If the slot name used by this worker is
+		 * different from the default slot name for the worker, this means the
+		 * current table had started to being synchronized by another worker
+		 * and replication slot. And this worker is reusing a replication slot
+		 * from a previous attempt. We do not need that replication slot
+		 * anymore.
 		 *
 		 * This has to be done after updating the state because otherwise if
 		 * there is an error while doing the database operations we won't be
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
+										MyLogicalRepWorker->rep_slot_id,
 										syncslotname,
 										sizeof(syncslotname));
 
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
-		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
-
-		CommitTransactionCommand();
-		pgstat_report_stat(false);
-
-		/*
-		 * Start a new transaction to clean up the tablesync origin tracking.
-		 * This transaction will be ended within the finish_sync_worker().
-		 * Now, even, if we fail to remove this here, the apply worker will
-		 * ensure to clean it up afterward.
+		 * We are safe to drop the replication tracking origin after this
+		 * point. Now, even, if we fail to remove this here, the apply worker
+		 * will ensure to clean it up afterward.
 		 *
 		 * We need to do this after the table state is set to SYNCDONE.
 		 * Otherwise, if an error occurs while performing the database
@@ -370,32 +359,73 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 * have been cleared before restart. So, the restarted worker will use
 		 * invalid replication progress state resulting in replay of
 		 * transactions that have already been applied.
+		 *
+		 * Firstly reset the origin session to remove the ownership of the
+		 * slot. This is needed to allow the origin to be dropped or reused
+		 * later.
 		 */
+		replorigin_session_reset();
+		replorigin_session_origin = InvalidRepOriginId;
+		replorigin_session_origin_lsn = InvalidXLogRecPtr;
+		replorigin_session_origin_timestamp = 0;
+
 		StartTransactionCommand();
+		if (MyLogicalRepWorker->slot_name && strcmp(syncslotname, MyLogicalRepWorker->slot_name) != 0)
+		{
+			/*
+			 * End streaming so that LogRepWorkerWalRcvConn can be used to
+			 * drop the slot.
+			 */
+			walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
 
-		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
+			ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+											   MyLogicalRepWorker->relid,
+											   originname,
+											   sizeof(originname));
+
+			/*
+			 * Drop replication origin
+			 *
+			 * There is a chance that the user is concurrently performing refresh
+			 * for the subscription where we remove the table state and its origin
+			 * or the apply worker would have removed this origin. So passing
+			 * missing_ok = true.
+			 */
+			replorigin_drop_by_name(originname, true, false);
+		}
 
 		/*
-		 * Resetting the origin session removes the ownership of the slot.
-		 * This is needed to allow the origin to be dropped.
+		 * We are safe to remove persisted replication slot and origin data,
+		 * since it's already in SYNCDONE state. They will not be needed
+		 * anymore.
 		 */
-		replorigin_session_reset();
-		replorigin_session_origin = InvalidRepOriginId;
-		replorigin_session_origin_lsn = InvalidXLogRecPtr;
-		replorigin_session_origin_timestamp = 0;
+		UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+							  MyLogicalRepWorker->relid,
+							  MyLogicalRepWorker->relstate,
+							  MyLogicalRepWorker->relstate_lsn,
+							  NULL,
+							  NULL);
+		ereport(DEBUG2,
+				(errmsg("process_syncing_tables_for_sync: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+						"NULL",
+						"NULL",
+						MyLogicalRepWorker->relstate,
+						MyLogicalRepWorker->relid,
+						MyLogicalRepWorker->subid)));
+		CommitTransactionCommand();
+		pgstat_report_stat(false);
 
 		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * There is a chance that the user is concurrently performing refresh
-		 * for the subscription where we remove the table state and its origin
-		 * or the apply worker would have removed this origin. So passing
-		 * missing_ok = true.
+		 * This should return the default origin name for the worker. Even if
+		 * the worker used a different origin for this table, it should be
+		 * dropped and removed from the catalog so far.
 		 */
-		replorigin_drop_by_name(originname, true, false);
+		StartTransactionCommand();
+		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+										   MyLogicalRepWorker->relid,
+										   originname,
+										   sizeof(originname));
 
 		/* Sync worker has completed synchronization of the current table. */
 		MyLogicalRepWorker->is_sync_completed = true;
@@ -492,6 +522,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 			if (current_lsn >= rstate->lsn)
 			{
 				char		originname[NAMEDATALEN];
+				bool		is_origin_null = true;
 
 				rstate->state = SUBREL_STATE_READY;
 				rstate->lsn = current_lsn;
@@ -512,18 +543,31 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				 * error while dropping we won't restart it to drop the
 				 * origin. So passing missing_ok = true.
 				 */
-				ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-												   rstate->relid,
-												   originname,
-												   sizeof(originname));
-				replorigin_drop_by_name(originname, true, false);
+				GetSubscriptionRelOrigin(MyLogicalRepWorker->subid,
+										 rstate->relid, originname,
+										 &is_origin_null);
+
+				if (!is_origin_null)
+				{
+					replorigin_drop_by_name(originname, true, false);
+				}
 
 				/*
 				 * Update the state to READY only after the origin cleanup.
 				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-										   rstate->relid, rstate->state,
-										   rstate->lsn);
+				UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+									  rstate->relid,
+									  rstate->state,
+									  rstate->lsn,
+									  NULL,
+									  NULL);
+				ereport(DEBUG2,
+					(errmsg("process_syncing_tables_for_apply: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+							"NULL", "NULL", rstate->state,
+							rstate->relid, MyLogicalRepWorker->subid)));
+
+				CommitTransactionCommand();
+				started_tx = false;
 			}
 		}
 		else
@@ -612,12 +656,25 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						TimestampDifferenceExceeds(hentry->last_start_time, now,
 												   wal_retrieve_retry_interval))
 					{
+						if (IsTransactionState())
+							CommitTransactionCommand();
+						StartTransactionCommand();
+						started_tx = true;
+
+						MySubscription->lastusedid++;
+						UpdateSubscriptionLastSlotId(MyLogicalRepWorker->subid,
+													 MySubscription->lastusedid);
+						ereport(DEBUG2,
+								(errmsg("process_syncing_tables_for_apply: incremented lastusedid to %lld for subscription %u",
+										(long long) MySubscription->lastusedid, MySubscription->oid)));
+
 						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
 												 MySubscription->oid,
 												 MySubscription->name,
 												 MyLogicalRepWorker->userid,
 												 rstate->relid,
-												 DSM_HANDLE_INVALID);
+												 DSM_HANDLE_INVALID,
+												 MySubscription->lastusedid);
 						hentry->last_start_time = now;
 					}
 				}
@@ -1240,8 +1297,8 @@ copy_table(Relation rel)
  * The name must not exceed NAMEDATALEN - 1 because of remote node constraints
  * on slot name length. We append system_identifier to avoid slot_name
  * collision with subscriptions in other clusters. With the current scheme
- * pg_%u_sync_%u_UINT64_FORMAT (3 + 10 + 6 + 10 + 20 + '\0'), the maximum
- * length of slot_name will be 50.
+ * pg_%u_sync_%lu_UINT64_FORMAT (3 + 10 + 6 + 20 + 20 + '\0'), the maximum
+ * length of slot_name will be 45.
  *
  * The returned slot name is stored in the supplied buffer (syncslotname) with
  * the given size.
@@ -1252,11 +1309,11 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
+ReplicationSlotNameForTablesync(Oid suboid, int64 slotid,
 								char *syncslotname, Size szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%lld_" UINT64_FORMAT, suboid,
+			(long long) slotid, GetSystemIdentifier());
 }
 
 /*
@@ -1282,6 +1339,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 	UserContext ucxt;
 	bool		must_use_password;
 	bool		run_as_owner;
+	char	   *prev_slotname;
 
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
@@ -1316,7 +1374,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
+									MyLogicalRepWorker->rep_slot_id,
 									slotname,
 									NAMEDATALEN);
 
@@ -1355,12 +1413,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC ||
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY);
 
+	/*
+	 * See if tablesync of the current relation has been started with another
+	 * replication slot.
+	 *
+	 * Read previous slot name from the catalog, if exists.
+	 */
+	prev_slotname = (char *) palloc(NAMEDATALEN);
+	StartTransactionCommand();
+	GetSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+									  MyLogicalRepWorker->relid,
+									  prev_slotname);
+
 	/* Assign the origin tracking record name. */
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   sizeof(originname));
 
+	CommitTransactionCommand();
+
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC)
 	{
 		/*
@@ -1374,10 +1446,53 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 		 * breakdown then it wouldn't have succeeded so trying it next time
 		 * seems like a better bet.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, slotname, true);
+		if (strlen(prev_slotname) > 0)
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);
+
+			StartTransactionCommand();
+			/* Replication origin might still exist. Try to drop */
+			replorigin_drop_by_name(originname, true, false);
+
+			/*
+			 * Remove replication slot and origin name from the relation's
+			 * catalog record
+			 */
+			UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+								  MyLogicalRepWorker->relid,
+								  MyLogicalRepWorker->relstate,
+								  MyLogicalRepWorker->relstate_lsn,
+								  NULL,
+								  NULL);
+			CommitTransactionCommand();
+			ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+						"NULL", "NULL", MyLogicalRepWorker->relstate,
+						MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+		}
 	}
 	else if (MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY)
 	{
+		/*
+		 * At this point, the table that is currently being synchronized
+		 * should have its replication slot name filled in the catalog. The
+		 * tablesync process was started with another sync worker and
+		 * replication slot. We need to continue using the same replication
+		 * slot in this worker too.
+		 */
+		if (strlen(prev_slotname) == 0)
+		{
+			elog(ERROR, "Replication slot could not be found for subscription %u, relation %u",
+				 MyLogicalRepWorker->subid,
+				 MyLogicalRepWorker->relid);
+		}
+
+		/*
+		 * Proceed with the correct replication slot. Use previously created
+		 * replication slot to sync this table.
+		 */
+		memcpy(slotname, prev_slotname, NAMEDATALEN);
+
 		/*
 		 * The COPY phase was previously done, but tablesync then crashed
 		 * before it was able to finish normally.
@@ -1397,7 +1512,9 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 
 		goto copy_table_done;
 	}
+	pfree(prev_slotname);
 
+	/* Preparing for table copy operation */
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 	MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
 	MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
@@ -1405,11 +1522,31 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   MyLogicalRepWorker->relstate,
-							   MyLogicalRepWorker->relstate_lsn);
+
+	/*
+	 * Refresh the originname in case of having non-existing origin
+	 * from previous failed sync attempts.
+	 * If that's the case, it should be removed from the catalog so far.
+	 * Then, we can continue by reusing the origin created by the current
+	 * worker instead of .
+	 */
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									MyLogicalRepWorker->relid,
+									originname,
+									sizeof(originname));
+
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  MyLogicalRepWorker->relstate,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					slotname, originname, MyLogicalRepWorker->relstate,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+
 	pgstat_report_stat(true);
 
 	StartTransactionCommand();
@@ -1437,48 +1574,96 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 						res->err)));
 	walrcv_clear_result(res);
 
+	originid = replorigin_by_name(originname, true);
+
 	/*
 	 * Create a new permanent logical decoding slot. This slot will be used
 	 * for the catchup phase after COPY is done, so tell it to use the
 	 * snapshot to make the final data consistent.
+	 *
+	 * Replication slot will only be created if either this is the first run
+	 * of the worker or we're not using a previous replication slot.
 	 */
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-
-	/*
-	 * Setup replication origin tracking. The purpose of doing this before the
-	 * copy is to avoid doing the copy again due to any error in setting up
-	 * origin tracking.
-	 */
-	originid = replorigin_by_name(originname, true);
-	if (!OidIsValid(originid))
+	if (!MyLogicalRepWorker->created_slot)
 	{
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						   slotname, false /* permanent */ , false /* two_phase */ ,
+						   CRS_USE_SNAPSHOT, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: created replication slot %s for subscription %u",
+						slotname, MyLogicalRepWorker->subid)));
+
 		/*
-		 * Origin tracking does not exist, so create it now.
-		 *
-		 * Then advance to the LSN got from walrcv_create_slot. This is WAL
-		 * logged for the purpose of recovery. Locks are to prevent the
-		 * replication origin from vanishing while advancing.
+		 * Remember that we created the slot so that we will not try to create
+		 * it again.
 		 */
-		originid = replorigin_create(originname);
-
-		LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
-		replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
-						   true /* go backward */ , true /* WAL log */ );
-		UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->created_slot = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
+		/*
+		 * Setup replication origin tracking. The purpose of doing this before
+		 * the copy is to avoid doing the copy again due to any error in
+		 * setting up origin tracking.
+		 */
+		if (!OidIsValid(originid))
+		{
+			/*
+			 * Origin tracking does not exist, so create it now.
+			 */
+			originid = replorigin_create(originname);
+		}
+		else
+		{
+			/*
+			 * At this point, there shouldn't be any existing replication
+			 * origin with the same name.
+			 */
+			ereport(ERROR,
+					(errcode(ERRCODE_DUPLICATE_OBJECT),
+					 errmsg("replication origin \"%s\" already exists",
+							originname)));
+		}
 	}
 	else
 	{
-		ereport(ERROR,
-				(errcode(ERRCODE_DUPLICATE_OBJECT),
-				 errmsg("replication origin \"%s\" already exists",
-						originname)));
+		/*
+		 * Do not create a new replication slot, reuse the existing one
+		 * instead. Use a new snapshot for the replication slot to ensure that
+		 * tablesync and apply proceses are consistent with each other.
+		 */
+		WalRcvStreamOptions options;
+		int			server_version;
+
+		server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+		options.proto.logical.proto_version =
+			server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+			server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+			LOGICALREP_PROTO_VERSION_NUM;
+		options.proto.logical.publication_names = MySubscription->publications;
+
+		walrcv_slot_snapshot(LogRepWorkerWalRcvConn, slotname, &options, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: reusing replication slot %s for relation %u in subscription %u",
+						slotname, MyLogicalRepWorker->relid,
+						MyLogicalRepWorker->subid)));
 	}
 
+	/*
+	 * Advance to the LSN got from walrcv_create_slot or walrcv_slot_snapshot.
+	 * This is WAL logged for the purpose of recovery. Locks are to prevent
+	 * the replication origin from vanishing while advancing.
+	 *
+	 * Then setup replication origin tracking.
+	 */
+	LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+	replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
+					   true /* go backward */ , true /* WAL log */ );
+	UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+
 	/*
 	 * Make sure that the copy command runs as the table owner, unless the
 	 * user has opted out of that behaviour.
@@ -1537,12 +1722,18 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 	 * Update the persisted state to indicate the COPY phase is done; make it
 	 * visible to others.
 	 */
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   SUBREL_STATE_FINISHEDCOPY,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  SUBREL_STATE_FINISHEDCOPY,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					originname, slotname, SUBREL_STATE_FINISHEDCOPY,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
 
 copy_table_done:
 
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index ca663445c1..8966e02b77 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -466,8 +466,16 @@ ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 {
 	if (OidIsValid(relid))
 	{
-		/* Replication origin name for tablesync workers. */
-		snprintf(originname, szoriginname, "pg_%u_%u", suboid, relid);
+		bool		is_null = true;
+
+		/*
+		 * Replication origin name for tablesync workers. First, look into the
+		 * catalog. If originname does not exist, then use the default name.
+		 */
+		GetSubscriptionRelOrigin(suboid, relid,
+								 originname, &is_null);
+		if (is_null)
+			snprintf(originname, szoriginname, "pg_%u_%lld", suboid, (long long) MyLogicalRepWorker->rep_slot_id);
 	}
 	else
 	{
@@ -4504,6 +4512,9 @@ start_table_sync(XLogRecPtr *origin_startpos,
 
 	/* allocate slot name in long-lived context */
 	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+
+	/* Keep the replication slot name used for this sync. */
+	MyLogicalRepWorker->slot_name = *myslotname;
 	pfree(syncslotname);
 }
 
@@ -4557,13 +4568,25 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 {
 	MyLogicalRepWorker->is_sync_completed = false;
 
+	/*
+	 * If it's already connected to the publisher, end streaming before using
+	 * the same connection for another iteration
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		TimeLineID tli;
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	}
+
 	/* Start table synchronization. */
 	start_table_sync(origin_startpos, &slotname, worker_slot);
 
+	StartTransactionCommand();
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   originname_size);
+	CommitTransactionCommand();
 
 	set_apply_error_context_origin(originname);
 
@@ -4603,11 +4626,10 @@ run_apply_worker(WalRcvStreamOptions *options,
 				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
 					errmsg("subscription has no replication slot set")));
 
-	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-									   originname, originname_size);
-
 	/* Setup replication origin tracking. */
 	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
 	originid = replorigin_by_name(originname, true);
 	if (!OidIsValid(originid))
 		originid = replorigin_create(originname);
@@ -4924,7 +4946,32 @@ TablesyncWorkerMain(Datum main_arg)
 			}
 
 			if (!is_table_found)
+			{
+				TimeLineID	tli;
+
+				/*
+				 * It is important to give an error if we are unable to drop the
+				 * slot, otherwise, it won't be dropped till the corresponding
+				 * subscription is dropped. So passing missing_ok = false.
+				 */
+				if (MyLogicalRepWorker->created_slot)
+				{
+					walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+					ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
+				}
+
+				/*
+				 * Drop replication origin before exiting.
+				 *
+				 * There is a chance that the user is concurrently performing refresh
+				 * for the subscription where we remove the table state and its origin
+				 * or the apply worker would have removed this origin. So passing
+				 * missing_ok = true.
+				 */
+				replorigin_drop_by_name(originname, true, false);
+
 				break;
+			}
 		}
 	}
 
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 1d40eebc78..7e13f59847 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -108,6 +108,9 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
 
 	/* Only publish data originating from the specified origin */
 	text		suborigin BKI_DEFAULT(LOGICALREP_ORIGIN_ANY);
+
+	/* The last used ID to create a replication slot for tablesync */
+	int64		sublastusedid BKI_DEFAULT(0);
 #endif
 } FormData_pg_subscription;
 
@@ -144,6 +147,8 @@ typedef struct Subscription
 	List	   *publications;	/* List of publication names to subscribe to */
 	char	   *origin;			/* Only publish data originating from the
 								 * specified origin */
+	int64		lastusedid;		/* Last used unique ID to create replication
+								 * slots in tablesync */
 } Subscription;
 
 /* Disallow streaming in-progress transactions. */
@@ -164,6 +169,7 @@ typedef struct Subscription
 extern Subscription *GetSubscription(Oid subid, bool missing_ok);
 extern void FreeSubscription(Subscription *sub);
 extern void DisableSubscription(Oid subid);
+extern void UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid);
 
 extern int	CountDBSubscriptions(Oid dbid);
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 60a2bcca23..185164d75e 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -44,6 +44,12 @@ CATALOG(pg_subscription_rel,6102,SubscriptionRelRelationId)
 											 * used for synchronization
 											 * coordination, or NULL if not
 											 * valid */
+	NameData	srrelslotname BKI_FORCE_NULL;	/* name of the replication
+												 * slot for relation in
+												 * subscription */
+	NameData	srreloriginname BKI_FORCE_NULL; /* origin name for relation in
+												 * subscription */
+
 #endif
 } FormData_pg_subscription_rel;
 
@@ -81,10 +87,16 @@ typedef struct SubscriptionRelState
 } SubscriptionRelState;
 
 extern void AddSubscriptionRelState(Oid subid, Oid relid, char state,
-									XLogRecPtr sublsn);
+									XLogRecPtr sublsn, char *relslotname, char *reloriginname);
 extern void UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 									   XLogRecPtr sublsn);
+extern void UpdateSubscriptionRel(Oid subid, Oid relid, char state,
+								  XLogRecPtr sublsn, char *relslotname, char *reloriginname);
+
 extern char GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn);
+extern void GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname);
+extern void GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull);
+
 extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 
 extern bool HasSubscriptionRelations(Oid subid);
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index a8a89dc784..31b2c41893 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -237,8 +237,9 @@ extern bool InvalidateObsoleteReplicationSlots(ReplicationSlotInvalidationCause
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
 extern bool ReplicationSlotName(int index, Name name);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, Size szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, int64 slotid, char *syncslotname, Size szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List *GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index af6fd339f7..97c7d2aa50 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -35,6 +35,23 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/*
+	 * Indicates if the sync worker created a replication slot for itself
+	 * in any point of its lifetime.
+	 * False means that the worker has not created a slot yet, and has been
+	 * reusing replication slots created by other workers so far.
+	 */
+	bool		created_slot;
+
+	/*
+	 * Unique identifier for replication slot to be created by tablesnync
+	 * workers, if needed.
+	 */
+	int64		rep_slot_id;
+
+	/* Replication slot name used by the worker. */
+	char	   *slot_name;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
@@ -239,7 +256,8 @@ extern LogicalRepWorker *logicalrep_worker_find(Oid subid, Oid relid,
 extern List *logicalrep_workers_find(Oid subid, bool only_running);
 extern bool logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname,
 									 Oid userid, Oid relid,
-									 dsm_handle subworker_dsm);
+									 dsm_handle subworker_dsm,
+									 int64 slotid);
 extern void logicalrep_worker_stop(Oid subid, Oid relid);
 extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
 extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
@@ -336,4 +354,7 @@ am_parallel_apply_worker(void)
 	return isParallelApplyWorker(MyLogicalRepWorker);
 }
 
+/* Invalid identifier to be used for naming replication slots */
+#define InvalidRepSlotId	0
+
 #endif							/* WORKER_INTERNAL_H */
-- 
2.27.0

#86Peter Smith
smithpb2250@gmail.com
In reply to: Hayato Kuroda (Fujitsu) (#85)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi. Here are some review comments for the patch v16-0001

======
Commit message.

1.
Also; most of the code shared by both worker types are already combined
in LogicalRepApplyLoop(). There is no need to combine the rest in
ApplyWorkerMain() anymore.

~

/are already/is already/

/Also;/Also,/

~~~

2.
This commit introduces TablesyncWorkerMain() as a new entry point for
tablesync workers and separates both type of workers from each other.
This aims to increase code readability and help to maintain logical
replication workers separately.

2a.
/This commit/This patch/

~

2b.
"and separates both type of workers from each other"

Maybe that part can all be removed. The following sentence says the
same again anyhow.

======
src/backend/replication/logical/worker.c

3.
 static void stream_write_change(char action, StringInfo s);
 static void stream_open_and_write_change(TransactionId xid, char
action, StringInfo s);
 static void stream_close_file(void);
+static void set_stream_options(WalRcvStreamOptions *options,
+    char *slotname,
+    XLogRecPtr *origin_startpos);

~

Maybe a blank line was needed here because this static should not be
grouped with the other functions that are grouped for "Serialize and
deserialize changes for a toplevel transaction." comment.

~~~

4. set_stream_options

+ /* set_stream_options
+  * Set logical replication streaming options.
+  *
+  * This function sets streaming options including replication slot name and
+  * origin start position. Workers need these options for logical replication.
+  */
+static void
+set_stream_options(WalRcvStreamOptions *options,

The indentation is not right for this function comment.

~~~

5. set_stream_options

+ /*
+ * Even when the two_phase mode is requested by the user, it remains as
+ * the tri-state PENDING until all tablesyncs have reached READY state.
+ * Only then, can it become ENABLED.
+ *
+ * Note: If the subscription has no tables then leave the state as
+ * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+ * work.
+ */
+ if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+ AllTablesyncsReady())
+ options->proto.logical.twophase = true;
+}

This part of the refactoring seems questionable...

IIUC this new function was extracted from code in originally in
function ApplyWorkerMain()

But in that original code, this fragment above was guarded by the condition
if (!am_tablesync_worker())

But now where is that condition? e.g. What is stopping tablesync
working from getting into this code it previously would not have
executed?

~~~

6.
  AbortOutOfAnyTransaction();
- pgstat_report_subscription_error(MySubscription->oid, !am_tablesync_worker());
+ pgstat_report_subscription_error(MySubscription->oid,
+ !am_tablesync_worker());

Does this change have anything to do with this patch? Is it a quirk of
running pg_indent?

~~~
7. run_tablesync_worker

Since the stated intent of the patch is the separation of apply and
tablesync workers then shouldn't this function belong in the
tablesync.c file?

~~~
8. run_tablesync_worker

+ * Runs the tablesync worker.
+ * It starts syncing tables. After a successful sync, sets streaming options
+ * and starts streaming to catchup.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,

Nicer to have a blank line after the first sentence of that function comment?

~~~
9. run_apply_worker

+/*
+ * Runs the leader apply worker.
+ * It sets up replication origin, streaming options and then starts streaming.
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,

Nicer to have a blank line after the first sentence of that function comment?

~~~
10. InitializeLogRepWorker

+/*
+ * Common initialization for logical replication workers; leader apply worker,
+ * parallel apply worker and tablesync worker.
  *
  * Initialize the database connection, in-memory subscription and necessary
  * config options.
  */
 void
-InitializeApplyWorker(void)
+InitializeLogRepWorker(void)

typo:

/workers;/workers:/

~~~
11. TablesyncWorkerMain

Since the stated intent of the patch is the separation of apply and
tablesync workers then shouldn't this function belong in the
tablesync.c file?

======
src/include/replication/worker_internal.h

12.
#define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)

+extern void finish_sync_worker(void);

~

I think the macro isParallelApplyWorker is associated with the am_XXX
inline functions that follow it, so it doesn’t seem the best place to
jam this extern in the middle of that.

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#87Hayato Kuroda (Fujitsu)
kuroda.hayato@fujitsu.com
In reply to: Hayato Kuroda (Fujitsu) (#85)
3 attachment(s)
RE: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Dear hackers,

Hi, I did a performance testing for v16 patch set.
Results show that patches significantly improves the performance in most cases.

# Method

Following tests were done 10 times per condition, and compared by median.
do_one_test.sh was used for the testing.

1.	Create tables on publisher
2.	Insert initial data on publisher
3.	Create tables on subscriber
4.	Create a replication slot (mysub_slot) on publisher
5.	Create a publication on publisher
6.	Create tables on subscriber
--- timer on ---
7.	Create subscription with pre-existing replication slot (mysub_slot)
8.	Wait until all srsubstate in pg_subscription_rel becomes 'r'
--- timer off ---

# Tested sources

I used three types of sources

* HEAD (f863d82)
* HEAD + 0001 + 0002
* HEAD + 0001 + 0002 + 0003

# Tested conditions

Following parameters were changed during the measurement.

### table size

* empty
* around 10kB

### number of tables

* 10
* 100
* 1000
* 2000

### max_sync_workers_per_subscription

* 2
* 4
* 8
* 16

## Results

Please see the attached image file. Each cell shows the improvement percentage of
measurement comapred with HEAD, HEAD + 0001 + 0002, and HEAD + 0001 + 0002 + 0003.

According to the measurement, we can say following things:

* In any cases the performance was improved from the HEAD.
* The improvement became more significantly if number of synced tables were increased.
* 0003 basically improved performance from first two patches
* Increasing workers could sometimes lead to lesser performance due to contention.
This was occurred when the number of tables were small. Moreover, this was not only happen by patchset - it happened even if we used HEAD.
Detailed analysis will be done later.

Mored deital, please see the excel file. It contains all the results of measurement.

## Detailed configuration

* Powerful machine was used:
- Number of CPU: 120
- Memory: 755 GB

* Both publisher and subscriber were on the same machine.
* Following GUC settings were used for both pub/sub:

```
wal_level = logical
shared_buffers = 40GB
max_worker_processes = 32
max_parallel_maintenance_workers = 24
max_parallel_workers = 32
synchronous_commit = off
checkpoint_timeout = 1d
max_wal_size = 24GB
min_wal_size = 15GB
autovacuum = off
max_wal_senders = 200
max_replication_slots = 200
```

Best Regards,
Hayato Kuroda
FUJITSU LIMITED

Attachments:

result_table.pngimage/png; name=result_table.pngDownload
perftest_result.xlsxapplication/vnd.openxmlformats-officedocument.spreadsheetml.sheet; name=perftest_result.xlsxDownload
do_one_test.shapplication/octet-stream; name=do_one_test.shDownload
#88Peter Smith
smithpb2250@gmail.com
In reply to: Hayato Kuroda (Fujitsu) (#85)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi, here are some review comments for patch v16-0002.

======
Commit message

1.
This commit allows reusing tablesync workers for syncing more than one
table sequentially during their lifetime, instead of exiting after
only syncing one table.

Before this commit, tablesync workers were capable of syncing only one
table. For each table, a new sync worker was launched and that worker would
exit when done processing the table.

Now, tablesync workers are not limited to processing only one
table. When done, they can move to processing another table in
the same subscription.

~

IMO that first paragraph can be removed because AFAIK the other
paragraphs are saying exactly the same thing but worded differently.

======
src/backend/replication/logical/tablesync.c

2. General -- for clean_sync_worker and finish_sync_worker

TBH, I found the separation of clean_sync_worker() and
finish_sync_worker() to be confusing. Can't it be rearranged to keep
the same function but just pass a boolean to tell it to exit or not
exit?

e.g.

finish_sync_worker(bool reuse_worker) { ... }

~~~

3. clean_sync_worker

  /*
- * Commit any outstanding transaction. This is the usual case, unless
- * there was nothing to do for the table.
+ * Commit any outstanding transaction. This is the usual case, unless there
+ * was nothing to do for the table.
  */

The word wrap seems OK, except the change seemed unrelated to this patch (??)

~~~

4.
+ /*
+ * Disconnect from publisher. Otherwise reused sync workers causes
+ * exceeding max_wal_senders
+ */

Missing period, and not an English sentence.

SUGGESTION (??)
Disconnect from the publisher otherwise reusing the sync worker can
error due to exceeding max_wal_senders.

~~~

5. finish_sync_worker

+/*
+ * Exit routine for synchronization worker.
+ */
+void
+pg_attribute_noreturn()
+finish_sync_worker(void)
+{
+ clean_sync_worker();
+
  /* And flush all writes. */
  XLogFlush(GetXLogWriteRecPtr());
  StartTransactionCommand();
  ereport(LOG,
- (errmsg("logical replication table synchronization worker for
subscription \"%s\", table \"%s\" has finished",
- MySubscription->name,
- get_rel_name(MyLogicalRepWorker->relid))));
+ (errmsg("logical replication table synchronization worker for
subscription \"%s\" has finished",
+ MySubscription->name)));
  CommitTransactionCommand();

In the original code, the XLogFlush was in a slightly different order
than in this refactored code. E.g. it came before signalling the apply
worker. Is it OK to be changed?

Keeping one function (suggested in #2) can maybe remove this potential issue.

======
src/backend/replication/logical/worker.c

6. LogicalRepApplyLoop

+ /*
+ * apply_dispatch() may have gone into apply_handle_commit()
+ * which can call process_syncing_tables_for_sync.
+ *
+ * process_syncing_tables_for_sync decides whether the sync of
+ * the current table is completed. If it is completed,
+ * streaming must be already ended. So, we can break the loop.
+ */
+ if (MyLogicalRepWorker->is_sync_completed)
+ {
+ endofstream = true;
+ break;
+ }
+

and

+ /*
+ * If is_sync_completed is true, this means that the tablesync
+ * worker is done with synchronization. Streaming has already been
+ * ended by process_syncing_tables_for_sync. We should move to the
+ * next table if needed, or exit.
+ */
+ if (MyLogicalRepWorker->is_sync_completed)
+ endofstream = true;

~

Instead of those code fragments above assigning 'endofstream' as a
side-effect, would it be the same (but tidier) to just modify the
other "breaking" condition below:

BEFORE:
/* Check if we need to exit the streaming loop. */
if (endofstream)
break;

AFTER:
/* Check if we need to exit the streaming loop. */
if (endofstream || MyLogicalRepWorker->is_sync_completed)
break;

~~~

7. LogicalRepApplyLoop

+ /*
+ * Tablesync workers should end streaming before exiting the main loop to
+ * drop replication slot. Only end streaming here for apply workers.
+ */
+ if (!am_tablesync_worker())
+ walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);

This comment does not seem very clear. Maybe it can be reworded:

SUGGESTION
End streaming here only for apply workers. Ending streaming for
tablesync workers is deferred until ... because ...

~~~

8. TablesyncWorkerMain

+ StartTransactionCommand();
+ ereport(LOG,
+ (errmsg("%s for subscription \"%s\" has moved to sync table \"%s\"
with relid %u.",
+ get_worker_name(),
+ MySubscription->name,
+ get_rel_name(MyLogicalRepWorker->relid),
+ MyLogicalRepWorker->relid)));
+ CommitTransactionCommand();

The "has moved to..." terminology is unusual. If you say something
"will be reused to..." then it matches better the commit message etc.

~~~

9.

+ if (!is_table_found)
+ break;

Instead of an infinite loop that is exited by this 'break' it might be
better to rearrange the logic slightly so the 'for' loop can exit
normally:

BEFORE:
for (;;)

AFTER
for (; !done;)

======
src/include/replication/worker_internal.h

10.
XLogRecPtr relstate_lsn;
slock_t relmutex;

+ /*
+ * Indicates whether tablesync worker has completed sycning its assigned
+ * table. If true, no need to continue with that table.
+ */
+ bool is_sync_completed;
+

10a.
Typo /sycning/syncing/

~

10b.
All the other tablesync-related fields of this struct are named as
relXXX, so I wonder if is better for this to follow the same pattern.
e.g. 'relsync_completed'

~

10c.
"If true, no need to continue with that table.".

I am not sure if this sentence is adding anything useful.

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#89Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Amit Kapila (#84)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi,

Amit Kapila <amit.kapila16@gmail.com>, 6 Tem 2023 Per, 06:56 tarihinde
şunu yazdı:

On Wed, Jul 5, 2023 at 1:48 AM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hayato Kuroda (Fujitsu) <kuroda.hayato@fujitsu.com>, 4 Tem 2023 Sal,
08:42 tarihinde şunu yazdı:

But in the later patch the tablesync worker tries to reuse the slot during the
synchronization, so in this case the application_name should be same as

slotname.

Fair enough. I am slightly afraid that if we can't show the benefits
with later patches then we may need to drop them but at this stage I
feel we need to investigate why those are not helping?

Agreed. Now I'm planning to do performance testing independently. We can discuss
based on that or Melih's one.

Here I attached what I use for performance testing of this patch.

I only benchmarked the patch set with reusing connections very roughly
so far. But seems like it improves quite significantly. For example,
it took 611 ms to sync 100 empty tables, it was 1782 ms without
reusing connections.
First 3 patches from the set actually bring a good amount of
improvement, but not sure about the later patches yet.

I suggest then we should focus first on those 3, get them committed
and then look at the remaining.

That sounds good. I'll do my best to address any review/concern from
reviewers now for the first 3 patches and hopefully those can get
committed first. I'll continue working on the remaining patches later.

--
Melih Mutlu
Microsoft

#90Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Hayato Kuroda (Fujitsu) (#85)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi,

Hayato Kuroda (Fujitsu) <kuroda.hayato@fujitsu.com>, 6 Tem 2023 Per,
12:47 tarihinde şunu yazdı:

Dear Melih,

Thanks for the 0003 patch. But it did not work for me. Can you create
a subscription successfully with patch 0003 applied?
I get the following error: " ERROR: table copy could not start
transaction on publisher: another command is already in progress".

You got the ERROR when all the patches (0001-0005) were applied, right?
I have focused on 0001 and 0002 only, so I missed something.
If it was not correct, please attach the logfile and test script what you did.

Yes, I did get an error with all patches applied. But with only 0001
and 0002, your version seems like working and mine does not.
What do you think about combining 0002 and 0003? Or should those stay separate?

Hi, I did a performance testing for v16 patch set.
Results show that patches significantly improves the performance in most cases.

# Method

Following tests were done 10 times per condition, and compared by median.
do_one_test.sh was used for the testing.

1.      Create tables on publisher
2.      Insert initial data on publisher
3.      Create tables on subscriber
4.      Create a replication slot (mysub_slot) on publisher
5.      Create a publication on publisher
6.      Create tables on subscriber
--- timer on ---
7.      Create subscription with pre-existing replication slot (mysub_slot)
8.      Wait until all srsubstate in pg_subscription_rel becomes 'r'
--- timer off ---

Thanks for taking the time to do testing and sharing the results. This
is also how I've been doing the testing since, but the process was
half scripted, half manual work.

According to the measurement, we can say following things:

* In any cases the performance was improved from the HEAD.
* The improvement became more significantly if number of synced tables were increased.

Yes, I believe it becomes more significant when workers spend less
time with actually copying data but more with other stuff like
launching workers, opening connections etc.

* 0003 basically improved performance from first two patches

Agree, 0003 is definitely a good addition which was missing earlier.

Thanks,
--
Melih Mutlu
Microsoft

#91Peter Smith
smithpb2250@gmail.com
In reply to: Hayato Kuroda (Fujitsu) (#85)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Here are some review comments for patch v16-00003

======
1. Commit Message.

The patch description is missing.

======
2. General.

+LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)

and

+start_table_sync(XLogRecPtr *origin_startpos,
+ char **myslotname,
+ int worker_slot)

and

@@ -4548,12 +4552,13 @@ run_tablesync_worker(WalRcvStreamOptions *options,
  char *slotname,
  char *originname,
  int originname_size,
- XLogRecPtr *origin_startpos)
+ XLogRecPtr *origin_startpos,
+ int worker_slot)

It seems the worker_slot is being passed all over the place as an
additional function argument so that it can be used to construct an
application_name. Is it possible/better to introduce a new
'MyLogicalRepWorker' field for the 'worker_slot' so it does not have
to be passed like this?

======
src/backend/replication/logical/tablesync.c

3.
+ /*
+ * Disconnect from publisher. Otherwise reused sync workers causes
+ * exceeding max_wal_senders.
+ */
+ if (LogRepWorkerWalRcvConn != NULL)
+ {
+ walrcv_disconnect(LogRepWorkerWalRcvConn);
+ LogRepWorkerWalRcvConn = NULL;
+ }
+

Why is this comment mentioning anything about "reused workers" at all?
The worker process exits in this function, right?

~~~

4. LogicalRepSyncTableStart

  /*
- * Here we use the slot name instead of the subscription name as the
- * application_name, so that it is different from the leader apply worker,
- * so that synchronous replication can distinguish them.
+ * Connect to publisher if not yet. The application_name must be also
+ * different from the leader apply worker because synchronous replication
+ * must distinguish them.
  */

I felt all the details in the 2nd part of this comment belong inside
the condition, not outside.

SUGGESTION
/* Connect to the publisher if haven't done so already. */

~~~

5.
+ if (LogRepWorkerWalRcvConn == NULL)
+ {
+ char application_name[NAMEDATALEN];
+
+ /*
+ * FIXME: set appropriate application_name. Previously, the slot name
+ * was used becasue the lifetime of the tablesync worker was same as
+ * that, but now the tablesync worker handles many slots during the
+ * synchronization so that it is not suitable. So what should be?
+ * Note that if the tablesync worker starts to reuse the replication
+ * slot during synchronization, we should use the slot name as
+ * application_name again.
+ */
+ snprintf(application_name, NAMEDATALEN, "pg_%u_sync_%i",
+ MySubscription->oid, worker_slot);
+ LogRepWorkerWalRcvConn =
+ walrcv_connect(MySubscription->conninfo, true,
+    must_use_password,
+    application_name, &err);
+ }

5a.
/becasue/because/

~

5b.
I am not sure about what name this should ideally use, but anyway for
uniqueness doesn't it still need to include the GetSystemIdentifier()
same as function ReplicationSlotNameForTablesync() was doing?

Maybe this can use the same function ReplicationSlotNameForTablesync()
can be used but just pass the worker_slot instead of the relid?

======
src/backend/replication/logical/worker.c

6. LogicalRepApplyLoop

  /*
  * Init the ApplyMessageContext which we clean up after each replication
- * protocol message.
+ * protocol message, if needed.
  */
- ApplyMessageContext = AllocSetContextCreate(ApplyContext,
- "ApplyMessageContext",
- ALLOCSET_DEFAULT_SIZES);
+ if (!ApplyMessageContext)
+ ApplyMessageContext = AllocSetContextCreate(ApplyContext,
+ "ApplyMessageContext",
+

Maybe slightly reword the comment.

BEFORE:
Init the ApplyMessageContext which we clean up after each replication
protocol message, if needed.

AFTER:
Init the ApplyMessageContext if needed. This context is cleaned up
after each replication protocol message.

======
src/backend/replication/walsender.c

7.
+ /*
+ * Initialize the flag again because this streaming may be
+ * second time.
+ */
+ streamingDoneSending = streamingDoneReceiving = false;

Isn't this only possible to be 2nd time because the "reuse tablesync
worker" might re-issue a START_REPLICATION again to the same
WALSender? So, should this flag reset ONLY be done for the logical
replication ('else' part), otherwise it should be asserted false?

e.g. Would it be better to be like this?

if (cmd->kind == REPLICATION_KIND_PHYSICAL)
{
Assert(!streamingDoneSending && !streamingDoneReceiving)
StartReplication(cmd);
}
else
{
/* Reset flags because reusing tablesync workers can mean this is the
second time here. */
streamingDoneSending = streamingDoneReceiving = false;
StartLogicalReplication(cmd);
}

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#92Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#90)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Tue, Jul 11, 2023 at 12:31 AM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi,

Hayato Kuroda (Fujitsu) <kuroda.hayato@fujitsu.com>, 6 Tem 2023 Per,
12:47 tarihinde şunu yazdı:

Dear Melih,

Thanks for the 0003 patch. But it did not work for me. Can you create
a subscription successfully with patch 0003 applied?
I get the following error: " ERROR: table copy could not start
transaction on publisher: another command is already in progress".

You got the ERROR when all the patches (0001-0005) were applied, right?
I have focused on 0001 and 0002 only, so I missed something.
If it was not correct, please attach the logfile and test script what you did.

Yes, I did get an error with all patches applied. But with only 0001
and 0002, your version seems like working and mine does not.
What do you think about combining 0002 and 0003? Or should those stay separate?

Even if patches 0003 and 0002 are to be combined, I think that should
not happen until after the "reuse" design is confirmed which way is
best.

e.g. IMO it might be easier to compare the different PoC designs for
patch 0002 if there is no extra logic involved.

PoC design#1 -- each tablesync decides for itself what to do next
after it finishes
PoC design#2 -- reuse tablesync using a "pool" of available workers

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#93Amit Kapila
amit.kapila16@gmail.com
In reply to: Melih Mutlu (#90)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Mon, Jul 10, 2023 at 8:01 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hayato Kuroda (Fujitsu) <kuroda.hayato@fujitsu.com>, 6 Tem 2023 Per,
12:47 tarihinde şunu yazdı:

Dear Melih,

Thanks for the 0003 patch. But it did not work for me. Can you create
a subscription successfully with patch 0003 applied?
I get the following error: " ERROR: table copy could not start
transaction on publisher: another command is already in progress".

You got the ERROR when all the patches (0001-0005) were applied, right?
I have focused on 0001 and 0002 only, so I missed something.
If it was not correct, please attach the logfile and test script what you did.

Yes, I did get an error with all patches applied. But with only 0001
and 0002, your version seems like working and mine does not.
What do you think about combining 0002 and 0003? Or should those stay separate?

I am fine either way but I think one minor advantage of keeping 0003
separate is that we can focus on some of the problems specific to that
patch. For example, the following comment in the 0003 patch: "FIXME:
set appropriate application_name...". I have given a suggestion to
address it in [1]/messages/by-id/CAA4eK1JOZHmy2o2F2wTCPKsjpwDiKZPOeTa_jt=wm2JLbf-jsg@mail.gmail.com and Kuroda-San seems to have addressed the same but
I am not sure if all of us agree with that or if there is any better
way to address it. What do you think?

* 0003 basically improved performance from first two patches

Agree, 0003 is definitely a good addition which was missing earlier.

+1.

[1]: /messages/by-id/CAA4eK1JOZHmy2o2F2wTCPKsjpwDiKZPOeTa_jt=wm2JLbf-jsg@mail.gmail.com

--
With Regards,
Amit Kapila.

#94Hayato Kuroda (Fujitsu)
kuroda.hayato@fujitsu.com
In reply to: Melih Mutlu (#90)
RE: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Dear Melih,

Thanks for the 0003 patch. But it did not work for me. Can you create
a subscription successfully with patch 0003 applied?
I get the following error: " ERROR: table copy could not start
transaction on publisher: another command is already in progress".

You got the ERROR when all the patches (0001-0005) were applied, right?
I have focused on 0001 and 0002 only, so I missed something.
If it was not correct, please attach the logfile and test script what you did.

Yes, I did get an error with all patches applied. But with only 0001
and 0002, your version seems like working and mine does not.

Hmm, really? IIUC I did not modify 0001 and 0002 patches, I just re-assigned the
version number. I compared between yours and mine, but no meaningful differences
were found. E.g., following command compared v4-0002 and v16-0002:

```
diff --git a/../reuse_workers/v4-0002-Reuse-Tablesync-Workers.patch b/../reuse_workers/hayato/v16-0002-Reuse-Tablesync-Workers.patch
index 5350216e98..7785a573e4 100644
--- a/../reuse_workers/v4-0002-Reuse-Tablesync-Workers.patch
+++ b/../reuse_workers/hayato/v16-0002-Reuse-Tablesync-Workers.patch
@@ -1,7 +1,7 @@
-From d482022b40e0a5ce1b74fd0e320cb5b45da2f671 Mon Sep 17 00:00:00 2001
+From db3e8e2d7aadea79126c5816bce8b06dc82f33c2 Mon Sep 17 00:00:00 2001
 From: Melih Mutlu <m.melihmutlu@gmail.com>
 Date: Tue, 4 Jul 2023 22:04:46 +0300
-Subject: [PATCH 2/5] Reuse Tablesync Workers
+Subject: [PATCH v16 2/5] Reuse Tablesync Workers

This commit allows reusing tablesync workers for syncing more than one
table sequentially during their lifetime, instead of exiting after
@@ -324,5 +324,5 @@ index 7aba034774..1e9f8e6e72 100644
static inline bool
am_tablesync_worker(void)
--
-2.25.1
+2.27.0
```

For confirmation, please attach the logfile and test script what you did
if you could reproduce?

What do you think about combining 0002 and 0003? Or should those stay
separate?

I have no strong opinion, but it may be useful to keep them pluggable.

Best Regards,
Hayato Kuroda
FUJITSU LIMITED

#95Hayato Kuroda (Fujitsu)
kuroda.hayato@fujitsu.com
In reply to: Peter Smith (#91)
5 attachment(s)
RE: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Dear Peter,

Thanks for reviewing! I'm not sure what should be, but I modified only my part - 0003.
PSA new patchset. Other patches were not changed.
(I attached till 0005 just in case, but I did not consider about 0004 and 0005)

======
1. Commit Message.

The patch description is missing.

Briefly added.

2. General.

+LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)

and

+start_table_sync(XLogRecPtr *origin_startpos,
+ char **myslotname,
+ int worker_slot)

and

@@ -4548,12 +4552,13 @@ run_tablesync_worker(WalRcvStreamOptions
*options,
char *slotname,
char *originname,
int originname_size,
- XLogRecPtr *origin_startpos)
+ XLogRecPtr *origin_startpos,
+ int worker_slot)

It seems the worker_slot is being passed all over the place as an
additional function argument so that it can be used to construct an
application_name. Is it possible/better to introduce a new
'MyLogicalRepWorker' field for the 'worker_slot' so it does not have
to be passed like this?

I'm not sure it should be, but I did. How do you think?

src/backend/replication/logical/tablesync.c

3.
+ /*
+ * Disconnect from publisher. Otherwise reused sync workers causes
+ * exceeding max_wal_senders.
+ */
+ if (LogRepWorkerWalRcvConn != NULL)
+ {
+ walrcv_disconnect(LogRepWorkerWalRcvConn);
+ LogRepWorkerWalRcvConn = NULL;
+ }
+

Why is this comment mentioning anything about "reused workers" at all?
The worker process exits in this function, right?

I considered that code again, and I found this part is not needed anymore.

Initially it was added in 0002, this is because workers established new connections
without exiting and walsenders on publisher might be remained. So This was correct
for 0002 patch.
But now, in 0003 patch, workers reuse connections, which means that no need to call
walrcv_disconnect() explicitly. It is done when processes are exit.

4. LogicalRepSyncTableStart

/*
- * Here we use the slot name instead of the subscription name as the
- * application_name, so that it is different from the leader apply worker,
- * so that synchronous replication can distinguish them.
+ * Connect to publisher if not yet. The application_name must be also
+ * different from the leader apply worker because synchronous replication
+ * must distinguish them.
*/

I felt all the details in the 2nd part of this comment belong inside
the condition, not outside.

SUGGESTION
/* Connect to the publisher if haven't done so already. */

Changed.

5.
+ if (LogRepWorkerWalRcvConn == NULL)
+ {
+ char application_name[NAMEDATALEN];
+
+ /*
+ * FIXME: set appropriate application_name. Previously, the slot name
+ * was used becasue the lifetime of the tablesync worker was same as
+ * that, but now the tablesync worker handles many slots during the
+ * synchronization so that it is not suitable. So what should be?
+ * Note that if the tablesync worker starts to reuse the replication
+ * slot during synchronization, we should use the slot name as
+ * application_name again.
+ */
+ snprintf(application_name, NAMEDATALEN, "pg_%u_sync_%i",
+ MySubscription->oid, worker_slot);
+ LogRepWorkerWalRcvConn =
+ walrcv_connect(MySubscription->conninfo, true,
+    must_use_password,
+    application_name, &err);
+ }

5a.
/becasue/because/

Modified. Also, comments were moved atop ApplicationNameForTablesync.
I was not sure when it is removed, but I kept it.

5b.
I am not sure about what name this should ideally use, but anyway for
uniqueness doesn't it still need to include the GetSystemIdentifier()
same as function ReplicationSlotNameForTablesync() was doing?

Maybe this can use the same function ReplicationSlotNameForTablesync()
can be used but just pass the worker_slot instead of the relid?

Good point. ApplicationNameForTablesync() was defined and used.

src/backend/replication/logical/worker.c

6. LogicalRepApplyLoop

/*
* Init the ApplyMessageContext which we clean up after each replication
- * protocol message.
+ * protocol message, if needed.
*/
- ApplyMessageContext = AllocSetContextCreate(ApplyContext,
- "ApplyMessageContext",
- ALLOCSET_DEFAULT_SIZES);
+ if (!ApplyMessageContext)
+ ApplyMessageContext = AllocSetContextCreate(ApplyContext,
+ "ApplyMessageContext",
+

Maybe slightly reword the comment.

BEFORE:
Init the ApplyMessageContext which we clean up after each replication
protocol message, if needed.

AFTER:
Init the ApplyMessageContext if needed. This context is cleaned up
after each replication protocol message.

Changed.

src/backend/replication/walsender.c

7.
+ /*
+ * Initialize the flag again because this streaming may be
+ * second time.
+ */
+ streamingDoneSending = streamingDoneReceiving = false;

Isn't this only possible to be 2nd time because the "reuse tablesync
worker" might re-issue a START_REPLICATION again to the same
WALSender? So, should this flag reset ONLY be done for the logical
replication ('else' part), otherwise it should be asserted false?

e.g. Would it be better to be like this?

if (cmd->kind == REPLICATION_KIND_PHYSICAL)
{
Assert(!streamingDoneSending && !streamingDoneReceiving)
StartReplication(cmd);
}
else
{
/* Reset flags because reusing tablesync workers can mean this is the
second time here. */
streamingDoneSending = streamingDoneReceiving = false;
StartLogicalReplication(cmd);
}

It's OK to modify the comment. But after considering more, I started to think that
any specification for physical replication should not be changed.
So I accepted comments only for the logical rep.

Best Regards,
Hayato Kuroda
FUJITSU LIMITED

Attachments:

v17-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchapplication/octet-stream; name=v17-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchDownload
From df3c09b1d32f2804adb2256f61db823843837f75 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Mon, 5 Jun 2023 15:04:41 +0300
Subject: [PATCH v17 1/5] Refactor to split Apply and Tablesync Workers

Both apply and tablesync workers were using ApplyWorkerMain() as entry
point. As the name implies, ApplyWorkerMain() should be considered as
the main function for apply workers. Tablesync worker's path was hidden
and does not have enough in common to share the same main function with
apply worker.

Also; most of the code shared by both worker types are already combined
in LogicalRepApplyLoop(). There is no need to combine the rest in
ApplyWorkerMain() anymore.

This commit introduces TablesyncWorkerMain() as a new entry point for
tablesync workers and separates both type of workers from each other.
This aims to increase code readability and help to maintain logical
replication workers separately.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/postmaster/bgworker.c             |   3 +
 .../replication/logical/applyparallelworker.c |   2 +-
 src/backend/replication/logical/launcher.c    |  32 +-
 src/backend/replication/logical/tablesync.c   |   2 +-
 src/backend/replication/logical/worker.c      | 388 +++++++++++-------
 src/include/replication/logicalworker.h       |   1 +
 src/include/replication/worker_internal.h     |   4 +-
 7 files changed, 270 insertions(+), 162 deletions(-)

diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 5b4bd71694..505e38376c 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -131,6 +131,9 @@ static const struct
 	},
 	{
 		"ParallelApplyWorkerMain", ParallelApplyWorkerMain
+	},
+	{
+		"TablesyncWorkerMain", TablesyncWorkerMain
 	}
 };
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 6fb96148f4..1d4e83c4c1 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -942,7 +942,7 @@ ParallelApplyWorkerMain(Datum main_arg)
 	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
 		MyLogicalRepWorker->reply_time = 0;
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 542af7d863..e231fa7f95 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -459,24 +459,30 @@ retry:
 	snprintf(bgw.bgw_library_name, MAXPGPATH, "postgres");
 
 	if (is_parallel_apply_worker)
+	{
 		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ParallelApplyWorkerMain");
-	else
-		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
-
-	if (OidIsValid(relid))
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication worker for subscription %u sync %u", subid, relid);
-	else if (is_parallel_apply_worker)
+				 "logical replication parallel apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
+	}
+	else if (OidIsValid(relid))
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "TablesyncWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication parallel apply worker for subscription %u", subid);
+				 "logical replication tablesync worker for subscription %u sync %u",
+				 subid,
+				 relid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication tablesync worker");
+	}
 	else
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication apply worker for subscription %u", subid);
-
-	if (is_parallel_apply_worker)
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
-	else
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication worker");
+				 "logical replication apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication apply worker");
+	}
 
 	bgw.bgw_restart_time = BGW_NEVER_RESTART;
 	bgw.bgw_notify_pid = MyProcPid;
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 6d461654ab..8125bbd170 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -131,7 +131,7 @@ static StringInfo copybuf = NULL;
 /*
  * Exit routine for synchronization worker.
  */
-static void
+void
 pg_attribute_noreturn()
 finish_sync_worker(void)
 {
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 0ee764d68f..96ddad356b 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -392,6 +392,9 @@ static void stream_open_file(Oid subid, TransactionId xid,
 static void stream_write_change(char action, StringInfo s);
 static void stream_open_and_write_change(TransactionId xid, char action, StringInfo s);
 static void stream_close_file(void);
+static void set_stream_options(WalRcvStreamOptions *options,
+							   char *slotname,
+							   XLogRecPtr *origin_startpos);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
@@ -4330,6 +4333,72 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+ /* set_stream_options
+  * 	Set logical replication streaming options.
+  *
+  * This function sets streaming options including replication slot name and
+  * origin start position. Workers need these options for logical replication.
+  */
+static void
+set_stream_options(WalRcvStreamOptions *options,
+				   char *slotname,
+				   XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+		options->proto.logical.twophase = true;
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4433,7 +4502,8 @@ start_apply(XLogRecPtr origin_startpos)
 			 * idle state.
 			 */
 			AbortOutOfAnyTransaction();
-			pgstat_report_subscription_error(MySubscription->oid, !am_tablesync_worker());
+			pgstat_report_subscription_error(MySubscription->oid,
+											 !am_tablesync_worker());
 
 			PG_RE_THROW();
 		}
@@ -4442,13 +4512,133 @@ start_apply(XLogRecPtr origin_startpos)
 }
 
 /*
- * Common initialization for leader apply worker and parallel apply worker.
+ * Runs the tablesync worker.
+ * It starts syncing tables. After a successful sync, sets streaming options
+ * and starts streaming to catchup.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Start table synchronization. */
+	start_table_sync(origin_startpos, &slotname);
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   originname_size);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	/* Start applying changes to catchup. */
+	start_apply(*origin_startpos);
+}
+
+/*
+ * Runs the leader apply worker.
+ * It sets up replication origin, streaming options and then starts streaming.
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
+{
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+	bool		must_use_password;
+
+	slotname = MySubscription->slotname;
+
+	/*
+	 * This shouldn't happen if the subscription is enabled, but guard
+	 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+	 * crash if slot is NULL.)
+	 */
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+
+	/* Is the use of a password mandatory? */
+	must_use_password = MySubscription->passwordrequired &&
+		!superuser_arg(MySubscription->owner);
+
+	/* Note that the superuser_arg call can access the DB */
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											must_use_password,
+											MySubscription->name, &err);
+
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+	 * We don't really use the output identify_system for anything but it
+	 * does some initializations on the upstream so let's still call it.
+	 */
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+	{
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
+	}
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+
+	/* Run the main loop. */
+	start_apply(*origin_startpos);
+}
+
+/*
+ * Common initialization for logical replication workers; leader apply worker,
+ * parallel apply worker and tablesync worker.
  *
  * Initialize the database connection, in-memory subscription and necessary
  * config options.
  */
 void
-InitializeApplyWorker(void)
+InitializeLogRepWorker(void)
 {
 	MemoryContext oldctx;
 
@@ -4512,7 +4702,8 @@ InitializeApplyWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has started",
+				(errmsg("%s for subscription \"%s\", table \"%s\" has started",
+						get_worker_name(),
 						MySubscription->name,
 						get_rel_name(MyLogicalRepWorker->relid))));
 	else
@@ -4533,7 +4724,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	InitializingApplyWorker = true;
 
@@ -4557,7 +4747,7 @@ ApplyWorkerMain(Datum main_arg)
 	/* Load the libpq-specific functions */
 	load_file("libpqwalreceiver", false);
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
@@ -4565,71 +4755,6 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-		bool		must_use_password;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-
-		/* Is the use of a password mandatory? */
-		must_use_password = MySubscription->passwordrequired &&
-			!superuser_arg(MySubscription->owner);
-
-		/* Note that the superuser_arg call can access the DB */
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												must_use_password,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
 	 * the subscription relation state.
@@ -4638,92 +4763,63 @@ ApplyWorkerMain(Datum main_arg)
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
+	/* This is leader apply worker */
+	run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
+	proc_exit(0);
+}
 
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
+/* Logical Replication Tablesync worker entry point */
+void
+TablesyncWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *myslotname = NULL;
+	WalRcvStreamOptions options;
+
+	/* Attach to slot */
+	logicalrep_worker_attach(worker_slot);
+
+	/* Setup signal handling */
+	pqsignal(SIGHUP, SignalHandlerForConfigReload);
+	pqsignal(SIGTERM, die);
+	BackgroundWorkerUnblockSignals();
 
 	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 * We don't currently need any ResourceOwner in a walreceiver process, but
+	 * if we did, we could call CreateAuxProcessResourceOwner here.
 	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
 
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
+	/* Initialise stats to a sanish value */
+	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
+		MyLogicalRepWorker->reply_time = GetCurrentTimestamp();
 
-	if (!am_tablesync_worker())
-	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
-		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+	/* Load the libpq-specific functions */
+	load_file("libpqwalreceiver", false);
 
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
-		}
-		else
-		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-		}
+	InitializeLogRepWorker();
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-								 MySubscription->name,
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-								 "?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
+	/* Connect to the origin and start the replication. */
+	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
+		 MySubscription->conninfo);
 
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+	/*
+	 * Setup callback for syscache so that we know when something changes in
+	 * the subscription relation state.
+	 */
+	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
+								  invalidate_syncing_table_states,
+								  (Datum) 0);
 
-	proc_exit(0);
+	run_tablesync_worker(&options,
+						 myslotname,
+						 originname,
+						 sizeof(originname),
+						 &origin_startpos);
+
+	finish_sync_worker();
 }
 
 /*
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 39588da79f..bbd71d0b42 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -18,6 +18,7 @@ extern PGDLLIMPORT volatile sig_atomic_t ParallelApplyMessagePending;
 
 extern void ApplyWorkerMain(Datum main_arg);
 extern void ParallelApplyWorkerMain(Datum main_arg);
+extern void TablesyncWorkerMain(Datum main_arg);
 
 extern bool IsLogicalWorker(void);
 extern bool IsLogicalParallelApplyWorker(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 343e781896..7aba034774 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -265,7 +265,7 @@ extern void maybe_reread_subscription(void);
 
 extern void stream_cleanup_files(Oid subid, TransactionId xid);
 
-extern void InitializeApplyWorker(void);
+extern void InitializeLogRepWorker(void);
 
 extern void store_flush_position(XLogRecPtr remote_lsn, XLogRecPtr local_lsn);
 
@@ -307,6 +307,8 @@ extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
 
 #define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)
 
+extern void finish_sync_worker(void);
+
 static inline bool
 am_tablesync_worker(void)
 {
-- 
2.27.0

v17-0002-Reuse-Tablesync-Workers.patchapplication/octet-stream; name=v17-0002-Reuse-Tablesync-Workers.patchDownload
From f1ab9af55b404420ec55c648d48af411acd1f3cf Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:04:46 +0300
Subject: [PATCH v17 2/5] Reuse Tablesync Workers

This commit allows reusing tablesync workers for syncing more than one
table sequentially during their lifetime, instead of exiting after
only syncing one table.

Before this commit, tablesync workers were capable of syncing only one
table. For each table, a new sync worker was launched and that worker would
exit when done processing the table.

Now, tablesync workers are not limited to processing only one
table. When done, they can move to processing another table in
the same subscription.

If there is a table that needs to be synced, an available tablesync
worker picks up that table and syncs it. Each tablesync worker
continues to pick new tables to sync until there are no tables left
requiring synchronization. If there was no available worker to
process the table, then a new tablesync worker will be launched,
provided the number of tablesync workers for the subscription does not
exceed max_sync_workers_per_subscription.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/replication/logical/launcher.c  |   1 +
 src/backend/replication/logical/tablesync.c |  50 ++++++--
 src/backend/replication/logical/worker.c    | 121 ++++++++++++++++++--
 src/include/replication/worker_internal.h   |   7 ++
 4 files changed, 158 insertions(+), 21 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e231fa7f95..72e5ef8a78 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -440,6 +440,7 @@ retry:
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
+	worker->is_sync_completed = false;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 8125bbd170..605c5bd4ec 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -129,15 +129,14 @@ static bool FetchTableStates(bool *started_tx);
 static StringInfo copybuf = NULL;
 
 /*
- * Exit routine for synchronization worker.
+ * Prepares the synchronization worker for reuse or exit.
  */
 void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+clean_sync_worker(void)
 {
 	/*
-	 * Commit any outstanding transaction. This is the usual case, unless
-	 * there was nothing to do for the table.
+	 * Commit any outstanding transaction. This is the usual case, unless there
+	 * was nothing to do for the table.
 	 */
 	if (IsTransactionState())
 	{
@@ -145,19 +144,38 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
+	/*
+	 * Disconnect from publisher. Otherwise reused sync workers causes
+	 * exceeding max_wal_senders
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		walrcv_disconnect(LogRepWorkerWalRcvConn);
+		LogRepWorkerWalRcvConn = NULL;
+	}
+
+	/* Find the leader apply worker and signal it. */
+	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
+}
+
+/*
+ * Exit routine for synchronization worker.
+ */
+void
+pg_attribute_noreturn()
+finish_sync_worker(void)
+{
+	clean_sync_worker();
+
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
 	StartTransactionCommand();
 	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
+			(errmsg("logical replication table synchronization worker for subscription \"%s\" has finished",
+					MySubscription->name)));
 	CommitTransactionCommand();
 
-	/* Find the leader apply worker and signal it. */
-	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
-
 	/* Stop gracefully */
 	proc_exit(0);
 }
@@ -379,7 +397,15 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 */
 		replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
+		/* Sync worker has completed synchronization of the current table. */
+		MyLogicalRepWorker->is_sync_completed = true;
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", relation \"%s\" with relid %u has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
+		CommitTransactionCommand();
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 96ddad356b..eae561db05 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3626,6 +3626,20 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can call process_syncing_tables_for_sync.
+				 *
+				 * process_syncing_tables_for_sync decides whether the sync of
+				 * the current table is completed. If it is completed,
+				 * streaming must be already ended. So, we can break the loop.
+				 */
+				if (MyLogicalRepWorker->is_sync_completed)
+				{
+					endofstream = true;
+					break;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -3645,6 +3659,15 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+
+			/*
+			 * If is_sync_completed is true, this means that the tablesync
+			 * worker is done with synchronization. Streaming has already been
+			 * ended by process_syncing_tables_for_sync. We should move to the
+			 * next table if needed, or exit.
+			 */
+			if (MyLogicalRepWorker->is_sync_completed)
+				endofstream = true;
 		}
 
 		/* Cleanup the memory. */
@@ -3747,8 +3770,12 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * Tablesync workers should end streaming before exiting the main loop to
+	 * drop replication slot. Only end streaming here for apply workers.
+	 */
+	if (!am_tablesync_worker())
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
 }
 
 /*
@@ -4523,6 +4550,8 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 					 int originname_size,
 					 XLogRecPtr *origin_startpos)
 {
+	MyLogicalRepWorker->is_sync_completed = false;
+
 	/* Start table synchronization. */
 	start_table_sync(origin_startpos, &slotname);
 
@@ -4702,10 +4731,11 @@ InitializeLogRepWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("%s for subscription \"%s\", table \"%s\" has started",
+				(errmsg("%s for subscription \"%s\", table \"%s\" with relid %u has started",
 						get_worker_name(),
 						MySubscription->name,
-						get_rel_name(MyLogicalRepWorker->relid))));
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
 	else
 		ereport(LOG,
 		/* translator: first %s is the name of logical replication worker */
@@ -4813,11 +4843,84 @@ TablesyncWorkerMain(Datum main_arg)
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	run_tablesync_worker(&options,
-						 myslotname,
-						 originname,
-						 sizeof(originname),
-						 &origin_startpos);
+	/*
+	 * The loop where worker does its job. It loops until there is no relation
+	 * left to sync.
+	 */
+	for (;;)
+	{
+		List	   *rstates;
+		ListCell   *lc;
+		bool 	is_table_found = false;
+
+		run_tablesync_worker(&options,
+							 myslotname,
+							 originname,
+							 sizeof(originname),
+							 &origin_startpos);
+
+		if (IsTransactionState())
+			CommitTransactionCommand();
+
+		if (MyLogicalRepWorker->is_sync_completed)
+		{
+			/* This transaction will be committed by clean_sync_worker. */
+			StartTransactionCommand();
+
+			/*
+			 * Check if there is any table whose relation state is still INIT.
+			 * If a table in INIT state is found, the worker will not be
+			 * finished, it will be reused instead.
+			 */
+			rstates = GetSubscriptionRelations(MySubscription->oid, true);
+
+			foreach(lc, rstates)
+			{
+				SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
+
+				if (rstate->state == SUBREL_STATE_SYNCDONE)
+					continue;
+
+				/*
+				 * Take exclusive lock to prevent any other sync worker from
+				 * picking the same table.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
+
+				/*
+				 * Pick the table for the next run if it is not already picked
+				 * up by another worker.
+				 */
+				if (!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+				{
+					/* Update worker state for the next table */
+					MyLogicalRepWorker->relid = rstate->relid;
+					MyLogicalRepWorker->relstate = rstate->state;
+					MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+					LWLockRelease(LogicalRepWorkerLock);
+
+					/* Found a table for next iteration */
+					is_table_found = true;
+					clean_sync_worker();
+
+					StartTransactionCommand();
+					ereport(LOG,
+							(errmsg("%s for subscription \"%s\" has moved to sync table \"%s\" with relid %u.",
+									get_worker_name(),
+									MySubscription->name,
+									get_rel_name(MyLogicalRepWorker->relid),
+									MyLogicalRepWorker->relid)));
+					CommitTransactionCommand();
+
+					break;
+				}
+				LWLockRelease(LogicalRepWorkerLock);
+			}
+
+			if (!is_table_found)
+				break;
+		}
+	}
 
 	finish_sync_worker();
 }
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 7aba034774..1e9f8e6e72 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -56,6 +56,12 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/*
+	 * Indicates whether tablesync worker has completed sycning its assigned
+	 * table. If true, no need to continue with that table.
+	 */
+	bool		is_sync_completed;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction or
@@ -308,6 +314,7 @@ extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
 #define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)
 
 extern void finish_sync_worker(void);
+extern void clean_sync_worker(void);
 
 static inline bool
 am_tablesync_worker(void)
-- 
2.27.0

v17-0003-Reuse-connection-when-tablesync-workers-change-t.patchapplication/octet-stream; name=v17-0003-Reuse-connection-when-tablesync-workers-change-t.patchDownload
From 2d9d0d564e022670106f8f00abd77c4d86474b20 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:13:52 +0300
Subject: [PATCH v17 3/5] Reuse connection when tablesync workers change the
 target

Previously tablesync workers establish new connections when it changes the syncing
table, but this might have additional overhead. This patch allows to reuse connections
instead.

As for the publisher node, this patch allows to reuse logical walsender processes
after the streaming is done once.
---
 src/backend/replication/logical/launcher.c  |  1 +
 src/backend/replication/logical/tablesync.c | 56 ++++++++++++++-------
 src/backend/replication/logical/worker.c    | 21 ++++----
 src/backend/replication/walsender.c         |  8 +++
 src/include/replication/worker_internal.h   |  3 ++
 5 files changed, 61 insertions(+), 28 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 72e5ef8a78..945619b603 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
 	worker->is_sync_completed = false;
+	worker->worker_slot = slot;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 605c5bd4ec..d455d97f2f 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -144,16 +144,6 @@ clean_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
-	/*
-	 * Disconnect from publisher. Otherwise reused sync workers causes
-	 * exceeding max_wal_senders
-	 */
-	if (LogRepWorkerWalRcvConn != NULL)
-	{
-		walrcv_disconnect(LogRepWorkerWalRcvConn);
-		LogRepWorkerWalRcvConn = NULL;
-	}
-
 	/* Find the leader apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
 }
@@ -1259,6 +1249,24 @@ ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
 			 relid, GetSystemIdentifier());
 }
 
+/*
+ * Determine the application_name for tablesync workers.
+ *
+ * FIXME: set appropriate application_name. Previously, the slot name was used
+ * because the lifetime of the tablesync worker was same as that, but now the
+ * tablesync worker handles many slots during the synchronization so that it is
+ * not suitable. So what should be? Note that if the tablesync worker starts to
+ * reuse the replication slot during synchronization, we should use the slot
+ * name as application_name again.
+ */
+static void
+ApplicationNameForTablesync(Oid suboid, int worker_slot,
+							char *application_name, Size szapp)
+{
+	snprintf(application_name, szapp, "pg_%u_sync_%i_" UINT64_FORMAT, suboid,
+			 worker_slot, GetSystemIdentifier());
+}
+
 /*
  * Start syncing the table in the sync worker.
  *
@@ -1320,15 +1328,25 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 									slotname,
 									NAMEDATALEN);
 
-	/*
-	 * Here we use the slot name instead of the subscription name as the
-	 * application_name, so that it is different from the leader apply worker,
-	 * so that synchronous replication can distinguish them.
-	 */
-	LogRepWorkerWalRcvConn =
-		walrcv_connect(MySubscription->conninfo, true,
-					   must_use_password,
-					   slotname, &err);
+	/* Connect to the publisher if haven't done so already. */
+	if (LogRepWorkerWalRcvConn == NULL)
+	{
+		char application_name[NAMEDATALEN];
+
+		/*
+		 * The application_name must be also different from the leader apply
+		 * worker because synchronous replication must distinguish them.
+		 */
+		ApplicationNameForTablesync(MySubscription->oid,
+									MyLogicalRepWorker->worker_slot,
+									application_name,
+									NAMEDATALEN);
+		LogRepWorkerWalRcvConn =
+			walrcv_connect(MySubscription->conninfo, true,
+						   must_use_password,
+						   application_name, &err);
+	}
+
 	if (LogRepWorkerWalRcvConn == NULL)
 		ereport(ERROR,
 				(errcode(ERRCODE_CONNECTION_FAILURE),
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index eae561db05..4eb67ebd26 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3499,20 +3499,22 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	ErrorContextCallback errcallback;
 
 	/*
-	 * Init the ApplyMessageContext which we clean up after each replication
-	 * protocol message.
+	 * Init the ApplyMessageContext if needed. This context is cleaned up
+	 * after each replication protocol message.
 	 */
-	ApplyMessageContext = AllocSetContextCreate(ApplyContext,
-												"ApplyMessageContext",
-												ALLOCSET_DEFAULT_SIZES);
+	if (!ApplyMessageContext)
+		ApplyMessageContext = AllocSetContextCreate(ApplyContext,
+													"ApplyMessageContext",
+													ALLOCSET_DEFAULT_SIZES);
 
 	/*
 	 * This memory context is used for per-stream data when the streaming mode
 	 * is enabled. This context is reset on each stream stop.
 	 */
-	LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
-													"LogicalStreamingContext",
-													ALLOCSET_DEFAULT_SIZES);
+	if (!LogicalStreamingContext)
+		LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
+														"LogicalStreamingContext",
+														ALLOCSET_DEFAULT_SIZES);
 
 	/* mark as idle, before starting to loop */
 	pgstat_report_activity(STATE_IDLE, NULL);
@@ -4468,7 +4470,8 @@ TwoPhaseTransactionGid(Oid subid, TransactionId xid, char *gid, int szgid)
  * are not repeatable.
  */
 static void
-start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+start_table_sync(XLogRecPtr *origin_startpos,
+				 char **myslotname)
 {
 	char	   *syncslotname = NULL;
 
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index d27ef2985d..a6de2de209 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1827,10 +1827,18 @@ exec_replication_command(const char *cmd_string)
 				set_ps_display(cmdtag);
 				PreventInTransactionBlock(true, cmdtag);
 
+
 				if (cmd->kind == REPLICATION_KIND_PHYSICAL)
 					StartReplication(cmd);
 				else
+				{
+					/*
+					 * Reset flags because reusing tablesync workers can mean
+					 * this is the second time here.
+					 */
+					streamingDoneSending = streamingDoneReceiving = false;
 					StartLogicalReplication(cmd);
+				}
 
 				/* dupe, but necessary per libpqrcv_endstreaming */
 				EndReplicationCommand(cmdtag);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 1e9f8e6e72..32783a8cdd 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -62,6 +62,9 @@ typedef struct LogicalRepWorker
 	 */
 	bool		is_sync_completed;
 
+	/* Indicates the slot number which corresponds to this LogicalRepWorker. */
+	int			worker_slot;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction or
-- 
2.27.0

v17-0004-Add-replication-protocol-cmd-to-create-a-snapsho.patchapplication/octet-stream; name=v17-0004-Add-replication-protocol-cmd-to-create-a-snapsho.patchDownload
From 67d4a99dc79848858309cd7ded89206e693d6765 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 13 Oct 2022 17:05:45 +0300
Subject: [PATCH v17 4/5] Add replication protocol cmd to create a snapshot

Introduced CREATE_REPLICATION_SNAPSHOT to be able to create and use a
snapshot without creating a new replication slot, but by using an
existing slot.

CREATE_REPLICATION_SNAPSHOT simply does what CREATE_REPLICATION_SLOT does
without creating a new replication slot.

CREATE_REPLICATION_SNAPSHOT command imports the snapshot into the current
transaction and returns consistent_point. The changes earlier than the
consistent_point will be applied by importing the snapshot. All changes
later than the consistent_point will be available to be consumed from
the replication slot.

This is useful for reusing replication slots in logical replication.
Otherwise, tablesync workers cannot start from a consistent point to copy
a relation and then apply changes by consuming from replication slot.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/protocol.sgml                    |  31 ++++++
 .../libpqwalreceiver/libpqwalreceiver.c       |  69 +++++++++++-
 src/backend/replication/logical/logical.c     |  40 ++++++-
 .../replication/logical/logicalfuncs.c        |   1 +
 src/backend/replication/repl_gram.y           |  18 ++-
 src/backend/replication/repl_scanner.l        |   2 +
 src/backend/replication/slotfuncs.c           |   1 +
 src/backend/replication/walsender.c           | 104 +++++++++++++++++-
 src/include/nodes/replnodes.h                 |  11 ++
 src/include/replication/logical.h             |   1 +
 src/include/replication/walreceiver.h         |  13 +++
 src/tools/pgindent/typedefs.list              |   2 +
 12 files changed, 289 insertions(+), 4 deletions(-)

diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index b11d9a6ba3..f7d5acee93 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2595,6 +2595,37 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
      </listitem>
     </varlistentry>
 
+    <varlistentry id="protocol-replication-replication-slot-snapshot">
+     <term><literal>CREATE_REPLICATION_SNAPSHOT</literal> <replaceable class="parameter">slot_name</replaceable> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
+      <indexterm><primary>CREATE_REPLICATION_SNAPSHOT</primary></indexterm>
+     </term>
+     <listitem>
+      <para>
+       Creates a snapshot including all the changes from the replication slot until
+       the point at which the replication slot becomes consistent. Then the snapshot
+       is used in the current transaction. This command is currently only supported
+       for logical replication slots.
+      </para>
+
+      <para>
+       In response to this command, the server will return a one-row result set,
+       containing the following field:
+       <variablelist>
+        <varlistentry>
+         <term><literal>consistent_point</literal> (<type>text</type>)</term>
+         <listitem>
+          <para>
+           The WAL location at which the slot became consistent.  This is the
+           earliest location from which streaming can start on this replication
+           slot.
+          </para>
+         </listitem>
+        </varlistentry>
+       </variablelist>
+      </para>
+     </listitem>
+    </varlistentry>
+
     <varlistentry id="protocol-replication-base-backup" xreflabel="BASE_BACKUP">
      <term><literal>BASE_BACKUP</literal> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
       <indexterm><primary>BASE_BACKUP</primary></indexterm>
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index 60d5c1fc40..ac929fe41e 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -82,6 +82,8 @@ static WalRcvExecResult *libpqrcv_exec(WalReceiverConn *conn,
 									   const int nRetTypes,
 									   const Oid *retTypes);
 static void libpqrcv_disconnect(WalReceiverConn *conn);
+static void libpqrcv_slot_snapshot(WalReceiverConn *conn, char *slotname,
+								   const WalRcvStreamOptions *options, XLogRecPtr *lsn);
 
 static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_connect = libpqrcv_connect,
@@ -98,7 +100,8 @@ static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_create_slot = libpqrcv_create_slot,
 	.walrcv_get_backend_pid = libpqrcv_get_backend_pid,
 	.walrcv_exec = libpqrcv_exec,
-	.walrcv_disconnect = libpqrcv_disconnect
+	.walrcv_disconnect = libpqrcv_disconnect,
+	.walrcv_slot_snapshot = libpqrcv_slot_snapshot
 };
 
 /* Prototypes for private functions */
@@ -982,6 +985,70 @@ libpqrcv_create_slot(WalReceiverConn *conn, const char *slotname,
 	return snapshot;
 }
 
+/*
+ * Create a new snapshot by using an existing replication slot.
+ */
+static void
+libpqrcv_slot_snapshot(WalReceiverConn *conn,
+					   char *slotname,
+					   const WalRcvStreamOptions *options,
+					   XLogRecPtr *lsn)
+{
+	StringInfoData cmd;
+	PGresult   *res;
+	char	   *pubnames_str;
+	List	   *pubnames;
+	char	   *pubnames_literal;
+
+	initStringInfo(&cmd);
+
+	/* Build the command. */
+	appendStringInfo(&cmd, "CREATE_REPLICATION_SNAPSHOT \"%s\"", slotname);
+	appendStringInfoString(&cmd, " (");
+	appendStringInfo(&cmd, " proto_version '%u'",
+					 options->proto.logical.proto_version);
+
+	/* Add publication names. */
+	pubnames = options->proto.logical.publication_names;
+	pubnames_str = stringlist_to_identifierstr(conn->streamConn, pubnames);
+	if (!pubnames_str)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("Could not create a snapshot by replication slot %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	pubnames_literal = PQescapeLiteral(conn->streamConn, pubnames_str,
+									   strlen(pubnames_str));
+	if (!pubnames_literal)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("Could not create a snapshot by replication slot %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	appendStringInfo(&cmd, ", publication_names %s", pubnames_literal);
+	PQfreemem(pubnames_literal);
+	pfree(pubnames_str);
+
+	appendStringInfoString(&cmd, " )");
+
+	/* Execute the command. */
+	res = libpqrcv_PQexec(conn->streamConn, cmd.data);
+	pfree(cmd.data);
+
+	if (PQresultStatus(res) != PGRES_TUPLES_OK)
+	{
+		PQclear(res);
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg("Could not create a snapshot by replication slot \"%s\": %s",
+						slotname, pchomp(PQerrorMessage(conn->streamConn)))));
+	}
+
+	if (lsn)
+		*lsn = DatumGetLSN(DirectFunctionCall1Coll(pg_lsn_in, InvalidOid,
+												   CStringGetDatum(PQgetvalue(res, 0, 0))));
+
+	PQclear(res);
+}
+
 /*
  * Return PID of remote backend process.
  */
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 41243d0187..3c11f8e1dd 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -476,6 +476,10 @@ CreateInitDecodingContext(const char *plugin,
  * fast_forward
  *		bypass the generation of logical changes.
  *
+ * need_full_snapshot
+ * 		if true, must obtain a snapshot able to read all tables;
+ *  	if false, one that can read only catalogs is acceptable.
+ *
  * xl_routine
  *		XLogReaderRoutine used by underlying xlogreader
  *
@@ -494,6 +498,7 @@ LogicalDecodingContext *
 CreateDecodingContext(XLogRecPtr start_lsn,
 					  List *output_plugin_options,
 					  bool fast_forward,
+					  bool need_full_snapshot,
 					  XLogReaderRoutine *xl_routine,
 					  LogicalOutputPluginWriterPrepareWrite prepare_write,
 					  LogicalOutputPluginWriterWrite do_write,
@@ -502,6 +507,7 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 	LogicalDecodingContext *ctx;
 	ReplicationSlot *slot;
 	MemoryContext old_context;
+	TransactionId xmin_horizon = InvalidTransactionId;
 
 	/* shorter lines... */
 	slot = MyReplicationSlot;
@@ -571,8 +577,40 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 		start_lsn = slot->data.confirmed_flush;
 	}
 
+
+	/*
+	 * We need to determine a safe xmin horizon to start decoding from if we
+	 * want to create a snapshot too. Otherwise we would end up with a
+	 * snapshot that cannot be imported since xmin value from the snapshot may
+	 * be less than the oldest safe xmin. To avoid this call
+	 * GetOldestSafeDecodingTransactionId() to return a safe xmin value, which
+	 * can be used while exporting/importing the snapshot.
+	 *
+	 * So we have to acquire the ProcArrayLock to prevent computation of new
+	 * xmin horizons by other backends, get the safe decoding xid, and inform
+	 * the slot machinery about the new limit. Once that's done the
+	 * ProcArrayLock can be released as the slot machinery now is protecting
+	 * against vacuum.
+	 */
+	if (need_full_snapshot)
+	{
+		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+		xmin_horizon = GetOldestSafeDecodingTransactionId(!need_full_snapshot);
+
+		SpinLockAcquire(&slot->mutex);
+		slot->effective_catalog_xmin = xmin_horizon;
+		slot->data.catalog_xmin = xmin_horizon;
+		slot->effective_xmin = xmin_horizon;
+		SpinLockRelease(&slot->mutex);
+
+		ReplicationSlotsComputeRequiredXmin(true);
+
+		LWLockRelease(ProcArrayLock);
+	}
+
 	ctx = StartupDecodingContext(output_plugin_options,
-								 start_lsn, InvalidTransactionId, false,
+								 start_lsn, xmin_horizon, need_full_snapshot,
 								 fast_forward, xl_routine, prepare_write,
 								 do_write, update_progress);
 
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index 55a24c02c9..85c5cdb633 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -208,6 +208,7 @@ pg_logical_slot_get_changes_guts(FunctionCallInfo fcinfo, bool confirm, bool bin
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									options,
 									false,
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/repl_gram.y b/src/backend/replication/repl_gram.y
index 0c874e33cf..957a5cc653 100644
--- a/src/backend/replication/repl_gram.y
+++ b/src/backend/replication/repl_gram.y
@@ -65,6 +65,7 @@ Node *replication_parse_result;
 %token K_CREATE_REPLICATION_SLOT
 %token K_DROP_REPLICATION_SLOT
 %token K_TIMELINE_HISTORY
+%token K_CREATE_REPLICATION_SNAPSHOT
 %token K_WAIT
 %token K_TIMELINE
 %token K_PHYSICAL
@@ -80,7 +81,7 @@ Node *replication_parse_result;
 %type <node>	command
 %type <node>	base_backup start_replication start_logical_replication
 				create_replication_slot drop_replication_slot identify_system
-				read_replication_slot timeline_history show
+				read_replication_slot timeline_history show create_replication_snapshot
 %type <list>	generic_option_list
 %type <defelt>	generic_option
 %type <uintval>	opt_timeline
@@ -114,6 +115,7 @@ command:
 			| read_replication_slot
 			| timeline_history
 			| show
+			| create_replication_snapshot
 			;
 
 /*
@@ -307,6 +309,19 @@ timeline_history:
 				}
 			;
 
+/*
+ * CREATE_REPLICATION_SNAPSHOT %s options
+ */
+create_replication_snapshot:
+			K_CREATE_REPLICATION_SNAPSHOT var_name plugin_options
+				{
+					CreateReplicationSnapshotCmd *n = makeNode(CreateReplicationSnapshotCmd);
+					n->slotname = $2;
+					n->options = $3;
+					$$ = (Node *) n;
+				}
+			;
+
 opt_physical:
 			K_PHYSICAL
 			| /* EMPTY */
@@ -400,6 +415,7 @@ ident_or_keyword:
 			| K_CREATE_REPLICATION_SLOT	{ $$ = "create_replication_slot"; }
 			| K_DROP_REPLICATION_SLOT		{ $$ = "drop_replication_slot"; }
 			| K_TIMELINE_HISTORY			{ $$ = "timeline_history"; }
+			| K_CREATE_REPLICATION_SNAPSHOT	{ $$ = "create_replication_snapshot"; }
 			| K_WAIT						{ $$ = "wait"; }
 			| K_TIMELINE					{ $$ = "timeline"; }
 			| K_PHYSICAL					{ $$ = "physical"; }
diff --git a/src/backend/replication/repl_scanner.l b/src/backend/replication/repl_scanner.l
index 1cc7fb858c..0cc2cdd0be 100644
--- a/src/backend/replication/repl_scanner.l
+++ b/src/backend/replication/repl_scanner.l
@@ -126,6 +126,7 @@ START_REPLICATION	{ return K_START_REPLICATION; }
 CREATE_REPLICATION_SLOT		{ return K_CREATE_REPLICATION_SLOT; }
 DROP_REPLICATION_SLOT		{ return K_DROP_REPLICATION_SLOT; }
 TIMELINE_HISTORY	{ return K_TIMELINE_HISTORY; }
+CREATE_REPLICATION_SNAPSHOT	{ return K_CREATE_REPLICATION_SNAPSHOT; }
 PHYSICAL			{ return K_PHYSICAL; }
 RESERVE_WAL			{ return K_RESERVE_WAL; }
 LOGICAL				{ return K_LOGICAL; }
@@ -303,6 +304,7 @@ replication_scanner_is_replication_command(void)
 		case K_DROP_REPLICATION_SLOT:
 		case K_READ_REPLICATION_SLOT:
 		case K_TIMELINE_HISTORY:
+		case K_CREATE_REPLICATION_SNAPSHOT:
 		case K_SHOW:
 			/* Yes; push back the first token so we can parse later. */
 			repl_pushed_back_token = first_token;
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 6035cf4816..c5b2d5b61f 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -486,6 +486,7 @@ pg_logical_replication_slot_advance(XLogRecPtr moveto)
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									NIL,
 									true,	/* fast_forward */
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index a6de2de209..7cd164b975 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -238,6 +238,7 @@ static void CreateReplicationSlot(CreateReplicationSlotCmd *cmd);
 static void DropReplicationSlot(DropReplicationSlotCmd *cmd);
 static void StartReplication(StartReplicationCmd *cmd);
 static void StartLogicalReplication(StartReplicationCmd *cmd);
+static void CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd);
 static void ProcessStandbyMessage(void);
 static void ProcessStandbyReplyMessage(void);
 static void ProcessStandbyHSFeedbackMessage(void);
@@ -1283,7 +1284,7 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	 * are reported early.
 	 */
 	logical_decoding_ctx =
-		CreateDecodingContext(cmd->startpoint, cmd->options, false,
+		CreateDecodingContext(cmd->startpoint, cmd->options, false, false,
 							  XL_ROUTINE(.page_read = logical_read_xlog_page,
 										 .segment_open = WalSndSegmentOpen,
 										 .segment_close = wal_segment_close),
@@ -1335,6 +1336,98 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	EndCommand(&qc, DestRemote, false);
 }
 
+/*
+ * Create a snapshot from an existing replication slot.
+ */
+static void
+CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd)
+{
+	Snapshot	snap;
+	LogicalDecodingContext *ctx;
+	char		xloc[MAXFNAMELEN];
+	DestReceiver *dest;
+	TupOutputState *tstate;
+	TupleDesc	tupdesc;
+	Datum		values[1];
+	bool		nulls[1] = {0};
+
+	Assert(!MyReplicationSlot);
+
+	CheckLogicalDecodingRequirements();
+
+	if (!IsTransactionBlock())
+		ereport(ERROR,
+				(errmsg("%s must be called inside a transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (XactIsoLevel != XACT_REPEATABLE_READ)
+		ereport(ERROR,
+				(errmsg("%s must be called in REPEATABLE READ isolation mode transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (!XactReadOnly)
+		ereport(ERROR,
+				(errmsg("%s must be called in a read only transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (FirstSnapshotSet)
+		ereport(ERROR,
+				(errmsg("%s must be called before any query",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (IsSubTransaction())
+		ereport(ERROR,
+				(errmsg("%s must not be called in a subtransaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	ReplicationSlotAcquire(cmd->slotname, false);
+
+	ctx = CreateDecodingContext(MyReplicationSlot->data.restart_lsn,
+								cmd->options,
+								false,
+								true,
+								XL_ROUTINE(.page_read = logical_read_xlog_page,
+										   .segment_open = WalSndSegmentOpen,
+										   .segment_close = wal_segment_close),
+								WalSndPrepareWrite, WalSndWriteData,
+								WalSndUpdateProgress);
+
+	/*
+	 * Signal that we don't need the timeout mechanism. We're just creating
+	 * the snapshot with the replication slot and don't yet accept feedback
+	 * messages or send keepalives. As we possibly need to wait for further
+	 * WAL the walsender would otherwise possibly be killed too soon.
+	 */
+	last_reply_timestamp = 0;
+
+	/* build initial snapshot, might take a while */
+	DecodingContextFindStartpoint(ctx);
+
+	snap = SnapBuildInitialSnapshot(ctx->snapshot_builder);
+	RestoreTransactionSnapshot(snap, MyProc);
+
+	/* Don't need the decoding context anymore */
+	FreeDecodingContext(ctx);
+
+	/* Create a tuple to send consistent WAL location */
+	snprintf(xloc, sizeof(xloc), "%X/%X",
+			 LSN_FORMAT_ARGS(MyReplicationSlot->data.confirmed_flush));
+
+	dest = CreateDestReceiver(DestRemoteSimple);
+	tupdesc = CreateTemplateTupleDesc(1);
+	TupleDescInitBuiltinEntry(tupdesc, (AttrNumber) 1, "consistent_point",
+							  TEXTOID, -1, 0);
+	tstate = begin_tup_output_tupdesc(dest, tupdesc, &TTSOpsVirtual);
+
+	/* consistent wal location */
+	values[0] = CStringGetTextDatum(xloc);
+
+	do_tup_output(tstate, values, nulls);
+	end_tup_output(tstate);
+
+	ReplicationSlotRelease();
+}
+
 /*
  * LogicalDecodingContext 'prepare_write' callback.
  *
@@ -1871,6 +1964,15 @@ exec_replication_command(const char *cmd_string)
 			}
 			break;
 
+		case T_CreateReplicationSnapshotCmd:
+			{
+				cmdtag = "CREATE_REPLICATION_SNAPSHOT";
+				set_ps_display(cmdtag);
+				CreateReplicationSnapshot((CreateReplicationSnapshotCmd *) cmd_node);
+				EndReplicationCommand(cmdtag);
+				break;
+			}
+
 		default:
 			elog(ERROR, "unrecognized replication command node tag: %u",
 				 cmd_node->type);
diff --git a/src/include/nodes/replnodes.h b/src/include/nodes/replnodes.h
index 4321ba8f86..154ab74f33 100644
--- a/src/include/nodes/replnodes.h
+++ b/src/include/nodes/replnodes.h
@@ -108,4 +108,15 @@ typedef struct TimeLineHistoryCmd
 	TimeLineID	timeline;
 } TimeLineHistoryCmd;
 
+/* ----------------------
+ *		CREATE_REPLICATION_SNAPSHOT command
+ * ----------------------
+ */
+typedef struct CreateReplicationSnapshotCmd
+{
+	NodeTag		type;
+	char	   *slotname;
+	List	   *options;
+} CreateReplicationSnapshotCmd;
+
 #endif							/* REPLNODES_H */
diff --git a/src/include/replication/logical.h b/src/include/replication/logical.h
index 5f49554ea0..6535786a0e 100644
--- a/src/include/replication/logical.h
+++ b/src/include/replication/logical.h
@@ -125,6 +125,7 @@ extern LogicalDecodingContext *CreateInitDecodingContext(const char *plugin,
 extern LogicalDecodingContext *CreateDecodingContext(XLogRecPtr start_lsn,
 													 List *output_plugin_options,
 													 bool fast_forward,
+													 bool need_full_snapshot,
 													 XLogReaderRoutine *xl_routine,
 													 LogicalOutputPluginWriterPrepareWrite prepare_write,
 													 LogicalOutputPluginWriterWrite do_write,
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 281626fa6f..f01e5f23a7 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -386,6 +386,16 @@ typedef WalRcvExecResult *(*walrcv_exec_fn) (WalReceiverConn *conn,
  */
 typedef void (*walrcv_disconnect_fn) (WalReceiverConn *conn);
 
+/*
+ * walrcv_slot_snapshot_fn
+ *
+ * Create a snapshot by an existing replication slot
+ */
+typedef void (*walrcv_slot_snapshot_fn) (WalReceiverConn *conn,
+										 char *slotname,
+										 const WalRcvStreamOptions *options,
+										 XLogRecPtr *lsn);
+
 typedef struct WalReceiverFunctionsType
 {
 	walrcv_connect_fn walrcv_connect;
@@ -403,6 +413,7 @@ typedef struct WalReceiverFunctionsType
 	walrcv_get_backend_pid_fn walrcv_get_backend_pid;
 	walrcv_exec_fn walrcv_exec;
 	walrcv_disconnect_fn walrcv_disconnect;
+	walrcv_slot_snapshot_fn walrcv_slot_snapshot;
 } WalReceiverFunctionsType;
 
 extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
@@ -437,6 +448,8 @@ extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
 	WalReceiverFunctions->walrcv_exec(conn, exec, nRetTypes, retTypes)
 #define walrcv_disconnect(conn) \
 	WalReceiverFunctions->walrcv_disconnect(conn)
+#define walrcv_slot_snapshot(conn, slotname, options, lsn) \
+	WalReceiverFunctions->walrcv_slot_snapshot(conn, slotname, options, lsn)
 
 static inline void
 walrcv_clear_result(WalRcvExecResult *walres)
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index e941fb6c82..cc023b10fa 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2348,6 +2348,7 @@ ReplicationSlotInvalidationCause
 ReplicationSlotOnDisk
 ReplicationSlotPersistency
 ReplicationSlotPersistentData
+CreateReplicationSnapshotCmd
 ReplicationState
 ReplicationStateCtl
 ReplicationStateOnDisk
@@ -3855,6 +3856,7 @@ walrcv_receive_fn
 walrcv_send_fn
 walrcv_server_version_fn
 walrcv_startstreaming_fn
+walrcv_slot_snapshot_fn
 wchar2mb_with_len_converter
 wchar_t
 win32_deadchild_waitinfo
-- 
2.27.0

v17-0005-Reuse-Replication-Slot-and-Origin-in-Tablesync.patchapplication/octet-stream; name=v17-0005-Reuse-Replication-Slot-and-Origin-in-Tablesync.patchDownload
From 56f75382e01bbf8cb6572b7302c4395f761b12d7 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH v17 5/5] Reuse Replication Slot and Origin in Tablesync

This commit allows reusing replication slots and origins during tablesync.

Earlier, a tablesync worker was creating a new replication slot and origin
each time it syncs a new table. With this patch, replication
slots/origins can be reusable for tablesync.

This reduces the overhead of creating/dropping replication slots and origins
and improves tablesync speed significantly especially for empty or small tables.

If the state of the current table is INIT or DATASYNC, tablesync worker needs a
repliation slot/origin. If the worker has not created slot and origin in
its previous runs, it will create those first. Otherwise the worker reuses
slot and origin created by the same worker in previous iterations earlier.
Tables in FINISHEDCOPY are expected to have a replication slot and origin.
Slot and origin names for such tables are persisted in
pg_subscription_rel catalog. Tablesync worker can fetch them and proceed
with existing slot and origin of FINISHEDCOPY tables and does not need to
create new ones.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/catalogs.sgml                    |  31 ++
 src/backend/catalog/pg_subscription.c         | 247 +++++++++++-
 src/backend/commands/subscriptioncmds.c       | 229 +++++++----
 .../replication/logical/applyparallelworker.c |   3 +-
 src/backend/replication/logical/launcher.c    |   8 +-
 src/backend/replication/logical/tablesync.c   | 373 +++++++++++++-----
 src/backend/replication/logical/worker.c      |  57 ++-
 src/include/catalog/pg_subscription.h         |   6 +
 src/include/catalog/pg_subscription_rel.h     |  14 +-
 src/include/replication/slot.h                |   3 +-
 src/include/replication/worker_internal.h     |  23 +-
 11 files changed, 803 insertions(+), 191 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 852cb30ae1..60718ab587 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8030,6 +8030,19 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        origin.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>sublastusedid</structfield> <type>int8</type>
+      </para>
+      <para>
+      The last used ID for tablesync workers. It acts as an unique identifier
+      for replication slots which are created by tablesync workers.
+      The last used ID needs to be persisted to make logical replication safely
+      proceed after any interruption. If sublastusedid is 0, then no table has
+      been synced yet.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
@@ -8114,6 +8127,24 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        otherwise null
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srrelslotname</structfield> <type>name</type>
+      </para>
+      <para>
+       Replication slot name that is used for synchronization of relation
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srreloriginname</structfield> <type>name</type>
+      </para>
+      <para>
+       Origin name that is used for tracking synchronization of relation
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index d07f88ce28..152fdaa310 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -108,6 +108,14 @@ GetSubscription(Oid subid, bool missing_ok)
 								   Anum_pg_subscription_suborigin);
 	sub->origin = TextDatumGetCString(datum);
 
+	/* Get last used id */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
+							tup,
+							Anum_pg_subscription_sublastusedid,
+							&isnull);
+	Assert(!isnull);
+	sub->lastusedid = DatumGetInt64(datum);
+
 	ReleaseSysCache(tup);
 
 	return sub;
@@ -199,6 +207,44 @@ DisableSubscription(Oid subid)
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update the last used replication slot ID for the given subscription.
+ */
+void
+UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid)
+{
+	Relation	rel;
+	bool		nulls[Natts_pg_subscription];
+	bool		replaces[Natts_pg_subscription];
+	Datum		values[Natts_pg_subscription];
+	HeapTuple	tup;
+
+	/* Look up the subscription in the catalog */
+	rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+	tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	/* Form a new tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	replaces[Anum_pg_subscription_sublastusedid - 1] = true;
+	values[Anum_pg_subscription_sublastusedid- 1] = Int64GetDatum(lastusedid);
+
+	/* Update the catalog */
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+	heap_freetuple(tup);
+
+	table_close(rel, NoLock);
+}
+
 /*
  * Convert text array to list of strings.
  *
@@ -228,7 +274,7 @@ textarray_to_stringlist(ArrayType *textarray)
  */
 void
 AddSubscriptionRelState(Oid subid, Oid relid, char state,
-						XLogRecPtr sublsn)
+						XLogRecPtr sublsn, char *relslotname, char *reloriginname)
 {
 	Relation	rel;
 	HeapTuple	tup;
@@ -257,6 +303,16 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
 	else
 		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -269,6 +325,60 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Internal function to modify columns for relation state update
+ */
+static void
+UpdateSubscriptionRelState_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char state,
+									XLogRecPtr sublsn)
+{
+	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
+	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+
+	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (sublsn != InvalidXLogRecPtr)
+		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
+	else
+		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication slot update
+ */
+static void
+UpdateSubscriptionRelReplicationSlot_internal(Datum *values,
+											bool *nulls,
+											bool *replaces,
+											char *relslotname)
+{
+	replaces[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication origin update
+ */
+static void
+UpdateSubscriptionRelOrigin_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char *reloriginname)
+{
+	replaces[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+}
+
 /*
  * Update the state of a subscription table.
  */
@@ -299,14 +409,56 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	memset(nulls, false, sizeof(nulls));
 	memset(replaces, false, sizeof(replaces));
 
-	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
-	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
 
-	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
-	if (sublsn != InvalidXLogRecPtr)
-		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
-	else
-		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update replication slot name, origin name and state of
+ * a subscription table in one transaction.
+ */
+void
+UpdateSubscriptionRel(Oid subid,
+					  Oid relid,
+					  char state,
+					  XLogRecPtr sublsn,
+					  char *relslotname,
+					  char *reloriginname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
+	UpdateSubscriptionRelOrigin_internal(values, nulls, replaces, reloriginname);
 
 	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
 							replaces);
@@ -318,6 +470,85 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Get origin name of subscription table.
+ *
+ * reloriginname's value has the replication origin name if the origin exists.
+ */
+void
+GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*originname;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srreloriginname, isnull);
+	if (!*isnull)
+	{
+		originname = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(reloriginname, originname, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
+/*
+ * Get replication slot name of subscription table.
+ *
+ * slotname's value has the replication slot name if the subscription has any.
+ */
+void
+GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*relrepslot;
+	bool		isnull;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srrelslotname, &isnull);
+	if (!isnull)
+	{
+		relrepslot = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(slotname, relrepslot, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
 /*
  * Get state of subscription table.
  *
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index d4e798baeb..4f5b3c572c 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -710,6 +710,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 		publicationListToArray(publications);
 	values[Anum_pg_subscription_suborigin - 1] =
 		CStringGetTextDatum(opts.origin);
+	values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(0);
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -773,7 +774,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 										 rv->schemaname, rv->relname);
 
 				AddSubscriptionRelState(subid, relid, table_state,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 			}
 
 			/*
@@ -864,6 +865,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
 	bool		must_use_password;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -943,7 +946,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				AddSubscriptionRelState(sub->oid, relid,
 										copy_data ? SUBREL_STATE_INIT : SUBREL_STATE_READY,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 				ereport(DEBUG1,
 						(errmsg_internal("table \"%s.%s\" added to subscription \"%s\"",
 										 rv->schemaname, rv->relname, sub->name)));
@@ -967,6 +970,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				char		state;
 				XLogRecPtr	statelsn;
+				char		slotname[NAMEDATALEN] = {0};
 
 				/*
 				 * Lock pg_subscription_rel with AccessExclusiveLock to
@@ -993,13 +997,36 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/*
+				 * Find the logical replication sync worker. If exists, store
+				 * the slot number for dropping associated replication slots
+				 * later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot_name);
+				}
+				else
+				{
+					/*
+					 * Sync of this relation might be failed in an earlier
+					 * attempt, but the replication slot might still exist.
+					 */
+					GetSubscriptionRelReplicationSlot(sub->oid, relid, slotname);
+					if (strlen(slotname) > 0)
+						sub_remove_slots = lappend(sub_remove_slots, slotname);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
 				 * tablesync origin.
 				 */
-				if (state != SUBREL_STATE_READY)
+				if (state != SUBREL_STATE_READY &&
+					state != SUBREL_STATE_SYNCDONE)
 				{
 					char		originname[NAMEDATALEN];
 
@@ -1027,31 +1054,24 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
-		 * the database operations we won't be able to rollback dropped slots.
+		 * Drop the replication slots associated with tablesync workers for
+		 * removed tables. This has to be at the end because otherwise if
+		 * there is an error while doing the database operations we won't be
+		 * able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			memcpy(syncslotname, lfirst(lc), sizeof(NAMEDATALEN));
+
+			/*
+			 * There is no certainty, maybe the slot does not exist yet. Also,
+			 * if we fail after removing some of the slots, next time, it will
+			 * again try to drop already dropped slots and fail. For these
+			 * reasons, we allow missing_ok = true for the drop.
+			 */
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1474,6 +1494,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	char	   *subname;
 	char	   *conninfo;
 	char	   *slotname;
+	int64		lastusedid;
 	List	   *subworkers;
 	ListCell   *lc;
 	char		originname[NAMEDATALEN];
@@ -1546,6 +1567,14 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	else
 		slotname = NULL;
 
+	/* Get the last used identifier by the subscription */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID, tup,
+							Anum_pg_subscription_sublastusedid, &isnull);
+	if (!isnull)
+		lastusedid = DatumGetInt64(datum);
+	else
+		lastusedid = 0;
+
 	/*
 	 * Since dropping a replication slot is not transactional, the replication
 	 * slot stays dropped even if the transaction rolls back.  So we cannot
@@ -1595,6 +1624,8 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	}
 	list_free(subworkers);
 
+	rstates = GetSubscriptionRelations(subid, true);
+
 	/*
 	 * Remove the no-longer-useful entry in the launcher's table of apply
 	 * worker start times.
@@ -1606,36 +1637,26 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	ApplyLauncherForgetWorkerStartTime(subid);
 
 	/*
-	 * Cleanup of tablesync replication origins.
-	 *
-	 * Any READY-state relations would already have dealt with clean-ups.
+	 * Cleanup of tablesync replication origins associated with the
+	 * subscription, if exists. Try to drop origins by creating all origin
+	 * names created for this subscription.
 	 *
 	 * Note that the state can't change because we have already stopped both
 	 * the apply and tablesync workers and they can't restart because of
 	 * exclusive lock on the subscription.
+	 *
+	 * XXX: This can be handled better instead of looping through all possible
 	 */
-	rstates = GetSubscriptionRelations(subid, true);
-	foreach(lc, rstates)
+	for (int64 i = 1; i <= lastusedid; i++)
 	{
-		SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-		Oid			relid = rstate->relid;
-
-		/* Only cleanup resources of tablesync workers */
-		if (!OidIsValid(relid))
-			continue;
+		char		originname_to_drop[NAMEDATALEN] = {0};
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * It is possible that the origin is not yet created for tablesync
-		 * worker so passing missing_ok = true. This can happen for the states
-		 * before SUBREL_STATE_FINISHEDCOPY.
-		 */
-		ReplicationOriginNameForLogicalRep(subid, relid, originname,
-										   sizeof(originname));
-		replorigin_drop_by_name(originname, true, false);
+		snprintf(originname_to_drop, sizeof(originname_to_drop), "pg_%u_%lld", subid, (long long) i);
+		/* missing_ok = true, since the origin might be already dropped. */
+		replorigin_drop_by_name(originname_to_drop, true, false);
 	}
 
+
 	/* Clean up dependencies */
 	deleteSharedDependencyRecordsFor(SubscriptionRelationId, subid, 0);
 
@@ -1694,39 +1715,17 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
+		List	   *slots = NULL;
 
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
 
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char	   *syncslotname = (char *) lfirst(lc);
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1743,6 +1742,71 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * Get the replication slot names associated with the subscription.
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok)
+{
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+					 " FROM pg_replication_slots"
+					 " WHERE slot_name LIKE 'pg_%i_sync_%%';",
+					 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+					errmsg("could not receive list of slots associated with the subscription %u, error: %s",
+					subid, res->err));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();
+		return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -2155,6 +2219,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -2165,18 +2230,20 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
-			char		syncslotname[NAMEDATALEN] = {0};
-
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
-				 syncslotname);
+				 worker->slot_name);
 		}
 	}
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 1d4e83c4c1..d2c70dffbc 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -440,7 +440,8 @@ pa_launch_parallel_worker(void)
 										MySubscription->name,
 										MyLogicalRepWorker->userid,
 										InvalidOid,
-										dsm_segment_handle(winfo->dsm_seg));
+										dsm_segment_handle(winfo->dsm_seg),
+										InvalidRepSlotId);
 
 	if (launched)
 	{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 945619b603..9f378c311e 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -304,7 +304,7 @@ logicalrep_workers_find(Oid subid, bool only_running)
  */
 bool
 logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
-						 Oid relid, dsm_handle subworker_dsm)
+						 Oid relid, dsm_handle subworker_dsm, int64 slotid)
 {
 	BackgroundWorker bgw;
 	BackgroundWorkerHandle *bgw_handle;
@@ -430,6 +430,9 @@ retry:
 	worker->launch_time = now;
 	worker->in_use = true;
 	worker->generation++;
+	worker->created_slot = false;
+	worker->rep_slot_id = slotid;
+	worker->slot_name = (char *) palloc(NAMEDATALEN);
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -1184,7 +1187,8 @@ ApplyLauncherMain(Datum main_arg)
 				ApplyLauncherSetWorkerStartTime(sub->oid, now);
 				logicalrep_worker_launch(sub->dbid, sub->oid, sub->name,
 										 sub->owner, InvalidOid,
-										 DSM_HANDLE_INVALID);
+										 DSM_HANDLE_INVALID,
+										 InvalidRepSlotId);
 			}
 			else
 			{
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index d455d97f2f..f6fd3b0a30 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -318,40 +318,29 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 								   MyLogicalRepWorker->relid,
 								   MyLogicalRepWorker->relstate,
 								   MyLogicalRepWorker->relstate_lsn);
+		CommitTransactionCommand();
 
 		/*
-		 * End streaming so that LogRepWorkerWalRcvConn can be used to drop
-		 * the slot.
-		 */
-		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
-
-		/*
-		 * Cleanup the tablesync slot.
+		 * Cleanup the tablesync slot. If the slot name used by this worker is
+		 * different from the default slot name for the worker, this means the
+		 * current table had started to being synchronized by another worker
+		 * and replication slot. And this worker is reusing a replication slot
+		 * from a previous attempt. We do not need that replication slot
+		 * anymore.
 		 *
 		 * This has to be done after updating the state because otherwise if
 		 * there is an error while doing the database operations we won't be
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
+										MyLogicalRepWorker->rep_slot_id,
 										syncslotname,
 										sizeof(syncslotname));
 
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
-		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
-
-		CommitTransactionCommand();
-		pgstat_report_stat(false);
-
-		/*
-		 * Start a new transaction to clean up the tablesync origin tracking.
-		 * This transaction will be ended within the finish_sync_worker().
-		 * Now, even, if we fail to remove this here, the apply worker will
-		 * ensure to clean it up afterward.
+		 * We are safe to drop the replication tracking origin after this
+		 * point. Now, even, if we fail to remove this here, the apply worker
+		 * will ensure to clean it up afterward.
 		 *
 		 * We need to do this after the table state is set to SYNCDONE.
 		 * Otherwise, if an error occurs while performing the database
@@ -360,32 +349,73 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 * have been cleared before restart. So, the restarted worker will use
 		 * invalid replication progress state resulting in replay of
 		 * transactions that have already been applied.
+		 *
+		 * Firstly reset the origin session to remove the ownership of the
+		 * slot. This is needed to allow the origin to be dropped or reused
+		 * later.
 		 */
+		replorigin_session_reset();
+		replorigin_session_origin = InvalidRepOriginId;
+		replorigin_session_origin_lsn = InvalidXLogRecPtr;
+		replorigin_session_origin_timestamp = 0;
+
 		StartTransactionCommand();
+		if (MyLogicalRepWorker->slot_name && strcmp(syncslotname, MyLogicalRepWorker->slot_name) != 0)
+		{
+			/*
+			 * End streaming so that LogRepWorkerWalRcvConn can be used to
+			 * drop the slot.
+			 */
+			walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
 
-		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
+			ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+											   MyLogicalRepWorker->relid,
+											   originname,
+											   sizeof(originname));
+
+			/*
+			 * Drop replication origin
+			 *
+			 * There is a chance that the user is concurrently performing refresh
+			 * for the subscription where we remove the table state and its origin
+			 * or the apply worker would have removed this origin. So passing
+			 * missing_ok = true.
+			 */
+			replorigin_drop_by_name(originname, true, false);
+		}
 
 		/*
-		 * Resetting the origin session removes the ownership of the slot.
-		 * This is needed to allow the origin to be dropped.
+		 * We are safe to remove persisted replication slot and origin data,
+		 * since it's already in SYNCDONE state. They will not be needed
+		 * anymore.
 		 */
-		replorigin_session_reset();
-		replorigin_session_origin = InvalidRepOriginId;
-		replorigin_session_origin_lsn = InvalidXLogRecPtr;
-		replorigin_session_origin_timestamp = 0;
+		UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+							  MyLogicalRepWorker->relid,
+							  MyLogicalRepWorker->relstate,
+							  MyLogicalRepWorker->relstate_lsn,
+							  NULL,
+							  NULL);
+		ereport(DEBUG2,
+				(errmsg("process_syncing_tables_for_sync: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+						"NULL",
+						"NULL",
+						MyLogicalRepWorker->relstate,
+						MyLogicalRepWorker->relid,
+						MyLogicalRepWorker->subid)));
+		CommitTransactionCommand();
+		pgstat_report_stat(false);
 
 		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * There is a chance that the user is concurrently performing refresh
-		 * for the subscription where we remove the table state and its origin
-		 * or the apply worker would have removed this origin. So passing
-		 * missing_ok = true.
+		 * This should return the default origin name for the worker. Even if
+		 * the worker used a different origin for this table, it should be
+		 * dropped and removed from the catalog so far.
 		 */
-		replorigin_drop_by_name(originname, true, false);
+		StartTransactionCommand();
+		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+										   MyLogicalRepWorker->relid,
+										   originname,
+										   sizeof(originname));
 
 		/* Sync worker has completed synchronization of the current table. */
 		MyLogicalRepWorker->is_sync_completed = true;
@@ -482,6 +512,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 			if (current_lsn >= rstate->lsn)
 			{
 				char		originname[NAMEDATALEN];
+				bool		is_origin_null = true;
 
 				rstate->state = SUBREL_STATE_READY;
 				rstate->lsn = current_lsn;
@@ -502,18 +533,31 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				 * error while dropping we won't restart it to drop the
 				 * origin. So passing missing_ok = true.
 				 */
-				ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-												   rstate->relid,
-												   originname,
-												   sizeof(originname));
-				replorigin_drop_by_name(originname, true, false);
+				GetSubscriptionRelOrigin(MyLogicalRepWorker->subid,
+										 rstate->relid, originname,
+										 &is_origin_null);
+
+				if (!is_origin_null)
+				{
+					replorigin_drop_by_name(originname, true, false);
+				}
 
 				/*
 				 * Update the state to READY only after the origin cleanup.
 				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-										   rstate->relid, rstate->state,
-										   rstate->lsn);
+				UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+									  rstate->relid,
+									  rstate->state,
+									  rstate->lsn,
+									  NULL,
+									  NULL);
+				ereport(DEBUG2,
+					(errmsg("process_syncing_tables_for_apply: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+							"NULL", "NULL", rstate->state,
+							rstate->relid, MyLogicalRepWorker->subid)));
+
+				CommitTransactionCommand();
+				started_tx = false;
 			}
 		}
 		else
@@ -602,12 +646,25 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						TimestampDifferenceExceeds(hentry->last_start_time, now,
 												   wal_retrieve_retry_interval))
 					{
+						if (IsTransactionState())
+							CommitTransactionCommand();
+						StartTransactionCommand();
+						started_tx = true;
+
+						MySubscription->lastusedid++;
+						UpdateSubscriptionLastSlotId(MyLogicalRepWorker->subid,
+													 MySubscription->lastusedid);
+						ereport(DEBUG2,
+								(errmsg("process_syncing_tables_for_apply: incremented lastusedid to %lld for subscription %u",
+										(long long) MySubscription->lastusedid, MySubscription->oid)));
+
 						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
 												 MySubscription->oid,
 												 MySubscription->name,
 												 MyLogicalRepWorker->userid,
 												 rstate->relid,
-												 DSM_HANDLE_INVALID);
+												 DSM_HANDLE_INVALID,
+												 MySubscription->lastusedid);
 						hentry->last_start_time = now;
 					}
 				}
@@ -1230,8 +1287,8 @@ copy_table(Relation rel)
  * The name must not exceed NAMEDATALEN - 1 because of remote node constraints
  * on slot name length. We append system_identifier to avoid slot_name
  * collision with subscriptions in other clusters. With the current scheme
- * pg_%u_sync_%u_UINT64_FORMAT (3 + 10 + 6 + 10 + 20 + '\0'), the maximum
- * length of slot_name will be 50.
+ * pg_%u_sync_%lu_UINT64_FORMAT (3 + 10 + 6 + 20 + 20 + '\0'), the maximum
+ * length of slot_name will be 45.
  *
  * The returned slot name is stored in the supplied buffer (syncslotname) with
  * the given size.
@@ -1242,11 +1299,11 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
+ReplicationSlotNameForTablesync(Oid suboid, int64 slotid,
 								char *syncslotname, Size szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%lld_" UINT64_FORMAT, suboid,
+			(long long) slotid, GetSystemIdentifier());
 }
 
 /*
@@ -1290,6 +1347,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	UserContext ucxt;
 	bool		must_use_password;
 	bool		run_as_owner;
+	char	   *prev_slotname;
 
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
@@ -1324,7 +1382,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
+									MyLogicalRepWorker->rep_slot_id,
 									slotname,
 									NAMEDATALEN);
 
@@ -1356,12 +1414,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC ||
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY);
 
+	/*
+	 * See if tablesync of the current relation has been started with another
+	 * replication slot.
+	 *
+	 * Read previous slot name from the catalog, if exists.
+	 */
+	prev_slotname = (char *) palloc(NAMEDATALEN);
+	StartTransactionCommand();
+	GetSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+									  MyLogicalRepWorker->relid,
+									  prev_slotname);
+
 	/* Assign the origin tracking record name. */
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   sizeof(originname));
 
+	CommitTransactionCommand();
+
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC)
 	{
 		/*
@@ -1375,10 +1447,53 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		 * breakdown then it wouldn't have succeeded so trying it next time
 		 * seems like a better bet.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, slotname, true);
+		if (strlen(prev_slotname) > 0)
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);
+
+			StartTransactionCommand();
+			/* Replication origin might still exist. Try to drop */
+			replorigin_drop_by_name(originname, true, false);
+
+			/*
+			 * Remove replication slot and origin name from the relation's
+			 * catalog record
+			 */
+			UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+								  MyLogicalRepWorker->relid,
+								  MyLogicalRepWorker->relstate,
+								  MyLogicalRepWorker->relstate_lsn,
+								  NULL,
+								  NULL);
+			CommitTransactionCommand();
+			ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+						"NULL", "NULL", MyLogicalRepWorker->relstate,
+						MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+		}
 	}
 	else if (MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY)
 	{
+		/*
+		 * At this point, the table that is currently being synchronized
+		 * should have its replication slot name filled in the catalog. The
+		 * tablesync process was started with another sync worker and
+		 * replication slot. We need to continue using the same replication
+		 * slot in this worker too.
+		 */
+		if (strlen(prev_slotname) == 0)
+		{
+			elog(ERROR, "Replication slot could not be found for subscription %u, relation %u",
+				 MyLogicalRepWorker->subid,
+				 MyLogicalRepWorker->relid);
+		}
+
+		/*
+		 * Proceed with the correct replication slot. Use previously created
+		 * replication slot to sync this table.
+		 */
+		memcpy(slotname, prev_slotname, NAMEDATALEN);
+
 		/*
 		 * The COPY phase was previously done, but tablesync then crashed
 		 * before it was able to finish normally.
@@ -1398,7 +1513,9 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 		goto copy_table_done;
 	}
+	pfree(prev_slotname);
 
+	/* Preparing for table copy operation */
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 	MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
 	MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
@@ -1406,11 +1523,31 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   MyLogicalRepWorker->relstate,
-							   MyLogicalRepWorker->relstate_lsn);
+
+	/*
+	 * Refresh the originname in case of having non-existing origin
+	 * from previous failed sync attempts.
+	 * If that's the case, it should be removed from the catalog so far.
+	 * Then, we can continue by reusing the origin created by the current
+	 * worker instead of .
+	 */
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									MyLogicalRepWorker->relid,
+									originname,
+									sizeof(originname));
+
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  MyLogicalRepWorker->relstate,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					slotname, originname, MyLogicalRepWorker->relstate,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+
 	pgstat_report_stat(true);
 
 	StartTransactionCommand();
@@ -1438,48 +1575,96 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						res->err)));
 	walrcv_clear_result(res);
 
+	originid = replorigin_by_name(originname, true);
+
 	/*
 	 * Create a new permanent logical decoding slot. This slot will be used
 	 * for the catchup phase after COPY is done, so tell it to use the
 	 * snapshot to make the final data consistent.
+	 *
+	 * Replication slot will only be created if either this is the first run
+	 * of the worker or we're not using a previous replication slot.
 	 */
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-
-	/*
-	 * Setup replication origin tracking. The purpose of doing this before the
-	 * copy is to avoid doing the copy again due to any error in setting up
-	 * origin tracking.
-	 */
-	originid = replorigin_by_name(originname, true);
-	if (!OidIsValid(originid))
+	if (!MyLogicalRepWorker->created_slot)
 	{
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						   slotname, false /* permanent */ , false /* two_phase */ ,
+						   CRS_USE_SNAPSHOT, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: created replication slot %s for subscription %u",
+						slotname, MyLogicalRepWorker->subid)));
+
 		/*
-		 * Origin tracking does not exist, so create it now.
-		 *
-		 * Then advance to the LSN got from walrcv_create_slot. This is WAL
-		 * logged for the purpose of recovery. Locks are to prevent the
-		 * replication origin from vanishing while advancing.
+		 * Remember that we created the slot so that we will not try to create
+		 * it again.
 		 */
-		originid = replorigin_create(originname);
-
-		LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
-		replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
-						   true /* go backward */ , true /* WAL log */ );
-		UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->created_slot = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
+		/*
+		 * Setup replication origin tracking. The purpose of doing this before
+		 * the copy is to avoid doing the copy again due to any error in
+		 * setting up origin tracking.
+		 */
+		if (!OidIsValid(originid))
+		{
+			/*
+			 * Origin tracking does not exist, so create it now.
+			 */
+			originid = replorigin_create(originname);
+		}
+		else
+		{
+			/*
+			 * At this point, there shouldn't be any existing replication
+			 * origin with the same name.
+			 */
+			ereport(ERROR,
+					(errcode(ERRCODE_DUPLICATE_OBJECT),
+					 errmsg("replication origin \"%s\" already exists",
+							originname)));
+		}
 	}
 	else
 	{
-		ereport(ERROR,
-				(errcode(ERRCODE_DUPLICATE_OBJECT),
-				 errmsg("replication origin \"%s\" already exists",
-						originname)));
+		/*
+		 * Do not create a new replication slot, reuse the existing one
+		 * instead. Use a new snapshot for the replication slot to ensure that
+		 * tablesync and apply proceses are consistent with each other.
+		 */
+		WalRcvStreamOptions options;
+		int			server_version;
+
+		server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+		options.proto.logical.proto_version =
+			server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+			server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+			LOGICALREP_PROTO_VERSION_NUM;
+		options.proto.logical.publication_names = MySubscription->publications;
+
+		walrcv_slot_snapshot(LogRepWorkerWalRcvConn, slotname, &options, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: reusing replication slot %s for relation %u in subscription %u",
+						slotname, MyLogicalRepWorker->relid,
+						MyLogicalRepWorker->subid)));
 	}
 
+	/*
+	 * Advance to the LSN got from walrcv_create_slot or walrcv_slot_snapshot.
+	 * This is WAL logged for the purpose of recovery. Locks are to prevent
+	 * the replication origin from vanishing while advancing.
+	 *
+	 * Then setup replication origin tracking.
+	 */
+	LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+	replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
+					   true /* go backward */ , true /* WAL log */ );
+	UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+
 	/*
 	 * Make sure that the copy command runs as the table owner, unless the
 	 * user has opted out of that behaviour.
@@ -1538,12 +1723,18 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	 * Update the persisted state to indicate the COPY phase is done; make it
 	 * visible to others.
 	 */
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   SUBREL_STATE_FINISHEDCOPY,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  SUBREL_STATE_FINISHEDCOPY,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					originname, slotname, SUBREL_STATE_FINISHEDCOPY,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
 
 copy_table_done:
 
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 4eb67ebd26..c3a6aa3894 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -466,8 +466,16 @@ ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 {
 	if (OidIsValid(relid))
 	{
-		/* Replication origin name for tablesync workers. */
-		snprintf(originname, szoriginname, "pg_%u_%u", suboid, relid);
+		bool		is_null = true;
+
+		/*
+		 * Replication origin name for tablesync workers. First, look into the
+		 * catalog. If originname does not exist, then use the default name.
+		 */
+		GetSubscriptionRelOrigin(suboid, relid,
+								 originname, &is_null);
+		if (is_null)
+			snprintf(originname, szoriginname, "pg_%u_%lld", suboid, (long long) MyLogicalRepWorker->rep_slot_id);
 	}
 	else
 	{
@@ -4503,6 +4511,9 @@ start_table_sync(XLogRecPtr *origin_startpos,
 
 	/* allocate slot name in long-lived context */
 	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+
+	/* Keep the replication slot name used for this sync. */
+	MyLogicalRepWorker->slot_name = *myslotname;
 	pfree(syncslotname);
 }
 
@@ -4555,13 +4566,25 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 {
 	MyLogicalRepWorker->is_sync_completed = false;
 
+	/*
+	 * If it's already connected to the publisher, end streaming before using
+	 * the same connection for another iteration
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		TimeLineID tli;
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	}
+
 	/* Start table synchronization. */
 	start_table_sync(origin_startpos, &slotname);
 
+	StartTransactionCommand();
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   originname_size);
+	CommitTransactionCommand();
 
 	set_apply_error_context_origin(originname);
 
@@ -4601,11 +4624,10 @@ run_apply_worker(WalRcvStreamOptions *options,
 				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
 					errmsg("subscription has no replication slot set")));
 
-	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-									   originname, originname_size);
-
 	/* Setup replication origin tracking. */
 	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
 	originid = replorigin_by_name(originname, true);
 	if (!OidIsValid(originid))
 		originid = replorigin_create(originname);
@@ -4921,7 +4943,32 @@ TablesyncWorkerMain(Datum main_arg)
 			}
 
 			if (!is_table_found)
+			{
+				TimeLineID	tli;
+
+				/*
+				 * It is important to give an error if we are unable to drop the
+				 * slot, otherwise, it won't be dropped till the corresponding
+				 * subscription is dropped. So passing missing_ok = false.
+				 */
+				if (MyLogicalRepWorker->created_slot)
+				{
+					walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+					ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
+				}
+
+				/*
+				 * Drop replication origin before exiting.
+				 *
+				 * There is a chance that the user is concurrently performing refresh
+				 * for the subscription where we remove the table state and its origin
+				 * or the apply worker would have removed this origin. So passing
+				 * missing_ok = true.
+				 */
+				replorigin_drop_by_name(originname, true, false);
+
 				break;
+			}
 		}
 	}
 
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 1d40eebc78..7e13f59847 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -108,6 +108,9 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
 
 	/* Only publish data originating from the specified origin */
 	text		suborigin BKI_DEFAULT(LOGICALREP_ORIGIN_ANY);
+
+	/* The last used ID to create a replication slot for tablesync */
+	int64		sublastusedid BKI_DEFAULT(0);
 #endif
 } FormData_pg_subscription;
 
@@ -144,6 +147,8 @@ typedef struct Subscription
 	List	   *publications;	/* List of publication names to subscribe to */
 	char	   *origin;			/* Only publish data originating from the
 								 * specified origin */
+	int64		lastusedid;		/* Last used unique ID to create replication
+								 * slots in tablesync */
 } Subscription;
 
 /* Disallow streaming in-progress transactions. */
@@ -164,6 +169,7 @@ typedef struct Subscription
 extern Subscription *GetSubscription(Oid subid, bool missing_ok);
 extern void FreeSubscription(Subscription *sub);
 extern void DisableSubscription(Oid subid);
+extern void UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid);
 
 extern int	CountDBSubscriptions(Oid dbid);
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 60a2bcca23..185164d75e 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -44,6 +44,12 @@ CATALOG(pg_subscription_rel,6102,SubscriptionRelRelationId)
 											 * used for synchronization
 											 * coordination, or NULL if not
 											 * valid */
+	NameData	srrelslotname BKI_FORCE_NULL;	/* name of the replication
+												 * slot for relation in
+												 * subscription */
+	NameData	srreloriginname BKI_FORCE_NULL; /* origin name for relation in
+												 * subscription */
+
 #endif
 } FormData_pg_subscription_rel;
 
@@ -81,10 +87,16 @@ typedef struct SubscriptionRelState
 } SubscriptionRelState;
 
 extern void AddSubscriptionRelState(Oid subid, Oid relid, char state,
-									XLogRecPtr sublsn);
+									XLogRecPtr sublsn, char *relslotname, char *reloriginname);
 extern void UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 									   XLogRecPtr sublsn);
+extern void UpdateSubscriptionRel(Oid subid, Oid relid, char state,
+								  XLogRecPtr sublsn, char *relslotname, char *reloriginname);
+
 extern char GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn);
+extern void GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname);
+extern void GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull);
+
 extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 
 extern bool HasSubscriptionRelations(Oid subid);
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index a8a89dc784..31b2c41893 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -237,8 +237,9 @@ extern bool InvalidateObsoleteReplicationSlots(ReplicationSlotInvalidationCause
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
 extern bool ReplicationSlotName(int index, Name name);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, Size szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, int64 slotid, char *syncslotname, Size szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List *GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 32783a8cdd..4b2148d13f 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -35,6 +35,23 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/*
+	 * Indicates if the sync worker created a replication slot for itself
+	 * in any point of its lifetime.
+	 * False means that the worker has not created a slot yet, and has been
+	 * reusing replication slots created by other workers so far.
+	 */
+	bool		created_slot;
+
+	/*
+	 * Unique identifier for replication slot to be created by tablesnync
+	 * workers, if needed.
+	 */
+	int64		rep_slot_id;
+
+	/* Replication slot name used by the worker. */
+	char	   *slot_name;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
@@ -242,7 +259,8 @@ extern LogicalRepWorker *logicalrep_worker_find(Oid subid, Oid relid,
 extern List *logicalrep_workers_find(Oid subid, bool only_running);
 extern bool logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname,
 									 Oid userid, Oid relid,
-									 dsm_handle subworker_dsm);
+									 dsm_handle subworker_dsm,
+									 int64 slotid);
 extern void logicalrep_worker_stop(Oid subid, Oid relid);
 extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
 extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
@@ -338,4 +356,7 @@ am_parallel_apply_worker(void)
 	return isParallelApplyWorker(MyLogicalRepWorker);
 }
 
+/* Invalid identifier to be used for naming replication slots */
+#define InvalidRepSlotId	0
+
 #endif							/* WORKER_INTERNAL_H */
-- 
2.27.0

#96Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Peter Smith (#92)
3 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Peter,

Peter Smith <smithpb2250@gmail.com>, 11 Tem 2023 Sal, 05:59 tarihinde şunu
yazdı:

Even if patches 0003 and 0002 are to be combined, I think that should
not happen until after the "reuse" design is confirmed which way is
best.

e.g. IMO it might be easier to compare the different PoC designs for
patch 0002 if there is no extra logic involved.

PoC design#1 -- each tablesync decides for itself what to do next
after it finishes
PoC design#2 -- reuse tablesync using a "pool" of available workers

Right. I made a patch 0003 to change 0002 so that tables will be assigned
to sync workers by apply worker.
It's a rough POC and ignores some edge cases. But this is what I think how
apply worker would take the responsibility of table assignments. Hope the
implementation makes sense and I'm not missing anything that may cause
degraded perforrmance.

PoC design#1 --> apply only patch 0001 and 0002
PoC design#2 --> apply all patches, 0001, 0002 and 0003

Here are some quick numbers with 100 empty tables.

+--------------+----------------+----------------+----------------+
|              | 2 sync workers | 4 sync workers | 8 sync workers |
+--------------+----------------+----------------+----------------+
| POC design#1 | 1909.873 ms    | 986.261 ms     | 552.404 ms     |
+--------------+----------------+----------------+----------------+
| POC design#2 | 4962.208 ms    | 1240.503 ms    | 1165.405 ms    |
+--------------+----------------+----------------+----------------+
| master       | 2666.008 ms    | 1462.012 ms    | 986.848 ms     |
+--------------+----------------+----------------+----------------+

Seems like design#1 is better than both design#2 and master overall. It's
surprising to see that even master beats design#2 in some cases though. Not
sure if that is expected or there are some places to improve design#2 even
more.

What do you think?

PS: I only attached the related patches and not the whole patch set. 0001
and 0002 may contain some of your earlier reviews, but I'll send a proper
updated set soon.

Thanks,
--
Melih Mutlu
Microsoft

Attachments:

v18-0002-Reuse-Tablesync-Workers.patchapplication/octet-stream; name=v18-0002-Reuse-Tablesync-Workers.patchDownload
From 5b2604f4303c72582bb2179c4d33370df5ffb0d6 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:04:46 +0300
Subject: [PATCH v18 2/3] Reuse Tablesync Workers

Before this patch, tablesync workers were capable of syncing only one
table. For each table, a new sync worker was launched and that worker would
exit when done processing the table.

Now, tablesync workers are not limited to processing only one
table. When done, they can move to processing another table in
the same subscription.

If there is a table that needs to be synced, an available tablesync
worker picks up that table and syncs it. Each tablesync worker
continues to pick new tables to sync until there are no tables left
requiring synchronization. If there was no available worker to
process the table, then a new tablesync worker will be launched,
provided the number of tablesync workers for the subscription does not
exceed max_sync_workers_per_subscription.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/replication/logical/launcher.c  |   1 +
 src/backend/replication/logical/tablesync.c | 139 ++++++++++++++++----
 src/backend/replication/logical/worker.c    |  50 ++++++-
 src/include/replication/worker_internal.h   |   7 +
 4 files changed, 171 insertions(+), 26 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e231fa7f95..72e5ef8a78 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -440,6 +440,7 @@ retry:
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
+	worker->is_sync_completed = false;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index a77d3e3032..46e6f7ea10 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -137,11 +137,11 @@ static StringInfo copybuf = NULL;
  */
 static void
 pg_attribute_noreturn()
-finish_sync_worker(void)
+finish_sync_worker(bool reuse_worker)
 {
 	/*
-	 * Commit any outstanding transaction. This is the usual case, unless
-	 * there was nothing to do for the table.
+	 * Commit any outstanding transaction. This is the usual case, unless there
+	 * was nothing to do for the table.
 	 */
 	if (IsTransactionState())
 	{
@@ -149,21 +149,33 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
-	/* And flush all writes. */
-	XLogFlush(GetXLogWriteRecPtr());
-
-	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
-	CommitTransactionCommand();
+	/*
+	 * Disconnect from the publisher otherwise reusing the sync worker can
+	 * error due to exceeding max_wal_senders.
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		walrcv_disconnect(LogRepWorkerWalRcvConn);
+		LogRepWorkerWalRcvConn = NULL;
+	}
 
 	/* Find the leader apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
 
-	/* Stop gracefully */
-	proc_exit(0);
+	if (!reuse_worker)
+	{
+		/* And flush all writes. */
+		XLogFlush(GetXLogWriteRecPtr());
+
+		StartTransactionCommand();
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\" has finished",
+						MySubscription->name)));
+		CommitTransactionCommand();
+
+		/* Stop gracefully */
+		proc_exit(0);
+	}
 }
 
 /*
@@ -383,7 +395,15 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 */
 		replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
+		/* Sync worker has completed synchronization of the current table. */
+		MyLogicalRepWorker->is_sync_completed = true;
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", relation \"%s\" with relid %u has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
+		CommitTransactionCommand();
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -1288,7 +1308,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		case SUBREL_STATE_SYNCDONE:
 		case SUBREL_STATE_READY:
 		case SUBREL_STATE_UNKNOWN:
-			finish_sync_worker();	/* doesn't return */
+			finish_sync_worker(true);	/* doesn't return */
 	}
 
 	/* Calculate the name of the tablesync slot. */
@@ -1645,6 +1665,8 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 					 int originname_size,
 					 XLogRecPtr *origin_startpos)
 {
+	MyLogicalRepWorker->is_sync_completed = false;
+
 	/* Start table synchronization. */
 	start_table_sync(origin_startpos, &slotname);
 
@@ -1707,13 +1729,86 @@ TablesyncWorkerMain(Datum main_arg)
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	run_tablesync_worker(&options,
-						 myslotname,
-						 originname,
-						 sizeof(originname),
-						 &origin_startpos);
+	/*
+	 * The loop where worker does its job. It loops until there is no relation
+	 * left to sync.
+	 */
+	for (;;)
+	{
+		List	   *rstates;
+		ListCell   *lc;
+		bool 	is_table_found = false;
+
+		run_tablesync_worker(&options,
+							 myslotname,
+							 originname,
+							 sizeof(originname),
+							 &origin_startpos);
+
+		if (IsTransactionState())
+			CommitTransactionCommand();
+
+		if (MyLogicalRepWorker->is_sync_completed)
+		{
+			/* This transaction will be committed by finish_sync_worker. */
+			StartTransactionCommand();
+
+			/*
+			 * Check if there is any table whose relation state is still INIT.
+			 * If a table in INIT state is found, the worker will not be
+			 * finished, it will be reused instead.
+			 */
+			rstates = GetSubscriptionRelations(MySubscription->oid, true);
+
+			foreach(lc, rstates)
+			{
+				SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
+
+				if (rstate->state == SUBREL_STATE_SYNCDONE)
+					continue;
+
+				/*
+				 * Take exclusive lock to prevent any other sync worker from
+				 * picking the same table.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
+
+				/*
+				 * Pick the table for the next run if it is not already picked
+				 * up by another worker.
+				 */
+				if (!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+				{
+					/* Update worker state for the next table */
+					MyLogicalRepWorker->relid = rstate->relid;
+					MyLogicalRepWorker->relstate = rstate->state;
+					MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+					LWLockRelease(LogicalRepWorkerLock);
+
+					/* Found a table for next iteration */
+					is_table_found = true;
+					finish_sync_worker(true);
+
+					StartTransactionCommand();
+					ereport(LOG,
+							(errmsg("%s for subscription \"%s\" will be reused to sync table \"%s\" with relid %u.",
+									get_worker_name(),
+									MySubscription->name,
+									get_rel_name(MyLogicalRepWorker->relid),
+									MyLogicalRepWorker->relid)));
+					CommitTransactionCommand();
+
+					break;
+				}
+				LWLockRelease(LogicalRepWorkerLock);
+			}
+
+			if (!is_table_found)
+				break;
+		}
+	}
 
-	finish_sync_worker();
+	finish_sync_worker(false);
 }
 
 /*
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index a9956834d0..5714028e84 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -433,6 +433,20 @@ static inline void reset_apply_error_context_info(void);
 static TransApplyAction get_transaction_apply_action(TransactionId xid,
 													 ParallelApplyWorkerInfo **winfo);
 
+/*
+ * Return the name of the logical replication worker.
+ */
+const char *
+get_worker_name(void)
+{
+	if (am_tablesync_worker())
+		return _("logical replication table synchronization worker");
+	else if (am_parallel_apply_worker())
+		return _("logical replication parallel apply worker");
+	else
+		return _("logical replication apply worker");
+}
+
 /*
  * Form the origin name for the subscription.
  *
@@ -3607,6 +3621,20 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can call process_syncing_tables_for_sync.
+				 *
+				 * process_syncing_tables_for_sync decides whether the sync of
+				 * the current table is completed. If it is completed,
+				 * streaming must be already ended. So, we can break the loop.
+				 */
+				if (MyLogicalRepWorker->is_sync_completed)
+				{
+					endofstream = true;
+					break;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -3626,6 +3654,15 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+
+			/*
+			 * If is_sync_completed is true, this means that the tablesync
+			 * worker is done with synchronization. Streaming has already been
+			 * ended by process_syncing_tables_for_sync. We should move to the
+			 * next table if needed, or exit.
+			 */
+			if (MyLogicalRepWorker->is_sync_completed)
+				endofstream = true;
 		}
 
 		/* Cleanup the memory. */
@@ -3728,8 +3765,12 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * End streaming here for only apply workers. Ending streaming for
+	 * tablesync workers is deferred until the worker exits its main loop.
+	 */
+	if (!am_tablesync_worker())
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
 }
 
 /*
@@ -4603,10 +4644,11 @@ InitializeLogRepWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("%s for subscription \"%s\", table \"%s\" has started",
+				(errmsg("%s for subscription \"%s\", table \"%s\" with relid %u has started",
 						get_worker_name(),
 						MySubscription->name,
-						get_rel_name(MyLogicalRepWorker->relid))));
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
 	else
 		ereport(LOG,
 				(errmsg("logical replication apply worker for subscription \"%s\" has started",
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index e7d03cb559..9c0237fe0b 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -57,6 +57,12 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/*
+	 * Indicates whether tablesync worker has completed sycning its assigned
+	 * table. If true, no need to continue with that table.
+	 */
+	bool		is_sync_completed;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction or
@@ -333,5 +339,6 @@ extern void set_stream_options(WalRcvStreamOptions *options,
 							   XLogRecPtr *origin_startpos);
 extern void start_apply(XLogRecPtr origin_startpos);
 extern void DisableSubscriptionAndExit(void);
+extern const char * get_worker_name(void);
 
 #endif							/* WORKER_INTERNAL_H */
-- 
2.25.1

v18-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchapplication/octet-stream; name=v18-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchDownload
From 630abad7d0283f071fcfaff06b0623a0d86b48c7 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Mon, 5 Jun 2023 15:04:41 +0300
Subject: [PATCH v18 1/3] Refactor to split Apply and Tablesync Workers

Both apply and tablesync workers were using ApplyWorkerMain() as entry
point. As the name implies, ApplyWorkerMain() should be considered as
the main function for apply workers. Tablesync worker's path was hidden
and does not have enough in common to share the same main function with
apply worker.

Also, most of the code shared by both worker types is already combined
in LogicalRepApplyLoop(). There is no need to combine the rest in
ApplyWorkerMain() anymore.

This patch introduces TablesyncWorkerMain() as a new entry point for
tablesync workers and separates both type of workers from each other.
This aims to increase code readability and help to maintain logical
replication workers separately.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/postmaster/bgworker.c             |   3 +
 .../replication/logical/applyparallelworker.c |   2 +-
 src/backend/replication/logical/launcher.c    |  32 +-
 src/backend/replication/logical/tablesync.c   | 132 +++++++
 src/backend/replication/logical/worker.c      | 365 ++++++++----------
 src/include/replication/logicalworker.h       |   1 +
 src/include/replication/worker_internal.h     |  10 +-
 7 files changed, 329 insertions(+), 216 deletions(-)

diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 5b4bd71694..505e38376c 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -131,6 +131,9 @@ static const struct
 	},
 	{
 		"ParallelApplyWorkerMain", ParallelApplyWorkerMain
+	},
+	{
+		"TablesyncWorkerMain", TablesyncWorkerMain
 	}
 };
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 6fb96148f4..1d4e83c4c1 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -942,7 +942,7 @@ ParallelApplyWorkerMain(Datum main_arg)
 	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
 		MyLogicalRepWorker->reply_time = 0;
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 542af7d863..e231fa7f95 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -459,24 +459,30 @@ retry:
 	snprintf(bgw.bgw_library_name, MAXPGPATH, "postgres");
 
 	if (is_parallel_apply_worker)
+	{
 		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ParallelApplyWorkerMain");
-	else
-		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
-
-	if (OidIsValid(relid))
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication worker for subscription %u sync %u", subid, relid);
-	else if (is_parallel_apply_worker)
+				 "logical replication parallel apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
+	}
+	else if (OidIsValid(relid))
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "TablesyncWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication parallel apply worker for subscription %u", subid);
+				 "logical replication tablesync worker for subscription %u sync %u",
+				 subid,
+				 relid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication tablesync worker");
+	}
 	else
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication apply worker for subscription %u", subid);
-
-	if (is_parallel_apply_worker)
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
-	else
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication worker");
+				 "logical replication apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication apply worker");
+	}
 
 	bgw.bgw_restart_time = BGW_NEVER_RESTART;
 	bgw.bgw_notify_pid = MyProcPid;
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 6d461654ab..a77d3e3032 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -104,17 +104,21 @@
 #include "nodes/makefuncs.h"
 #include "parser/parse_relation.h"
 #include "pgstat.h"
+#include "postmaster/interrupt.h"
 #include "replication/logicallauncher.h"
 #include "replication/logicalrelation.h"
+#include "replication/logicalworker.h"
 #include "replication/walreceiver.h"
 #include "replication/worker_internal.h"
 #include "replication/slot.h"
 #include "replication/origin.h"
 #include "storage/ipc.h"
 #include "storage/lmgr.h"
+#include "tcop/tcopprot.h"
 #include "utils/acl.h"
 #include "utils/array.h"
 #include "utils/builtins.h"
+#include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
 #include "utils/rls.h"
@@ -1584,6 +1588,134 @@ FetchTableStates(bool *started_tx)
 	return has_subrels;
 }
 
+/*
+ * Execute the initial sync with error handling. Disable the subscription,
+ * if it's required.
+ *
+ * Allocate the slot name in long-lived context on return. Note that we don't
+ * handle FATAL errors which are probably because of system resource error and
+ * are not repeatable.
+ */
+static void
+start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+{
+	char	   *syncslotname = NULL;
+
+	Assert(am_tablesync_worker());
+
+	PG_TRY();
+	{
+		/* Call initial sync. */
+		syncslotname = LogicalRepSyncTableStart(origin_startpos);
+	}
+	PG_CATCH();
+	{
+		if (MySubscription->disableonerr)
+			DisableSubscriptionAndExit();
+		else
+		{
+			/*
+			 * Report the worker failed during table synchronization. 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();
+
+	/* allocate slot name in long-lived context */
+	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+	pfree(syncslotname);
+}
+
+/*
+ * Runs the tablesync worker.
+ *
+ * It starts syncing tables. After a successful sync, sets streaming options
+ * and starts streaming to catchup.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Start table synchronization. */
+	start_table_sync(origin_startpos, &slotname);
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   originname_size);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	/* Start applying changes to catchup. */
+	start_apply(*origin_startpos);
+}
+
+/* Logical Replication Tablesync worker entry point */
+void
+TablesyncWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *myslotname = NULL;
+	WalRcvStreamOptions options;
+
+	/* Attach to slot */
+	logicalrep_worker_attach(worker_slot);
+
+	/* Setup signal handling */
+	pqsignal(SIGHUP, SignalHandlerForConfigReload);
+	pqsignal(SIGTERM, die);
+	BackgroundWorkerUnblockSignals();
+
+	/*
+	 * We don't currently need any ResourceOwner in a walreceiver process, but
+	 * if we did, we could call CreateAuxProcessResourceOwner here.
+	 */
+
+	/* Initialise stats to a sanish value */
+	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
+		MyLogicalRepWorker->reply_time = GetCurrentTimestamp();
+
+	/* Load the libpq-specific functions */
+	load_file("libpqwalreceiver", false);
+
+	InitializeLogRepWorker();
+
+	/* Connect to the origin and start the replication. */
+	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
+		 MySubscription->conninfo);
+
+	/*
+	 * Setup callback for syscache so that we know when something changes in
+	 * the subscription relation state.
+	 */
+	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
+								  invalidate_syncing_table_states,
+								  (Datum) 0);
+
+	run_tablesync_worker(&options,
+						 myslotname,
+						 originname,
+						 sizeof(originname),
+						 &origin_startpos);
+
+	finish_sync_worker();
+}
+
 /*
  * If the subscription has no tables then return false.
  *
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index dd353fd1cb..a9956834d0 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -395,8 +395,6 @@ static void stream_close_file(void);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
-static void DisableSubscriptionAndExit(void);
-
 static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
 static void apply_handle_insert_internal(ApplyExecutionData *edata,
 										 ResultRelInfo *relinfo,
@@ -4313,6 +4311,71 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+ /*
+  * Sets streaming options including replication slot name and origin start
+  * position. Workers need these options for logical replication.
+  */
+void
+set_stream_options(WalRcvStreamOptions *options,
+				   char *slotname,
+				   XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (!am_tablesync_worker() &&
+		MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+		options->proto.logical.twophase = true;
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4347,24 +4410,18 @@ TwoPhaseTransactionGid(Oid subid, TransactionId xid, char *gid, int szgid)
 }
 
 /*
- * Execute the initial sync with error handling. Disable the subscription,
- * if it's required.
+ * Run the apply loop with error handling. Disable the subscription,
+ * if necessary.
  *
- * Allocate the slot name in long-lived context on return. Note that we don't
- * handle FATAL errors which are probably because of system resource error and
- * are not repeatable.
+ * Note that we don't handle FATAL errors which are probably because
+ * of system resource error and are not repeatable.
  */
-static void
-start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+void
+start_apply(XLogRecPtr origin_startpos)
 {
-	char	   *syncslotname = NULL;
-
-	Assert(am_tablesync_worker());
-
 	PG_TRY();
 	{
-		/* Call initial sync. */
-		syncslotname = LogicalRepSyncTableStart(origin_startpos);
+		LogicalRepApplyLoop(origin_startpos);
 	}
 	PG_CATCH();
 	{
@@ -4373,65 +4430,118 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 		else
 		{
 			/*
-			 * Report the worker failed during table synchronization. Abort
-			 * the current transaction so that the stats message is sent in an
+			 * Report the worker failed while applying changes. Abort the
+			 * current transaction so that the stats message is sent in an
 			 * idle state.
 			 */
 			AbortOutOfAnyTransaction();
-			pgstat_report_subscription_error(MySubscription->oid, false);
+			pgstat_report_subscription_error(MySubscription->oid, !am_tablesync_worker());
 
 			PG_RE_THROW();
 		}
 	}
 	PG_END_TRY();
-
-	/* allocate slot name in long-lived context */
-	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
-	pfree(syncslotname);
 }
 
 /*
- * Run the apply loop with error handling. Disable the subscription,
- * if necessary.
+ * Runs the leader apply worker.
  *
- * Note that we don't handle FATAL errors which are probably because
- * of system resource error and are not repeatable.
+ * It sets up replication origin, streaming options and then starts streaming.
  */
 static void
-start_apply(XLogRecPtr origin_startpos)
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
 {
-	PG_TRY();
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+	bool		must_use_password;
+
+	slotname = MySubscription->slotname;
+
+	/*
+	 * This shouldn't happen if the subscription is enabled, but guard
+	 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+	 * crash if slot is NULL.)
+	 */
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+
+	/* Is the use of a password mandatory? */
+	must_use_password = MySubscription->passwordrequired &&
+		!superuser_arg(MySubscription->owner);
+
+	/* Note that the superuser_arg call can access the DB */
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											must_use_password,
+											MySubscription->name, &err);
+
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+	 * We don't really use the output identify_system for anything but it
+	 * does some initializations on the upstream so let's still call it.
+	 */
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
 	{
-		LogicalRepApplyLoop(origin_startpos);
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
 	}
-	PG_CATCH();
-	{
-		if (MySubscription->disableonerr)
-			DisableSubscriptionAndExit();
-		else
-		{
-			/*
-			 * Report the worker failed while applying changes. Abort the
-			 * current transaction so that the stats message is sent in an
-			 * idle state.
-			 */
-			AbortOutOfAnyTransaction();
-			pgstat_report_subscription_error(MySubscription->oid, !am_tablesync_worker());
 
-			PG_RE_THROW();
-		}
-	}
-	PG_END_TRY();
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+
+	/* Run the main loop. */
+	start_apply(*origin_startpos);
 }
 
 /*
- * Common initialization for leader apply worker and parallel apply worker.
+ * Common initialization for leader apply worker, parallel apply worker and
+ * tablesync worker.
  *
  * Initialize the database connection, in-memory subscription and necessary
  * config options.
  */
 void
-InitializeApplyWorker(void)
+InitializeLogRepWorker(void)
 {
 	MemoryContext oldctx;
 
@@ -4493,7 +4603,8 @@ InitializeApplyWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has started",
+				(errmsg("%s for subscription \"%s\", table \"%s\" has started",
+						get_worker_name(),
 						MySubscription->name,
 						get_rel_name(MyLogicalRepWorker->relid))));
 	else
@@ -4513,7 +4624,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	InitializingApplyWorker = true;
 
@@ -4537,7 +4647,7 @@ ApplyWorkerMain(Datum main_arg)
 	/* Load the libpq-specific functions */
 	load_file("libpqwalreceiver", false);
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
@@ -4545,71 +4655,6 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-		bool		must_use_password;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-
-		/* Is the use of a password mandatory? */
-		must_use_password = MySubscription->passwordrequired &&
-			!superuser_arg(MySubscription->owner);
-
-		/* Note that the superuser_arg call can access the DB */
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												must_use_password,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
 	 * the subscription relation state.
@@ -4618,90 +4663,8 @@ ApplyWorkerMain(Datum main_arg)
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
-
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
-
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-
-	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
-	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
-
-	if (!am_tablesync_worker())
-	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
-		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
-		}
-		else
-		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-		}
-
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-								 MySubscription->name,
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-								 "?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
-
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+	/* This is leader apply worker */
+	run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 
 	proc_exit(0);
 }
@@ -4710,7 +4673,7 @@ ApplyWorkerMain(Datum main_arg)
  * After error recovery, disable the subscription in a new transaction
  * and exit cleanly.
  */
-static void
+void
 DisableSubscriptionAndExit(void)
 {
 	/*
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 39588da79f..bbd71d0b42 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -18,6 +18,7 @@ extern PGDLLIMPORT volatile sig_atomic_t ParallelApplyMessagePending;
 
 extern void ApplyWorkerMain(Datum main_arg);
 extern void ParallelApplyWorkerMain(Datum main_arg);
+extern void TablesyncWorkerMain(Datum main_arg);
 
 extern bool IsLogicalWorker(void);
 extern bool IsLogicalParallelApplyWorker(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 343e781896..e7d03cb559 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -19,6 +19,7 @@
 #include "datatype/timestamp.h"
 #include "miscadmin.h"
 #include "replication/logicalrelation.h"
+#include "replication/walreceiver.h"
 #include "storage/buffile.h"
 #include "storage/fileset.h"
 #include "storage/lock.h"
@@ -265,7 +266,7 @@ extern void maybe_reread_subscription(void);
 
 extern void stream_cleanup_files(Oid subid, TransactionId xid);
 
-extern void InitializeApplyWorker(void);
+extern void InitializeLogRepWorker(void);
 
 extern void store_flush_position(XLogRecPtr remote_lsn, XLogRecPtr local_lsn);
 
@@ -326,4 +327,11 @@ am_parallel_apply_worker(void)
 	return isParallelApplyWorker(MyLogicalRepWorker);
 }
 
+
+extern void set_stream_options(WalRcvStreamOptions *options,
+							   char *slotname,
+							   XLogRecPtr *origin_startpos);
+extern void start_apply(XLogRecPtr origin_startpos);
+extern void DisableSubscriptionAndExit(void);
+
 #endif							/* WORKER_INTERNAL_H */
-- 
2.25.1

v18-0003-apply-worker-assigns-tables.patchapplication/octet-stream; name=v18-0003-apply-worker-assigns-tables.patchDownload
From 6238a3cb7c3a261192182c5b3656f822be19621f Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 13 Jul 2023 17:15:47 +0300
Subject: [PATCH v18 3/3] apply worker assigns tables

---
 src/backend/replication/logical/launcher.c  |  32 ++++++
 src/backend/replication/logical/tablesync.c | 121 ++++++++++----------
 src/include/replication/worker_internal.h   |   1 +
 3 files changed, 94 insertions(+), 60 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 72e5ef8a78..d40c17c6f3 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -297,6 +297,38 @@ logicalrep_workers_find(Oid subid, bool only_running)
 	return res;
 }
 
+/*
+ * Return a logical rep worker in ready state
+ */
+LogicalRepWorker *
+logicalrep_worker_find_syncdone(Oid subid, bool only_running)
+{
+	int			i;
+	LogicalRepWorker *res = NULL;
+
+	Assert(LWLockHeldByMe(LogicalRepWorkerLock));
+
+	/* Search for attached worker for a given subscription id. */
+	for (i = 0; i < max_logical_replication_workers; i++)
+	{
+		LogicalRepWorker *w = &LogicalRepCtx->workers[i];
+
+		/* Skip parallel apply workers. */
+		if (isParallelApplyWorker(w))
+			continue;
+
+		if (w->in_use && w->subid == subid &&
+			w->relstate == SUBREL_STATE_SYNCDONE &&
+			(!only_running || w->proc))
+		{
+			res = w;
+			break;
+		}
+	}
+
+	return res;
+}
+
 /*
  * Start new logical replication background worker, if possible.
  *
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 46e6f7ea10..51c1a607f3 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -136,7 +136,6 @@ static StringInfo copybuf = NULL;
  * Exit routine for synchronization worker.
  */
 static void
-pg_attribute_noreturn()
 finish_sync_worker(bool reuse_worker)
 {
 	/*
@@ -247,6 +246,12 @@ wait_for_worker_state_change(char expected_state)
 
 		CHECK_FOR_INTERRUPTS();
 
+		/* No table needs sync anymore. Apply worker wants this sync worker to exit. */
+		if (!OidIsValid(MyLogicalRepWorker->relid))
+		{
+			return false;
+		}
+
 		/*
 		 * Done if already in correct state.  (We assume this fetch is atomic
 		 * enough to not give a misleading answer if we do it with no lock.)
@@ -469,8 +474,27 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 	 */
 	else if (table_states_not_ready == NIL && last_start_times)
 	{
+		List	   *workers;
+
 		hash_destroy(last_start_times);
 		last_start_times = NULL;
+
+		/* Let all sync workers exit */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		workers = logicalrep_workers_find(MyLogicalRepWorker->subid, false);
+		foreach(lc, workers)
+		{
+			LogicalRepWorker *w = (LogicalRepWorker *) lfirst(lc);
+
+			if (OidIsValid(w->relid))
+			{
+				SpinLockAcquire(&w->relmutex);
+				w->relid = InvalidOid;
+				SpinLockRelease(&w->relmutex);
+				logicalrep_worker_wakeup_ptr(w);
+			}
+		}
+		LWLockRelease(LogicalRepWorkerLock);
 	}
 
 	/*
@@ -619,6 +643,33 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						hentry->last_start_time = now;
 					}
 				}
+				else
+				{
+					/*
+					 * We reached the max_sync_workers_per_subscription limit.
+					 * Check if there is an existing sync worker waiting for
+					 * new table to sync.
+					 */
+					LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+
+					syncworker = logicalrep_worker_find_syncdone(MyLogicalRepWorker->subid, false);
+
+					if (syncworker)
+					{
+						SpinLockAcquire(&syncworker->relmutex);
+						syncworker->relid = rstate->relid;
+						syncworker->relstate = rstate->state;
+						syncworker->relstate_lsn = rstate->lsn;
+						SpinLockRelease(&syncworker->relmutex);
+
+						if (syncworker->proc)
+						{
+							logicalrep_worker_wakeup_ptr(syncworker);
+						}
+					}
+
+					LWLockRelease(LogicalRepWorkerLock);
+				}
 			}
 		}
 	}
@@ -1308,7 +1359,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		case SUBREL_STATE_SYNCDONE:
 		case SUBREL_STATE_READY:
 		case SUBREL_STATE_UNKNOWN:
-			finish_sync_worker(true);	/* doesn't return */
+			finish_sync_worker(false);	/* doesn't return */
 	}
 
 	/* Calculate the name of the tablesync slot. */
@@ -1735,10 +1786,6 @@ TablesyncWorkerMain(Datum main_arg)
 	 */
 	for (;;)
 	{
-		List	   *rstates;
-		ListCell   *lc;
-		bool 	is_table_found = false;
-
 		run_tablesync_worker(&options,
 							 myslotname,
 							 originname,
@@ -1748,63 +1795,17 @@ TablesyncWorkerMain(Datum main_arg)
 		if (IsTransactionState())
 			CommitTransactionCommand();
 
+		finish_sync_worker(true);
+
 		if (MyLogicalRepWorker->is_sync_completed)
 		{
-			/* This transaction will be committed by finish_sync_worker. */
-			StartTransactionCommand();
-
-			/*
-			 * Check if there is any table whose relation state is still INIT.
-			 * If a table in INIT state is found, the worker will not be
-			 * finished, it will be reused instead.
-			 */
-			rstates = GetSubscriptionRelations(MySubscription->oid, true);
-
-			foreach(lc, rstates)
-			{
-				SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-
-				if (rstate->state == SUBREL_STATE_SYNCDONE)
-					continue;
-
-				/*
-				 * Take exclusive lock to prevent any other sync worker from
-				 * picking the same table.
-				 */
-				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
-
-				/*
-				 * Pick the table for the next run if it is not already picked
-				 * up by another worker.
-				 */
-				if (!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
-				{
-					/* Update worker state for the next table */
-					MyLogicalRepWorker->relid = rstate->relid;
-					MyLogicalRepWorker->relstate = rstate->state;
-					MyLogicalRepWorker->relstate_lsn = rstate->lsn;
-					LWLockRelease(LogicalRepWorkerLock);
-
-					/* Found a table for next iteration */
-					is_table_found = true;
-					finish_sync_worker(true);
-
-					StartTransactionCommand();
-					ereport(LOG,
-							(errmsg("%s for subscription \"%s\" will be reused to sync table \"%s\" with relid %u.",
-									get_worker_name(),
-									MySubscription->name,
-									get_rel_name(MyLogicalRepWorker->relid),
-									MyLogicalRepWorker->relid)));
-					CommitTransactionCommand();
-
-					break;
-				}
-				LWLockRelease(LogicalRepWorkerLock);
-			}
+			/* wait for apply worker to assign a new table with INIT state. */
+			wait_for_worker_state_change(SUBREL_STATE_INIT);
+		}
 
-			if (!is_table_found)
-				break;
+		if (!OidIsValid(MyLogicalRepWorker->relid))
+		{
+			break;
 		}
 	}
 
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 9c0237fe0b..7aa0f2be63 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -238,6 +238,7 @@ 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 LogicalRepWorker * logicalrep_worker_find_syncdone(Oid subid, bool only_running);
 extern bool logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname,
 									 Oid userid, Oid relid,
 									 dsm_handle subworker_dsm);
-- 
2.25.1

#97Amit Kapila
amit.kapila16@gmail.com
In reply to: Melih Mutlu (#96)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Fri, Jul 14, 2023 at 1:58 AM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Here are some quick numbers with 100 empty tables.

+--------------+----------------+----------------+----------------+
|              | 2 sync workers | 4 sync workers | 8 sync workers |
+--------------+----------------+----------------+----------------+
| POC design#1 | 1909.873 ms    | 986.261 ms     | 552.404 ms     |
+--------------+----------------+----------------+----------------+
| POC design#2 | 4962.208 ms    | 1240.503 ms    | 1165.405 ms    |
+--------------+----------------+----------------+----------------+
| master       | 2666.008 ms    | 1462.012 ms    | 986.848 ms     |
+--------------+----------------+----------------+----------------+

Seems like design#1 is better than both design#2 and master overall. It's surprising to see that even master beats design#2 in some cases though. Not sure if that is expected or there are some places to improve design#2 even more.

Yeah, it is quite surprising that Design#2 is worse than master. I
suspect there is something wrong going on with your Design#2 patch.
One area to check is whether apply worker is able to quickly assign
the new relations to tablesync workers. Note that currently after the
first time assigning the tables to workers, the apply worker may wait
before processing the next set of tables in the main loop of
LogicalRepApplyLoop(). The other minor point about design#2
implementation is that you may want to first assign the allocated
tablesync workers before trying to launch a new worker.

PS: I only attached the related patches and not the whole patch set. 0001 and 0002 may contain some of your earlier reviews, but I'll send a proper updated set soon.

Yeah, that would be helpful.

--
With Regards,
Amit Kapila.

#98Peter Smith
smithpb2250@gmail.com
In reply to: Hayato Kuroda (Fujitsu) (#95)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Kuroda-san.

Here are some review comments for the v17-0003 patch. They are all minor.

======
Commit message

1.
Previously tablesync workers establish new connections when it changes
the syncing
table, but this might have additional overhead. This patch allows to
reuse connections
instead.

~

/This patch allows to reuse connections instead./This patch allows the
existing connection to be reused./

~~~

2.
As for the publisher node, this patch allows to reuse logical
walsender processes
after the streaming is done once.

~

Is this paragraph even needed? Since the connection is reused then it
already implies the other end (the Wlasender) is being reused, right?

======
src/backend/replication/logical/tablesync.c

3.
+ * FIXME: set appropriate application_name. Previously, the slot name was used
+ * because the lifetime of the tablesync worker was same as that, but now the
+ * tablesync worker handles many slots during the synchronization so that it is
+ * not suitable. So what should be? Note that if the tablesync worker starts to
+ * reuse the replication slot during synchronization, we should use the slot
+ * name as application_name again.
+ */
+static void
+ApplicationNameForTablesync(Oid suboid, int worker_slot,
+ char *application_name, Size szapp)

3a.
I felt that most of this FIXME comment belongs with the calling code,
not here.

3b.
Also, maybe it needs some rewording -- I didn't understand exactly
what it is trying to say.

~~~

4.
- /*
- * Here we use the slot name instead of the subscription name as the
- * application_name, so that it is different from the leader apply worker,
- * so that synchronous replication can distinguish them.
- */
- LogRepWorkerWalRcvConn =
- walrcv_connect(MySubscription->conninfo, true,
-    must_use_password,
-    slotname, &err);
+ /* Connect to the publisher if haven't done so already. */
+ if (LogRepWorkerWalRcvConn == NULL)
+ {
+ char application_name[NAMEDATALEN];
+
+ /*
+ * The application_name must be also different from the leader apply
+ * worker because synchronous replication must distinguish them.
+ */
+ ApplicationNameForTablesync(MySubscription->oid,
+ MyLogicalRepWorker->worker_slot,
+ application_name,
+ NAMEDATALEN);
+ LogRepWorkerWalRcvConn =
+ walrcv_connect(MySubscription->conninfo, true,
+    must_use_password,
+    application_name, &err);
+ }
+

Should the comment mention the "subscription name" as it did before?

SUGGESTION
The application_name must differ from the subscription name (used by
the leader apply worker) because synchronous replication has to be
able to distinguish this worker from the leader apply worker.

======
src/backend/replication/logical/worker.c

5.
-start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+start_table_sync(XLogRecPtr *origin_startpos,
+ char **myslotname)

This is a wrapping change only. It looks like an unnecessary hangover
from a previous version of 0003.

======
src/backend/replication/walsender.c

6. exec_replication_command

+
if (cmd->kind == REPLICATION_KIND_PHYSICAL)
StartReplication(cmd);
~

The extra blank line does not belong in this patch.

======
src/include/replication/worker_internal.h

+ /* Indicates the slot number which corresponds to this LogicalRepWorker. */
+ int worker_slot;
+

6a
I think this field is very fundamental, so IMO it should be defined at
the top of the struct, maybe nearby the other 'in_use' and
'generation' fields.

~

6b.
Also, since this is already a "worker" struct so there is no need to
have "worker" in the field name again -- just "slot_number" or
"slotnum" might be a better name.

And then the comment can also be simplified.

SUGGESTION
/* Slot number of this worker. */
int slotnum;

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#99Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Amit Kapila (#97)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi,

Amit Kapila <amit.kapila16@gmail.com>, 14 Tem 2023 Cum, 11:11 tarihinde
şunu yazdı:

Yeah, it is quite surprising that Design#2 is worse than master. I
suspect there is something wrong going on with your Design#2 patch.
One area to check is whether apply worker is able to quickly assign
the new relations to tablesync workers. Note that currently after the
first time assigning the tables to workers, the apply worker may wait
before processing the next set of tables in the main loop of
LogicalRepApplyLoop(). The other minor point about design#2
implementation is that you may want to first assign the allocated
tablesync workers before trying to launch a new worker.

It's not actually worse than master all the time. It seems like it's just
unreliable.
Here are some consecutive runs for both designs and master.

design#1 = 1621,527 ms, 1788,533 ms, 1645,618 ms, 1702,068 ms, 1745,753 ms
design#2 = 2089,077 ms, 1864,571 ms, 4574,799 ms, 5422,217 ms, 1905,944 ms
master = 2815,138 ms, 2481,954 ms , 2594,413 ms, 2620,690 ms, 2489,323 ms

And apply worker was not busy with applying anything during these
experiments since there were not any writes to the publisher. I'm not sure
how that would also affect the performance if there were any writes.

Thanks,
--
Melih Mutlu
Microsoft

#100Amit Kapila
amit.kapila16@gmail.com
In reply to: Melih Mutlu (#99)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Fri, Jul 14, 2023 at 3:07 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Amit Kapila <amit.kapila16@gmail.com>, 14 Tem 2023 Cum, 11:11 tarihinde şunu yazdı:

Yeah, it is quite surprising that Design#2 is worse than master. I
suspect there is something wrong going on with your Design#2 patch.
One area to check is whether apply worker is able to quickly assign
the new relations to tablesync workers. Note that currently after the
first time assigning the tables to workers, the apply worker may wait
before processing the next set of tables in the main loop of
LogicalRepApplyLoop(). The other minor point about design#2
implementation is that you may want to first assign the allocated
tablesync workers before trying to launch a new worker.

It's not actually worse than master all the time. It seems like it's just unreliable.
Here are some consecutive runs for both designs and master.

design#1 = 1621,527 ms, 1788,533 ms, 1645,618 ms, 1702,068 ms, 1745,753 ms
design#2 = 2089,077 ms, 1864,571 ms, 4574,799 ms, 5422,217 ms, 1905,944 ms
master = 2815,138 ms, 2481,954 ms , 2594,413 ms, 2620,690 ms, 2489,323 ms

And apply worker was not busy with applying anything during these experiments since there were not any writes to the publisher. I'm not sure how that would also affect the performance if there were any writes.

Yeah, this is a valid point. I think this is in favor of the Design#1
approach we are discussing here. One thing I was thinking whether we
can do anything to alleviate the contention at the higher worker
count. One possibility is to have some kind of available worker list
which can be used to pick up the next worker instead of checking all
the workers while assigning the next table. We can probably explore it
separately once the first three patches are ready because anyway, this
will be an optimization atop the Design#1 approach.

--
With Regards,
Amit Kapila.

#101Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Amit Kapila (#100)
5 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi,

PFA updated patches. Rebased 0003 with minor changes. Addressed Peter's
reviews for 0001 and 0002 with some small comments below.

Peter Smith <smithpb2250@gmail.com>, 10 Tem 2023 Pzt, 10:09 tarihinde şunu
yazdı:

6. LogicalRepApplyLoop

+ /*
+ * apply_dispatch() may have gone into apply_handle_commit()
+ * which can call process_syncing_tables_for_sync.
+ *
+ * process_syncing_tables_for_sync decides whether the sync of
+ * the current table is completed. If it is completed,
+ * streaming must be already ended. So, we can break the loop.
+ */
+ if (MyLogicalRepWorker->is_sync_completed)
+ {
+ endofstream = true;
+ break;
+ }
+

and

+ /*
+ * If is_sync_completed is true, this means that the tablesync
+ * worker is done with synchronization. Streaming has already been
+ * ended by process_syncing_tables_for_sync. We should move to the
+ * next table if needed, or exit.
+ */
+ if (MyLogicalRepWorker->is_sync_completed)
+ endofstream = true;

~

Instead of those code fragments above assigning 'endofstream' as a
side-effect, would it be the same (but tidier) to just modify the
other "breaking" condition below:

BEFORE:
/* Check if we need to exit the streaming loop. */
if (endofstream)
break;

AFTER:
/* Check if we need to exit the streaming loop. */
if (endofstream || MyLogicalRepWorker->is_sync_completed)
break;

First place you mentioned also breaks the infinite loop. Such an if
statement is needed there with or without endofstream assignment.

I think if there is a flag to break a loop, using that flag to indicate
that we should exit the loop seems more appropriate to me. I see that it
would be a bit tidier without endofstream = true lines, but I feel like it
would also be less readable.

I don't have a strong opinion though. I'm just keeping them as they are for
now, but I can change them if you disagree.

10b.
All the other tablesync-related fields of this struct are named as
relXXX, so I wonder if is better for this to follow the same pattern.
e.g. 'relsync_completed'

Aren't those start with rel because they're related to the relation that
the tablesync worker is syncing? is_sync_completed is not a relation
specific field. I'm okay with changing the name but feel like
relsync_completed would be misleading.

Thanks,
--
Melih Mutlu
Microsoft

Attachments:

v19-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchapplication/octet-stream; name=v19-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchDownload
From 64014cf3a7f5525279ac18815709192f86a41c9c Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Mon, 5 Jun 2023 15:04:41 +0300
Subject: [PATCH v19 1/5] Refactor to split Apply and Tablesync Workers

Both apply and tablesync workers were using ApplyWorkerMain() as entry
point. As the name implies, ApplyWorkerMain() should be considered as
the main function for apply workers. Tablesync worker's path was hidden
and does not have enough in common to share the same main function with
apply worker.

Also, most of the code shared by both worker types is already combined
in LogicalRepApplyLoop(). There is no need to combine the rest in
ApplyWorkerMain() anymore.

This patch introduces TablesyncWorkerMain() as a new entry point for
tablesync workers. This aims to increase code readability and help to
maintain logical replication workers separately.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/postmaster/bgworker.c             |   3 +
 .../replication/logical/applyparallelworker.c |   2 +-
 src/backend/replication/logical/launcher.c    |  32 +-
 src/backend/replication/logical/tablesync.c   | 132 +++++++
 src/backend/replication/logical/worker.c      | 365 ++++++++----------
 src/include/replication/logicalworker.h       |   1 +
 src/include/replication/worker_internal.h     |  10 +-
 7 files changed, 329 insertions(+), 216 deletions(-)

diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 5b4bd71694..505e38376c 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -131,6 +131,9 @@ static const struct
 	},
 	{
 		"ParallelApplyWorkerMain", ParallelApplyWorkerMain
+	},
+	{
+		"TablesyncWorkerMain", TablesyncWorkerMain
 	}
 };
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 6fb96148f4..1d4e83c4c1 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -942,7 +942,7 @@ ParallelApplyWorkerMain(Datum main_arg)
 	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
 		MyLogicalRepWorker->reply_time = 0;
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 542af7d863..e231fa7f95 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -459,24 +459,30 @@ retry:
 	snprintf(bgw.bgw_library_name, MAXPGPATH, "postgres");
 
 	if (is_parallel_apply_worker)
+	{
 		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ParallelApplyWorkerMain");
-	else
-		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
-
-	if (OidIsValid(relid))
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication worker for subscription %u sync %u", subid, relid);
-	else if (is_parallel_apply_worker)
+				 "logical replication parallel apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
+	}
+	else if (OidIsValid(relid))
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "TablesyncWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication parallel apply worker for subscription %u", subid);
+				 "logical replication tablesync worker for subscription %u sync %u",
+				 subid,
+				 relid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication tablesync worker");
+	}
 	else
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication apply worker for subscription %u", subid);
-
-	if (is_parallel_apply_worker)
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
-	else
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication worker");
+				 "logical replication apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication apply worker");
+	}
 
 	bgw.bgw_restart_time = BGW_NEVER_RESTART;
 	bgw.bgw_notify_pid = MyProcPid;
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 6d461654ab..a77d3e3032 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -104,17 +104,21 @@
 #include "nodes/makefuncs.h"
 #include "parser/parse_relation.h"
 #include "pgstat.h"
+#include "postmaster/interrupt.h"
 #include "replication/logicallauncher.h"
 #include "replication/logicalrelation.h"
+#include "replication/logicalworker.h"
 #include "replication/walreceiver.h"
 #include "replication/worker_internal.h"
 #include "replication/slot.h"
 #include "replication/origin.h"
 #include "storage/ipc.h"
 #include "storage/lmgr.h"
+#include "tcop/tcopprot.h"
 #include "utils/acl.h"
 #include "utils/array.h"
 #include "utils/builtins.h"
+#include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
 #include "utils/rls.h"
@@ -1584,6 +1588,134 @@ FetchTableStates(bool *started_tx)
 	return has_subrels;
 }
 
+/*
+ * Execute the initial sync with error handling. Disable the subscription,
+ * if it's required.
+ *
+ * Allocate the slot name in long-lived context on return. Note that we don't
+ * handle FATAL errors which are probably because of system resource error and
+ * are not repeatable.
+ */
+static void
+start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+{
+	char	   *syncslotname = NULL;
+
+	Assert(am_tablesync_worker());
+
+	PG_TRY();
+	{
+		/* Call initial sync. */
+		syncslotname = LogicalRepSyncTableStart(origin_startpos);
+	}
+	PG_CATCH();
+	{
+		if (MySubscription->disableonerr)
+			DisableSubscriptionAndExit();
+		else
+		{
+			/*
+			 * Report the worker failed during table synchronization. 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();
+
+	/* allocate slot name in long-lived context */
+	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+	pfree(syncslotname);
+}
+
+/*
+ * Runs the tablesync worker.
+ *
+ * It starts syncing tables. After a successful sync, sets streaming options
+ * and starts streaming to catchup.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Start table synchronization. */
+	start_table_sync(origin_startpos, &slotname);
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   originname_size);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	/* Start applying changes to catchup. */
+	start_apply(*origin_startpos);
+}
+
+/* Logical Replication Tablesync worker entry point */
+void
+TablesyncWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *myslotname = NULL;
+	WalRcvStreamOptions options;
+
+	/* Attach to slot */
+	logicalrep_worker_attach(worker_slot);
+
+	/* Setup signal handling */
+	pqsignal(SIGHUP, SignalHandlerForConfigReload);
+	pqsignal(SIGTERM, die);
+	BackgroundWorkerUnblockSignals();
+
+	/*
+	 * We don't currently need any ResourceOwner in a walreceiver process, but
+	 * if we did, we could call CreateAuxProcessResourceOwner here.
+	 */
+
+	/* Initialise stats to a sanish value */
+	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
+		MyLogicalRepWorker->reply_time = GetCurrentTimestamp();
+
+	/* Load the libpq-specific functions */
+	load_file("libpqwalreceiver", false);
+
+	InitializeLogRepWorker();
+
+	/* Connect to the origin and start the replication. */
+	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
+		 MySubscription->conninfo);
+
+	/*
+	 * Setup callback for syscache so that we know when something changes in
+	 * the subscription relation state.
+	 */
+	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
+								  invalidate_syncing_table_states,
+								  (Datum) 0);
+
+	run_tablesync_worker(&options,
+						 myslotname,
+						 originname,
+						 sizeof(originname),
+						 &origin_startpos);
+
+	finish_sync_worker();
+}
+
 /*
  * If the subscription has no tables then return false.
  *
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index dd353fd1cb..e7d43e297e 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -395,8 +395,6 @@ static void stream_close_file(void);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
-static void DisableSubscriptionAndExit(void);
-
 static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
 static void apply_handle_insert_internal(ApplyExecutionData *edata,
 										 ResultRelInfo *relinfo,
@@ -4313,6 +4311,71 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+/*
+ * Sets streaming options including replication slot name and origin start
+ * position. Workers need these options for logical replication.
+ */
+void
+set_stream_options(WalRcvStreamOptions *options,
+				   char *slotname,
+				   XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (!am_tablesync_worker() &&
+		MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+		options->proto.logical.twophase = true;
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4347,24 +4410,18 @@ TwoPhaseTransactionGid(Oid subid, TransactionId xid, char *gid, int szgid)
 }
 
 /*
- * Execute the initial sync with error handling. Disable the subscription,
- * if it's required.
+ * Run the apply loop with error handling. Disable the subscription,
+ * if necessary.
  *
- * Allocate the slot name in long-lived context on return. Note that we don't
- * handle FATAL errors which are probably because of system resource error and
- * are not repeatable.
+ * Note that we don't handle FATAL errors which are probably because
+ * of system resource error and are not repeatable.
  */
-static void
-start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+void
+start_apply(XLogRecPtr origin_startpos)
 {
-	char	   *syncslotname = NULL;
-
-	Assert(am_tablesync_worker());
-
 	PG_TRY();
 	{
-		/* Call initial sync. */
-		syncslotname = LogicalRepSyncTableStart(origin_startpos);
+		LogicalRepApplyLoop(origin_startpos);
 	}
 	PG_CATCH();
 	{
@@ -4373,65 +4430,118 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 		else
 		{
 			/*
-			 * Report the worker failed during table synchronization. Abort
-			 * the current transaction so that the stats message is sent in an
+			 * Report the worker failed while applying changes. Abort the
+			 * current transaction so that the stats message is sent in an
 			 * idle state.
 			 */
 			AbortOutOfAnyTransaction();
-			pgstat_report_subscription_error(MySubscription->oid, false);
+			pgstat_report_subscription_error(MySubscription->oid, !am_tablesync_worker());
 
 			PG_RE_THROW();
 		}
 	}
 	PG_END_TRY();
-
-	/* allocate slot name in long-lived context */
-	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
-	pfree(syncslotname);
 }
 
 /*
- * Run the apply loop with error handling. Disable the subscription,
- * if necessary.
+ * Runs the leader apply worker.
  *
- * Note that we don't handle FATAL errors which are probably because
- * of system resource error and are not repeatable.
+ * It sets up replication origin, streaming options and then starts streaming.
  */
 static void
-start_apply(XLogRecPtr origin_startpos)
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
 {
-	PG_TRY();
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+	bool		must_use_password;
+
+	slotname = MySubscription->slotname;
+
+	/*
+	 * This shouldn't happen if the subscription is enabled, but guard
+	 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+	 * crash if slot is NULL.)
+	 */
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+
+	/* Is the use of a password mandatory? */
+	must_use_password = MySubscription->passwordrequired &&
+		!superuser_arg(MySubscription->owner);
+
+	/* Note that the superuser_arg call can access the DB */
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											must_use_password,
+											MySubscription->name, &err);
+
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+	 * We don't really use the output identify_system for anything but it
+	 * does some initializations on the upstream so let's still call it.
+	 */
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
 	{
-		LogicalRepApplyLoop(origin_startpos);
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
 	}
-	PG_CATCH();
-	{
-		if (MySubscription->disableonerr)
-			DisableSubscriptionAndExit();
-		else
-		{
-			/*
-			 * Report the worker failed while applying changes. Abort the
-			 * current transaction so that the stats message is sent in an
-			 * idle state.
-			 */
-			AbortOutOfAnyTransaction();
-			pgstat_report_subscription_error(MySubscription->oid, !am_tablesync_worker());
 
-			PG_RE_THROW();
-		}
-	}
-	PG_END_TRY();
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+
+	/* Run the main loop. */
+	start_apply(*origin_startpos);
 }
 
 /*
- * Common initialization for leader apply worker and parallel apply worker.
+ * Common initialization for leader apply worker, parallel apply worker and
+ * tablesync worker.
  *
  * Initialize the database connection, in-memory subscription and necessary
  * config options.
  */
 void
-InitializeApplyWorker(void)
+InitializeLogRepWorker(void)
 {
 	MemoryContext oldctx;
 
@@ -4493,7 +4603,8 @@ InitializeApplyWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has started",
+				(errmsg("%s for subscription \"%s\", table \"%s\" has started",
+						get_worker_name(),
 						MySubscription->name,
 						get_rel_name(MyLogicalRepWorker->relid))));
 	else
@@ -4513,7 +4624,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	InitializingApplyWorker = true;
 
@@ -4537,7 +4647,7 @@ ApplyWorkerMain(Datum main_arg)
 	/* Load the libpq-specific functions */
 	load_file("libpqwalreceiver", false);
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
@@ -4545,71 +4655,6 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-		bool		must_use_password;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-
-		/* Is the use of a password mandatory? */
-		must_use_password = MySubscription->passwordrequired &&
-			!superuser_arg(MySubscription->owner);
-
-		/* Note that the superuser_arg call can access the DB */
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												must_use_password,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
 	 * the subscription relation state.
@@ -4618,90 +4663,8 @@ ApplyWorkerMain(Datum main_arg)
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
-
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
-
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-
-	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
-	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
-
-	if (!am_tablesync_worker())
-	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
-		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
-		}
-		else
-		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-		}
-
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-								 MySubscription->name,
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-								 "?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
-
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+	/* This is leader apply worker */
+	run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 
 	proc_exit(0);
 }
@@ -4710,7 +4673,7 @@ ApplyWorkerMain(Datum main_arg)
  * After error recovery, disable the subscription in a new transaction
  * and exit cleanly.
  */
-static void
+void
 DisableSubscriptionAndExit(void)
 {
 	/*
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 39588da79f..bbd71d0b42 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -18,6 +18,7 @@ extern PGDLLIMPORT volatile sig_atomic_t ParallelApplyMessagePending;
 
 extern void ApplyWorkerMain(Datum main_arg);
 extern void ParallelApplyWorkerMain(Datum main_arg);
+extern void TablesyncWorkerMain(Datum main_arg);
 
 extern bool IsLogicalWorker(void);
 extern bool IsLogicalParallelApplyWorker(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 343e781896..e7d03cb559 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -19,6 +19,7 @@
 #include "datatype/timestamp.h"
 #include "miscadmin.h"
 #include "replication/logicalrelation.h"
+#include "replication/walreceiver.h"
 #include "storage/buffile.h"
 #include "storage/fileset.h"
 #include "storage/lock.h"
@@ -265,7 +266,7 @@ extern void maybe_reread_subscription(void);
 
 extern void stream_cleanup_files(Oid subid, TransactionId xid);
 
-extern void InitializeApplyWorker(void);
+extern void InitializeLogRepWorker(void);
 
 extern void store_flush_position(XLogRecPtr remote_lsn, XLogRecPtr local_lsn);
 
@@ -326,4 +327,11 @@ am_parallel_apply_worker(void)
 	return isParallelApplyWorker(MyLogicalRepWorker);
 }
 
+
+extern void set_stream_options(WalRcvStreamOptions *options,
+							   char *slotname,
+							   XLogRecPtr *origin_startpos);
+extern void start_apply(XLogRecPtr origin_startpos);
+extern void DisableSubscriptionAndExit(void);
+
 #endif							/* WORKER_INTERNAL_H */
-- 
2.25.1

v19-0002-Reuse-Tablesync-Workers.patchapplication/octet-stream; name=v19-0002-Reuse-Tablesync-Workers.patchDownload
From 226dd086edb780351cc72e4c6a1513ad9bde77c4 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:04:46 +0300
Subject: [PATCH v19 2/5] Reuse Tablesync Workers

Before this patch, tablesync workers were capable of syncing only one
table. For each table, a new sync worker was launched and that worker would
exit when done processing the table.

Now, tablesync workers are not limited to processing only one
table. When done, they can move to processing another table in
the same subscription.

If there is a table that needs to be synced, an available tablesync
worker picks up that table and syncs it. Each tablesync worker
continues to pick new tables to sync until there are no tables left
requiring synchronization. If there was no available worker to
process the table, then a new tablesync worker will be launched,
provided the number of tablesync workers for the subscription does not
exceed max_sync_workers_per_subscription.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/replication/logical/launcher.c  |   1 +
 src/backend/replication/logical/tablesync.c | 134 +++++++++++++++++---
 src/backend/replication/logical/worker.c    |  50 +++++++-
 src/include/replication/worker_internal.h   |   7 +
 4 files changed, 169 insertions(+), 23 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e231fa7f95..72e5ef8a78 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -440,6 +440,7 @@ retry:
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
+	worker->is_sync_completed = false;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index a77d3e3032..bc7d943b87 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -134,10 +134,11 @@ static StringInfo copybuf = NULL;
 
 /*
  * Exit routine for synchronization worker.
+ *
+ * If reuse_worker is false, the worker will not be reused and exit.
  */
 static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+finish_sync_worker(bool reuse_worker)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -149,21 +150,33 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
+	/*
+	 * Disconnect from the publisher otherwise reusing the sync worker can
+	 * error due to exceeding max_wal_senders.
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		walrcv_disconnect(LogRepWorkerWalRcvConn);
+		LogRepWorkerWalRcvConn = NULL;
+	}
+
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
-	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
-	CommitTransactionCommand();
-
 	/* Find the leader apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
 
-	/* Stop gracefully */
-	proc_exit(0);
+	if (!reuse_worker)
+	{
+		StartTransactionCommand();
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\" has finished",
+						MySubscription->name)));
+		CommitTransactionCommand();
+
+		/* Stop gracefully */
+		proc_exit(0);
+	}
 }
 
 /*
@@ -383,7 +396,15 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 */
 		replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
+		/* Sync worker has completed synchronization of the current table. */
+		MyLogicalRepWorker->is_sync_completed = true;
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", relation \"%s\" with relid %u has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
+		CommitTransactionCommand();
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -1288,7 +1309,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		case SUBREL_STATE_SYNCDONE:
 		case SUBREL_STATE_READY:
 		case SUBREL_STATE_UNKNOWN:
-			finish_sync_worker();	/* doesn't return */
+			finish_sync_worker(false);	/* doesn't return */
 	}
 
 	/* Calculate the name of the tablesync slot. */
@@ -1645,6 +1666,8 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 					 int originname_size,
 					 XLogRecPtr *origin_startpos)
 {
+	MyLogicalRepWorker->is_sync_completed = false;
+
 	/* Start table synchronization. */
 	start_table_sync(origin_startpos, &slotname);
 
@@ -1672,6 +1695,7 @@ TablesyncWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
+	bool 		done = false;
 
 	/* Attach to slot */
 	logicalrep_worker_attach(worker_slot);
@@ -1707,13 +1731,85 @@ TablesyncWorkerMain(Datum main_arg)
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	run_tablesync_worker(&options,
-						 myslotname,
-						 originname,
-						 sizeof(originname),
-						 &origin_startpos);
+	/*
+	 * The loop where worker does its job. It loops until there is no relation
+	 * left to sync.
+	 */
+	for (;!done;)
+	{
+		List	   *rstates;
+		ListCell   *lc;
+
+		run_tablesync_worker(&options,
+							 myslotname,
+							 originname,
+							 sizeof(originname),
+							 &origin_startpos);
+
+		if (IsTransactionState())
+			CommitTransactionCommand();
+
+		if (MyLogicalRepWorker->is_sync_completed)
+		{
+			/* tablesync is done unless a table that needs syncning is found */
+			done = true;
+
+			/* This transaction will be committed by finish_sync_worker. */
+			StartTransactionCommand();
+
+			/*
+			 * Check if there is any table whose relation state is still INIT.
+			 * If a table in INIT state is found, the worker will not be
+			 * finished, it will be reused instead.
+			 */
+			rstates = GetSubscriptionRelations(MySubscription->oid, true);
+
+			foreach(lc, rstates)
+			{
+				SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
+
+				if (rstate->state == SUBREL_STATE_SYNCDONE)
+					continue;
+
+				/*
+				 * Take exclusive lock to prevent any other sync worker from
+				 * picking the same table.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
+
+				/*
+				 * Pick the table for the next run if it is not already picked
+				 * up by another worker.
+				 */
+				if (!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+				{
+					/* Update worker state for the next table */
+					MyLogicalRepWorker->relid = rstate->relid;
+					MyLogicalRepWorker->relstate = rstate->state;
+					MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+					LWLockRelease(LogicalRepWorkerLock);
+
+					/* Found a table for next iteration */
+					finish_sync_worker(true);
+
+					StartTransactionCommand();
+					ereport(LOG,
+							(errmsg("%s for subscription \"%s\" will be reused to sync table \"%s\" with relid %u.",
+									get_worker_name(),
+									MySubscription->name,
+									get_rel_name(MyLogicalRepWorker->relid),
+									MyLogicalRepWorker->relid)));
+					CommitTransactionCommand();
+
+					done = false;
+					break;
+				}
+				LWLockRelease(LogicalRepWorkerLock);
+			}
+		}
+	}
 
-	finish_sync_worker();
+	finish_sync_worker(false);
 }
 
 /*
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index e7d43e297e..e5d3f10240 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -433,6 +433,20 @@ static inline void reset_apply_error_context_info(void);
 static TransApplyAction get_transaction_apply_action(TransactionId xid,
 													 ParallelApplyWorkerInfo **winfo);
 
+/*
+ * Return the name of the logical replication worker.
+ */
+const char *
+get_worker_name(void)
+{
+	if (am_tablesync_worker())
+		return _("logical replication table synchronization worker");
+	else if (am_parallel_apply_worker())
+		return _("logical replication parallel apply worker");
+	else
+		return _("logical replication apply worker");
+}
+
 /*
  * Form the origin name for the subscription.
  *
@@ -3607,6 +3621,20 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can call process_syncing_tables_for_sync.
+				 *
+				 * process_syncing_tables_for_sync decides whether the sync of
+				 * the current table is completed. If it is completed,
+				 * streaming must be already ended. So, we can break the loop.
+				 */
+				if (MyLogicalRepWorker->is_sync_completed)
+				{
+					endofstream = true;
+					break;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -3626,6 +3654,15 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+
+			/*
+			 * If is_sync_completed is true, this means that the tablesync
+			 * worker is done with synchronization. Streaming has already been
+			 * ended by process_syncing_tables_for_sync. We should move to the
+			 * next table if needed, or exit.
+			 */
+			if (MyLogicalRepWorker->is_sync_completed)
+				endofstream = true;
 		}
 
 		/* Cleanup the memory. */
@@ -3728,8 +3765,12 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * End streaming here for only apply workers. Ending streaming for
+	 * tablesync workers is deferred until the worker exits its main loop.
+	 */
+	if (!am_tablesync_worker())
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
 }
 
 /*
@@ -4603,10 +4644,11 @@ InitializeLogRepWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("%s for subscription \"%s\", table \"%s\" has started",
+				(errmsg("%s for subscription \"%s\", table \"%s\" with relid %u has started",
 						get_worker_name(),
 						MySubscription->name,
-						get_rel_name(MyLogicalRepWorker->relid))));
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
 	else
 		ereport(LOG,
 				(errmsg("logical replication apply worker for subscription \"%s\" has started",
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index e7d03cb559..819c7dca5c 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -57,6 +57,12 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/*
+	 * Indicates whether tablesync worker has completed syncing its assigned
+	 * table.
+	 */
+	bool		is_sync_completed;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction or
@@ -333,5 +339,6 @@ extern void set_stream_options(WalRcvStreamOptions *options,
 							   XLogRecPtr *origin_startpos);
 extern void start_apply(XLogRecPtr origin_startpos);
 extern void DisableSubscriptionAndExit(void);
+extern const char * get_worker_name(void);
 
 #endif							/* WORKER_INTERNAL_H */
-- 
2.25.1

v19-0003-Reuse-connection-when-tablesync-workers-change-t.patchapplication/octet-stream; name=v19-0003-Reuse-connection-when-tablesync-workers-change-t.patchDownload
From c15916c8e3f806e18f97e91329f4038b0e16e6a0 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:13:52 +0300
Subject: [PATCH v19 3/5] Reuse connection when tablesync workers change the
 target

Previously tablesync workers establish new connections when it changes the syncing
table, but this might have additional overhead. This patch allows to reuse connections
instead.

As for the publisher node, this patch allows to reuse logical walsender processes
after the streaming is done once.
---
 src/backend/replication/logical/launcher.c  |  1 +
 src/backend/replication/logical/tablesync.c | 56 ++++++++++++++-------
 src/backend/replication/logical/worker.c    | 18 ++++---
 src/backend/replication/walsender.c         |  7 +++
 src/include/replication/worker_internal.h   |  3 ++
 5 files changed, 58 insertions(+), 27 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 72e5ef8a78..945619b603 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
 	worker->is_sync_completed = false;
+	worker->worker_slot = slot;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index bc7d943b87..d380abfa47 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -150,16 +150,6 @@ finish_sync_worker(bool reuse_worker)
 		pgstat_report_stat(true);
 	}
 
-	/*
-	 * Disconnect from the publisher otherwise reusing the sync worker can
-	 * error due to exceeding max_wal_senders.
-	 */
-	if (LogRepWorkerWalRcvConn != NULL)
-	{
-		walrcv_disconnect(LogRepWorkerWalRcvConn);
-		LogRepWorkerWalRcvConn = NULL;
-	}
-
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
@@ -1258,6 +1248,24 @@ ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
 			 relid, GetSystemIdentifier());
 }
 
+/*
+ * Determine the application_name for tablesync workers.
+ *
+ * FIXME: set appropriate application_name. Previously, the slot name was used
+ * because the lifetime of the tablesync worker was same as that, but now the
+ * tablesync worker handles many slots during the synchronization so that it is
+ * not suitable. So what should be? Note that if the tablesync worker starts to
+ * reuse the replication slot during synchronization, we should use the slot
+ * name as application_name again.
+ */
+static void
+ApplicationNameForTablesync(Oid suboid, int worker_slot,
+							char *application_name, Size szapp)
+{
+	snprintf(application_name, szapp, "pg_%u_sync_%i_" UINT64_FORMAT, suboid,
+			 worker_slot, GetSystemIdentifier());
+}
+
 /*
  * Start syncing the table in the sync worker.
  *
@@ -1319,15 +1327,25 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 									slotname,
 									NAMEDATALEN);
 
-	/*
-	 * Here we use the slot name instead of the subscription name as the
-	 * application_name, so that it is different from the leader apply worker,
-	 * so that synchronous replication can distinguish them.
-	 */
-	LogRepWorkerWalRcvConn =
-		walrcv_connect(MySubscription->conninfo, true,
-					   must_use_password,
-					   slotname, &err);
+	/* Connect to the publisher if haven't done so already. */
+	if (LogRepWorkerWalRcvConn == NULL)
+	{
+		char application_name[NAMEDATALEN];
+
+		/*
+		 * The application_name must be also different from the leader apply
+		 * worker because synchronous replication must distinguish them.
+		 */
+		ApplicationNameForTablesync(MySubscription->oid,
+									MyLogicalRepWorker->worker_slot,
+									application_name,
+									NAMEDATALEN);
+		LogRepWorkerWalRcvConn =
+			walrcv_connect(MySubscription->conninfo, true,
+						   must_use_password,
+						   application_name, &err);
+	}
+
 	if (LogRepWorkerWalRcvConn == NULL)
 		ereport(ERROR,
 				(errcode(ERRCODE_CONNECTION_FAILURE),
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index e5d3f10240..10a356cc28 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3494,20 +3494,22 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	ErrorContextCallback errcallback;
 
 	/*
-	 * Init the ApplyMessageContext which we clean up after each replication
-	 * protocol message.
+	 * Init the ApplyMessageContext if needed. This context is cleaned up
+	 * after each replication protocol message.
 	 */
-	ApplyMessageContext = AllocSetContextCreate(ApplyContext,
-												"ApplyMessageContext",
-												ALLOCSET_DEFAULT_SIZES);
+	if (!ApplyMessageContext)
+		ApplyMessageContext = AllocSetContextCreate(ApplyContext,
+													"ApplyMessageContext",
+													ALLOCSET_DEFAULT_SIZES);
 
 	/*
 	 * This memory context is used for per-stream data when the streaming mode
 	 * is enabled. This context is reset on each stream stop.
 	 */
-	LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
-													"LogicalStreamingContext",
-													ALLOCSET_DEFAULT_SIZES);
+	if (!LogicalStreamingContext)
+		LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
+														"LogicalStreamingContext",
+														ALLOCSET_DEFAULT_SIZES);
 
 	/* mark as idle, before starting to loop */
 	pgstat_report_activity(STATE_IDLE, NULL);
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index d27ef2985d..2f3e93cc40 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1830,7 +1830,14 @@ exec_replication_command(const char *cmd_string)
 				if (cmd->kind == REPLICATION_KIND_PHYSICAL)
 					StartReplication(cmd);
 				else
+				{
+					/*
+					 * Reset flags because reusing tablesync workers can mean
+					 * this is the second time here.
+					 */
+					streamingDoneSending = streamingDoneReceiving = false;
 					StartLogicalReplication(cmd);
+				}
 
 				/* dupe, but necessary per libpqrcv_endstreaming */
 				EndReplicationCommand(cmdtag);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 819c7dca5c..36c4ff3f59 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -63,6 +63,9 @@ typedef struct LogicalRepWorker
 	 */
 	bool		is_sync_completed;
 
+	/* Indicates the slot number which corresponds to this LogicalRepWorker. */
+	int			worker_slot;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction or
-- 
2.25.1

v19-0004-Add-replication-protocol-cmd-to-create-a-snapsho.patchapplication/octet-stream; name=v19-0004-Add-replication-protocol-cmd-to-create-a-snapsho.patchDownload
From 72418c1e233abd163f35616d9897600de9d8794f Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 13 Oct 2022 17:05:45 +0300
Subject: [PATCH v19 4/5] Add replication protocol cmd to create a snapshot

Introduced CREATE_REPLICATION_SNAPSHOT to be able to create and use a
snapshot without creating a new replication slot, but by using an
existing slot.

CREATE_REPLICATION_SNAPSHOT simply does what CREATE_REPLICATION_SLOT does
without creating a new replication slot.

CREATE_REPLICATION_SNAPSHOT command imports the snapshot into the current
transaction and returns consistent_point. The changes earlier than the
consistent_point will be applied by importing the snapshot. All changes
later than the consistent_point will be available to be consumed from
the replication slot.

This is useful for reusing replication slots in logical replication.
Otherwise, tablesync workers cannot start from a consistent point to copy
a relation and then apply changes by consuming from replication slot.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/protocol.sgml                    |  31 ++++++
 .../libpqwalreceiver/libpqwalreceiver.c       |  69 +++++++++++-
 src/backend/replication/logical/logical.c     |  40 ++++++-
 .../replication/logical/logicalfuncs.c        |   1 +
 src/backend/replication/repl_gram.y           |  18 ++-
 src/backend/replication/repl_scanner.l        |   2 +
 src/backend/replication/slotfuncs.c           |   1 +
 src/backend/replication/walsender.c           | 104 +++++++++++++++++-
 src/include/nodes/replnodes.h                 |  11 ++
 src/include/replication/logical.h             |   1 +
 src/include/replication/walreceiver.h         |  13 +++
 src/tools/pgindent/typedefs.list              |   2 +
 12 files changed, 289 insertions(+), 4 deletions(-)

diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index b11d9a6ba3..f7d5acee93 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2595,6 +2595,37 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
      </listitem>
     </varlistentry>
 
+    <varlistentry id="protocol-replication-replication-slot-snapshot">
+     <term><literal>CREATE_REPLICATION_SNAPSHOT</literal> <replaceable class="parameter">slot_name</replaceable> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
+      <indexterm><primary>CREATE_REPLICATION_SNAPSHOT</primary></indexterm>
+     </term>
+     <listitem>
+      <para>
+       Creates a snapshot including all the changes from the replication slot until
+       the point at which the replication slot becomes consistent. Then the snapshot
+       is used in the current transaction. This command is currently only supported
+       for logical replication slots.
+      </para>
+
+      <para>
+       In response to this command, the server will return a one-row result set,
+       containing the following field:
+       <variablelist>
+        <varlistentry>
+         <term><literal>consistent_point</literal> (<type>text</type>)</term>
+         <listitem>
+          <para>
+           The WAL location at which the slot became consistent.  This is the
+           earliest location from which streaming can start on this replication
+           slot.
+          </para>
+         </listitem>
+        </varlistentry>
+       </variablelist>
+      </para>
+     </listitem>
+    </varlistentry>
+
     <varlistentry id="protocol-replication-base-backup" xreflabel="BASE_BACKUP">
      <term><literal>BASE_BACKUP</literal> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
       <indexterm><primary>BASE_BACKUP</primary></indexterm>
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index 60d5c1fc40..ac929fe41e 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -82,6 +82,8 @@ static WalRcvExecResult *libpqrcv_exec(WalReceiverConn *conn,
 									   const int nRetTypes,
 									   const Oid *retTypes);
 static void libpqrcv_disconnect(WalReceiverConn *conn);
+static void libpqrcv_slot_snapshot(WalReceiverConn *conn, char *slotname,
+								   const WalRcvStreamOptions *options, XLogRecPtr *lsn);
 
 static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_connect = libpqrcv_connect,
@@ -98,7 +100,8 @@ static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_create_slot = libpqrcv_create_slot,
 	.walrcv_get_backend_pid = libpqrcv_get_backend_pid,
 	.walrcv_exec = libpqrcv_exec,
-	.walrcv_disconnect = libpqrcv_disconnect
+	.walrcv_disconnect = libpqrcv_disconnect,
+	.walrcv_slot_snapshot = libpqrcv_slot_snapshot
 };
 
 /* Prototypes for private functions */
@@ -982,6 +985,70 @@ libpqrcv_create_slot(WalReceiverConn *conn, const char *slotname,
 	return snapshot;
 }
 
+/*
+ * Create a new snapshot by using an existing replication slot.
+ */
+static void
+libpqrcv_slot_snapshot(WalReceiverConn *conn,
+					   char *slotname,
+					   const WalRcvStreamOptions *options,
+					   XLogRecPtr *lsn)
+{
+	StringInfoData cmd;
+	PGresult   *res;
+	char	   *pubnames_str;
+	List	   *pubnames;
+	char	   *pubnames_literal;
+
+	initStringInfo(&cmd);
+
+	/* Build the command. */
+	appendStringInfo(&cmd, "CREATE_REPLICATION_SNAPSHOT \"%s\"", slotname);
+	appendStringInfoString(&cmd, " (");
+	appendStringInfo(&cmd, " proto_version '%u'",
+					 options->proto.logical.proto_version);
+
+	/* Add publication names. */
+	pubnames = options->proto.logical.publication_names;
+	pubnames_str = stringlist_to_identifierstr(conn->streamConn, pubnames);
+	if (!pubnames_str)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("Could not create a snapshot by replication slot %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	pubnames_literal = PQescapeLiteral(conn->streamConn, pubnames_str,
+									   strlen(pubnames_str));
+	if (!pubnames_literal)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("Could not create a snapshot by replication slot %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	appendStringInfo(&cmd, ", publication_names %s", pubnames_literal);
+	PQfreemem(pubnames_literal);
+	pfree(pubnames_str);
+
+	appendStringInfoString(&cmd, " )");
+
+	/* Execute the command. */
+	res = libpqrcv_PQexec(conn->streamConn, cmd.data);
+	pfree(cmd.data);
+
+	if (PQresultStatus(res) != PGRES_TUPLES_OK)
+	{
+		PQclear(res);
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg("Could not create a snapshot by replication slot \"%s\": %s",
+						slotname, pchomp(PQerrorMessage(conn->streamConn)))));
+	}
+
+	if (lsn)
+		*lsn = DatumGetLSN(DirectFunctionCall1Coll(pg_lsn_in, InvalidOid,
+												   CStringGetDatum(PQgetvalue(res, 0, 0))));
+
+	PQclear(res);
+}
+
 /*
  * Return PID of remote backend process.
  */
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 41243d0187..3c11f8e1dd 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -476,6 +476,10 @@ CreateInitDecodingContext(const char *plugin,
  * fast_forward
  *		bypass the generation of logical changes.
  *
+ * need_full_snapshot
+ * 		if true, must obtain a snapshot able to read all tables;
+ *  	if false, one that can read only catalogs is acceptable.
+ *
  * xl_routine
  *		XLogReaderRoutine used by underlying xlogreader
  *
@@ -494,6 +498,7 @@ LogicalDecodingContext *
 CreateDecodingContext(XLogRecPtr start_lsn,
 					  List *output_plugin_options,
 					  bool fast_forward,
+					  bool need_full_snapshot,
 					  XLogReaderRoutine *xl_routine,
 					  LogicalOutputPluginWriterPrepareWrite prepare_write,
 					  LogicalOutputPluginWriterWrite do_write,
@@ -502,6 +507,7 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 	LogicalDecodingContext *ctx;
 	ReplicationSlot *slot;
 	MemoryContext old_context;
+	TransactionId xmin_horizon = InvalidTransactionId;
 
 	/* shorter lines... */
 	slot = MyReplicationSlot;
@@ -571,8 +577,40 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 		start_lsn = slot->data.confirmed_flush;
 	}
 
+
+	/*
+	 * We need to determine a safe xmin horizon to start decoding from if we
+	 * want to create a snapshot too. Otherwise we would end up with a
+	 * snapshot that cannot be imported since xmin value from the snapshot may
+	 * be less than the oldest safe xmin. To avoid this call
+	 * GetOldestSafeDecodingTransactionId() to return a safe xmin value, which
+	 * can be used while exporting/importing the snapshot.
+	 *
+	 * So we have to acquire the ProcArrayLock to prevent computation of new
+	 * xmin horizons by other backends, get the safe decoding xid, and inform
+	 * the slot machinery about the new limit. Once that's done the
+	 * ProcArrayLock can be released as the slot machinery now is protecting
+	 * against vacuum.
+	 */
+	if (need_full_snapshot)
+	{
+		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+		xmin_horizon = GetOldestSafeDecodingTransactionId(!need_full_snapshot);
+
+		SpinLockAcquire(&slot->mutex);
+		slot->effective_catalog_xmin = xmin_horizon;
+		slot->data.catalog_xmin = xmin_horizon;
+		slot->effective_xmin = xmin_horizon;
+		SpinLockRelease(&slot->mutex);
+
+		ReplicationSlotsComputeRequiredXmin(true);
+
+		LWLockRelease(ProcArrayLock);
+	}
+
 	ctx = StartupDecodingContext(output_plugin_options,
-								 start_lsn, InvalidTransactionId, false,
+								 start_lsn, xmin_horizon, need_full_snapshot,
 								 fast_forward, xl_routine, prepare_write,
 								 do_write, update_progress);
 
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index 55a24c02c9..85c5cdb633 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -208,6 +208,7 @@ pg_logical_slot_get_changes_guts(FunctionCallInfo fcinfo, bool confirm, bool bin
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									options,
 									false,
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/repl_gram.y b/src/backend/replication/repl_gram.y
index 0c874e33cf..957a5cc653 100644
--- a/src/backend/replication/repl_gram.y
+++ b/src/backend/replication/repl_gram.y
@@ -65,6 +65,7 @@ Node *replication_parse_result;
 %token K_CREATE_REPLICATION_SLOT
 %token K_DROP_REPLICATION_SLOT
 %token K_TIMELINE_HISTORY
+%token K_CREATE_REPLICATION_SNAPSHOT
 %token K_WAIT
 %token K_TIMELINE
 %token K_PHYSICAL
@@ -80,7 +81,7 @@ Node *replication_parse_result;
 %type <node>	command
 %type <node>	base_backup start_replication start_logical_replication
 				create_replication_slot drop_replication_slot identify_system
-				read_replication_slot timeline_history show
+				read_replication_slot timeline_history show create_replication_snapshot
 %type <list>	generic_option_list
 %type <defelt>	generic_option
 %type <uintval>	opt_timeline
@@ -114,6 +115,7 @@ command:
 			| read_replication_slot
 			| timeline_history
 			| show
+			| create_replication_snapshot
 			;
 
 /*
@@ -307,6 +309,19 @@ timeline_history:
 				}
 			;
 
+/*
+ * CREATE_REPLICATION_SNAPSHOT %s options
+ */
+create_replication_snapshot:
+			K_CREATE_REPLICATION_SNAPSHOT var_name plugin_options
+				{
+					CreateReplicationSnapshotCmd *n = makeNode(CreateReplicationSnapshotCmd);
+					n->slotname = $2;
+					n->options = $3;
+					$$ = (Node *) n;
+				}
+			;
+
 opt_physical:
 			K_PHYSICAL
 			| /* EMPTY */
@@ -400,6 +415,7 @@ ident_or_keyword:
 			| K_CREATE_REPLICATION_SLOT	{ $$ = "create_replication_slot"; }
 			| K_DROP_REPLICATION_SLOT		{ $$ = "drop_replication_slot"; }
 			| K_TIMELINE_HISTORY			{ $$ = "timeline_history"; }
+			| K_CREATE_REPLICATION_SNAPSHOT	{ $$ = "create_replication_snapshot"; }
 			| K_WAIT						{ $$ = "wait"; }
 			| K_TIMELINE					{ $$ = "timeline"; }
 			| K_PHYSICAL					{ $$ = "physical"; }
diff --git a/src/backend/replication/repl_scanner.l b/src/backend/replication/repl_scanner.l
index 1cc7fb858c..0cc2cdd0be 100644
--- a/src/backend/replication/repl_scanner.l
+++ b/src/backend/replication/repl_scanner.l
@@ -126,6 +126,7 @@ START_REPLICATION	{ return K_START_REPLICATION; }
 CREATE_REPLICATION_SLOT		{ return K_CREATE_REPLICATION_SLOT; }
 DROP_REPLICATION_SLOT		{ return K_DROP_REPLICATION_SLOT; }
 TIMELINE_HISTORY	{ return K_TIMELINE_HISTORY; }
+CREATE_REPLICATION_SNAPSHOT	{ return K_CREATE_REPLICATION_SNAPSHOT; }
 PHYSICAL			{ return K_PHYSICAL; }
 RESERVE_WAL			{ return K_RESERVE_WAL; }
 LOGICAL				{ return K_LOGICAL; }
@@ -303,6 +304,7 @@ replication_scanner_is_replication_command(void)
 		case K_DROP_REPLICATION_SLOT:
 		case K_READ_REPLICATION_SLOT:
 		case K_TIMELINE_HISTORY:
+		case K_CREATE_REPLICATION_SNAPSHOT:
 		case K_SHOW:
 			/* Yes; push back the first token so we can parse later. */
 			repl_pushed_back_token = first_token;
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 6035cf4816..c5b2d5b61f 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -486,6 +486,7 @@ pg_logical_replication_slot_advance(XLogRecPtr moveto)
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									NIL,
 									true,	/* fast_forward */
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 2f3e93cc40..88f74fb76a 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -238,6 +238,7 @@ static void CreateReplicationSlot(CreateReplicationSlotCmd *cmd);
 static void DropReplicationSlot(DropReplicationSlotCmd *cmd);
 static void StartReplication(StartReplicationCmd *cmd);
 static void StartLogicalReplication(StartReplicationCmd *cmd);
+static void CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd);
 static void ProcessStandbyMessage(void);
 static void ProcessStandbyReplyMessage(void);
 static void ProcessStandbyHSFeedbackMessage(void);
@@ -1283,7 +1284,7 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	 * are reported early.
 	 */
 	logical_decoding_ctx =
-		CreateDecodingContext(cmd->startpoint, cmd->options, false,
+		CreateDecodingContext(cmd->startpoint, cmd->options, false, false,
 							  XL_ROUTINE(.page_read = logical_read_xlog_page,
 										 .segment_open = WalSndSegmentOpen,
 										 .segment_close = wal_segment_close),
@@ -1335,6 +1336,98 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	EndCommand(&qc, DestRemote, false);
 }
 
+/*
+ * Create a snapshot from an existing replication slot.
+ */
+static void
+CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd)
+{
+	Snapshot	snap;
+	LogicalDecodingContext *ctx;
+	char		xloc[MAXFNAMELEN];
+	DestReceiver *dest;
+	TupOutputState *tstate;
+	TupleDesc	tupdesc;
+	Datum		values[1];
+	bool		nulls[1] = {0};
+
+	Assert(!MyReplicationSlot);
+
+	CheckLogicalDecodingRequirements();
+
+	if (!IsTransactionBlock())
+		ereport(ERROR,
+				(errmsg("%s must be called inside a transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (XactIsoLevel != XACT_REPEATABLE_READ)
+		ereport(ERROR,
+				(errmsg("%s must be called in REPEATABLE READ isolation mode transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (!XactReadOnly)
+		ereport(ERROR,
+				(errmsg("%s must be called in a read only transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (FirstSnapshotSet)
+		ereport(ERROR,
+				(errmsg("%s must be called before any query",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (IsSubTransaction())
+		ereport(ERROR,
+				(errmsg("%s must not be called in a subtransaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	ReplicationSlotAcquire(cmd->slotname, false);
+
+	ctx = CreateDecodingContext(MyReplicationSlot->data.restart_lsn,
+								cmd->options,
+								false,
+								true,
+								XL_ROUTINE(.page_read = logical_read_xlog_page,
+										   .segment_open = WalSndSegmentOpen,
+										   .segment_close = wal_segment_close),
+								WalSndPrepareWrite, WalSndWriteData,
+								WalSndUpdateProgress);
+
+	/*
+	 * Signal that we don't need the timeout mechanism. We're just creating
+	 * the snapshot with the replication slot and don't yet accept feedback
+	 * messages or send keepalives. As we possibly need to wait for further
+	 * WAL the walsender would otherwise possibly be killed too soon.
+	 */
+	last_reply_timestamp = 0;
+
+	/* build initial snapshot, might take a while */
+	DecodingContextFindStartpoint(ctx);
+
+	snap = SnapBuildInitialSnapshot(ctx->snapshot_builder);
+	RestoreTransactionSnapshot(snap, MyProc);
+
+	/* Don't need the decoding context anymore */
+	FreeDecodingContext(ctx);
+
+	/* Create a tuple to send consistent WAL location */
+	snprintf(xloc, sizeof(xloc), "%X/%X",
+			 LSN_FORMAT_ARGS(MyReplicationSlot->data.confirmed_flush));
+
+	dest = CreateDestReceiver(DestRemoteSimple);
+	tupdesc = CreateTemplateTupleDesc(1);
+	TupleDescInitBuiltinEntry(tupdesc, (AttrNumber) 1, "consistent_point",
+							  TEXTOID, -1, 0);
+	tstate = begin_tup_output_tupdesc(dest, tupdesc, &TTSOpsVirtual);
+
+	/* consistent wal location */
+	values[0] = CStringGetTextDatum(xloc);
+
+	do_tup_output(tstate, values, nulls);
+	end_tup_output(tstate);
+
+	ReplicationSlotRelease();
+}
+
 /*
  * LogicalDecodingContext 'prepare_write' callback.
  *
@@ -1870,6 +1963,15 @@ exec_replication_command(const char *cmd_string)
 			}
 			break;
 
+		case T_CreateReplicationSnapshotCmd:
+			{
+				cmdtag = "CREATE_REPLICATION_SNAPSHOT";
+				set_ps_display(cmdtag);
+				CreateReplicationSnapshot((CreateReplicationSnapshotCmd *) cmd_node);
+				EndReplicationCommand(cmdtag);
+				break;
+			}
+
 		default:
 			elog(ERROR, "unrecognized replication command node tag: %u",
 				 cmd_node->type);
diff --git a/src/include/nodes/replnodes.h b/src/include/nodes/replnodes.h
index 4321ba8f86..154ab74f33 100644
--- a/src/include/nodes/replnodes.h
+++ b/src/include/nodes/replnodes.h
@@ -108,4 +108,15 @@ typedef struct TimeLineHistoryCmd
 	TimeLineID	timeline;
 } TimeLineHistoryCmd;
 
+/* ----------------------
+ *		CREATE_REPLICATION_SNAPSHOT command
+ * ----------------------
+ */
+typedef struct CreateReplicationSnapshotCmd
+{
+	NodeTag		type;
+	char	   *slotname;
+	List	   *options;
+} CreateReplicationSnapshotCmd;
+
 #endif							/* REPLNODES_H */
diff --git a/src/include/replication/logical.h b/src/include/replication/logical.h
index 5f49554ea0..6535786a0e 100644
--- a/src/include/replication/logical.h
+++ b/src/include/replication/logical.h
@@ -125,6 +125,7 @@ extern LogicalDecodingContext *CreateInitDecodingContext(const char *plugin,
 extern LogicalDecodingContext *CreateDecodingContext(XLogRecPtr start_lsn,
 													 List *output_plugin_options,
 													 bool fast_forward,
+													 bool need_full_snapshot,
 													 XLogReaderRoutine *xl_routine,
 													 LogicalOutputPluginWriterPrepareWrite prepare_write,
 													 LogicalOutputPluginWriterWrite do_write,
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 281626fa6f..f01e5f23a7 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -386,6 +386,16 @@ typedef WalRcvExecResult *(*walrcv_exec_fn) (WalReceiverConn *conn,
  */
 typedef void (*walrcv_disconnect_fn) (WalReceiverConn *conn);
 
+/*
+ * walrcv_slot_snapshot_fn
+ *
+ * Create a snapshot by an existing replication slot
+ */
+typedef void (*walrcv_slot_snapshot_fn) (WalReceiverConn *conn,
+										 char *slotname,
+										 const WalRcvStreamOptions *options,
+										 XLogRecPtr *lsn);
+
 typedef struct WalReceiverFunctionsType
 {
 	walrcv_connect_fn walrcv_connect;
@@ -403,6 +413,7 @@ typedef struct WalReceiverFunctionsType
 	walrcv_get_backend_pid_fn walrcv_get_backend_pid;
 	walrcv_exec_fn walrcv_exec;
 	walrcv_disconnect_fn walrcv_disconnect;
+	walrcv_slot_snapshot_fn walrcv_slot_snapshot;
 } WalReceiverFunctionsType;
 
 extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
@@ -437,6 +448,8 @@ extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
 	WalReceiverFunctions->walrcv_exec(conn, exec, nRetTypes, retTypes)
 #define walrcv_disconnect(conn) \
 	WalReceiverFunctions->walrcv_disconnect(conn)
+#define walrcv_slot_snapshot(conn, slotname, options, lsn) \
+	WalReceiverFunctions->walrcv_slot_snapshot(conn, slotname, options, lsn)
 
 static inline void
 walrcv_clear_result(WalRcvExecResult *walres)
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index e941fb6c82..cc023b10fa 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2348,6 +2348,7 @@ ReplicationSlotInvalidationCause
 ReplicationSlotOnDisk
 ReplicationSlotPersistency
 ReplicationSlotPersistentData
+CreateReplicationSnapshotCmd
 ReplicationState
 ReplicationStateCtl
 ReplicationStateOnDisk
@@ -3855,6 +3856,7 @@ walrcv_receive_fn
 walrcv_send_fn
 walrcv_server_version_fn
 walrcv_startstreaming_fn
+walrcv_slot_snapshot_fn
 wchar2mb_with_len_converter
 wchar_t
 win32_deadchild_waitinfo
-- 
2.25.1

v19-0005-Reuse-Replication-Slot-and-Origin-in-Tablesync.patchapplication/octet-stream; name=v19-0005-Reuse-Replication-Slot-and-Origin-in-Tablesync.patchDownload
From c2f1b67f0188c4eca0717563963a5f269d71fe11 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH v19 5/5] Reuse Replication Slot and Origin in Tablesync

This commit allows reusing replication slots and origins during tablesync.

Earlier, a tablesync worker was creating a new replication slot and origin
each time it syncs a new table. With this patch, replication
slots/origins can be reusable for tablesync.

This reduces the overhead of creating/dropping replication slots and origins
and improves tablesync speed significantly especially for empty or small tables.

If the state of the current table is INIT or DATASYNC, tablesync worker needs a
repliation slot/origin. If the worker has not created slot and origin in
its previous runs, it will create those first. Otherwise the worker reuses
slot and origin created by the same worker in previous iterations earlier.
Tables in FINISHEDCOPY are expected to have a replication slot and origin.
Slot and origin names for such tables are persisted in
pg_subscription_rel catalog. Tablesync worker can fetch them and proceed
with existing slot and origin of FINISHEDCOPY tables and does not need to
create new ones.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/catalogs.sgml                    |  31 ++
 src/backend/catalog/pg_subscription.c         | 247 ++++++++++-
 src/backend/commands/subscriptioncmds.c       | 229 ++++++----
 .../replication/logical/applyparallelworker.c |   3 +-
 src/backend/replication/logical/launcher.c    |   8 +-
 src/backend/replication/logical/tablesync.c   | 414 ++++++++++++++----
 src/backend/replication/logical/worker.c      |  17 +-
 src/include/catalog/pg_subscription.h         |   6 +
 src/include/catalog/pg_subscription_rel.h     |  14 +-
 src/include/replication/slot.h                |   3 +-
 src/include/replication/worker_internal.h     |  23 +-
 11 files changed, 804 insertions(+), 191 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 307ad88b50..200a6b2ae9 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8031,6 +8031,19 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        origin.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>sublastusedid</structfield> <type>int8</type>
+      </para>
+      <para>
+      The last used ID for tablesync workers. It acts as an unique identifier
+      for replication slots which are created by tablesync workers.
+      The last used ID needs to be persisted to make logical replication safely
+      proceed after any interruption. If sublastusedid is 0, then no table has
+      been synced yet.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
@@ -8115,6 +8128,24 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        otherwise null
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srrelslotname</structfield> <type>name</type>
+      </para>
+      <para>
+       Replication slot name that is used for synchronization of relation
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srreloriginname</structfield> <type>name</type>
+      </para>
+      <para>
+       Origin name that is used for tracking synchronization of relation
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index d07f88ce28..152fdaa310 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -108,6 +108,14 @@ GetSubscription(Oid subid, bool missing_ok)
 								   Anum_pg_subscription_suborigin);
 	sub->origin = TextDatumGetCString(datum);
 
+	/* Get last used id */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
+							tup,
+							Anum_pg_subscription_sublastusedid,
+							&isnull);
+	Assert(!isnull);
+	sub->lastusedid = DatumGetInt64(datum);
+
 	ReleaseSysCache(tup);
 
 	return sub;
@@ -199,6 +207,44 @@ DisableSubscription(Oid subid)
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update the last used replication slot ID for the given subscription.
+ */
+void
+UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid)
+{
+	Relation	rel;
+	bool		nulls[Natts_pg_subscription];
+	bool		replaces[Natts_pg_subscription];
+	Datum		values[Natts_pg_subscription];
+	HeapTuple	tup;
+
+	/* Look up the subscription in the catalog */
+	rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+	tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	/* Form a new tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	replaces[Anum_pg_subscription_sublastusedid - 1] = true;
+	values[Anum_pg_subscription_sublastusedid- 1] = Int64GetDatum(lastusedid);
+
+	/* Update the catalog */
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+	heap_freetuple(tup);
+
+	table_close(rel, NoLock);
+}
+
 /*
  * Convert text array to list of strings.
  *
@@ -228,7 +274,7 @@ textarray_to_stringlist(ArrayType *textarray)
  */
 void
 AddSubscriptionRelState(Oid subid, Oid relid, char state,
-						XLogRecPtr sublsn)
+						XLogRecPtr sublsn, char *relslotname, char *reloriginname)
 {
 	Relation	rel;
 	HeapTuple	tup;
@@ -257,6 +303,16 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
 	else
 		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -269,6 +325,60 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Internal function to modify columns for relation state update
+ */
+static void
+UpdateSubscriptionRelState_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char state,
+									XLogRecPtr sublsn)
+{
+	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
+	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+
+	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (sublsn != InvalidXLogRecPtr)
+		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
+	else
+		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication slot update
+ */
+static void
+UpdateSubscriptionRelReplicationSlot_internal(Datum *values,
+											bool *nulls,
+											bool *replaces,
+											char *relslotname)
+{
+	replaces[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication origin update
+ */
+static void
+UpdateSubscriptionRelOrigin_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char *reloriginname)
+{
+	replaces[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+}
+
 /*
  * Update the state of a subscription table.
  */
@@ -299,14 +409,56 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	memset(nulls, false, sizeof(nulls));
 	memset(replaces, false, sizeof(replaces));
 
-	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
-	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
 
-	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
-	if (sublsn != InvalidXLogRecPtr)
-		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
-	else
-		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update replication slot name, origin name and state of
+ * a subscription table in one transaction.
+ */
+void
+UpdateSubscriptionRel(Oid subid,
+					  Oid relid,
+					  char state,
+					  XLogRecPtr sublsn,
+					  char *relslotname,
+					  char *reloriginname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
+	UpdateSubscriptionRelOrigin_internal(values, nulls, replaces, reloriginname);
 
 	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
 							replaces);
@@ -318,6 +470,85 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Get origin name of subscription table.
+ *
+ * reloriginname's value has the replication origin name if the origin exists.
+ */
+void
+GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*originname;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srreloriginname, isnull);
+	if (!*isnull)
+	{
+		originname = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(reloriginname, originname, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
+/*
+ * Get replication slot name of subscription table.
+ *
+ * slotname's value has the replication slot name if the subscription has any.
+ */
+void
+GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*relrepslot;
+	bool		isnull;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srrelslotname, &isnull);
+	if (!isnull)
+	{
+		relrepslot = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(slotname, relrepslot, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
 /*
  * Get state of subscription table.
  *
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index d4e798baeb..4f5b3c572c 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -710,6 +710,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 		publicationListToArray(publications);
 	values[Anum_pg_subscription_suborigin - 1] =
 		CStringGetTextDatum(opts.origin);
+	values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(0);
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -773,7 +774,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 										 rv->schemaname, rv->relname);
 
 				AddSubscriptionRelState(subid, relid, table_state,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 			}
 
 			/*
@@ -864,6 +865,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
 	bool		must_use_password;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -943,7 +946,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				AddSubscriptionRelState(sub->oid, relid,
 										copy_data ? SUBREL_STATE_INIT : SUBREL_STATE_READY,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 				ereport(DEBUG1,
 						(errmsg_internal("table \"%s.%s\" added to subscription \"%s\"",
 										 rv->schemaname, rv->relname, sub->name)));
@@ -967,6 +970,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				char		state;
 				XLogRecPtr	statelsn;
+				char		slotname[NAMEDATALEN] = {0};
 
 				/*
 				 * Lock pg_subscription_rel with AccessExclusiveLock to
@@ -993,13 +997,36 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/*
+				 * Find the logical replication sync worker. If exists, store
+				 * the slot number for dropping associated replication slots
+				 * later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot_name);
+				}
+				else
+				{
+					/*
+					 * Sync of this relation might be failed in an earlier
+					 * attempt, but the replication slot might still exist.
+					 */
+					GetSubscriptionRelReplicationSlot(sub->oid, relid, slotname);
+					if (strlen(slotname) > 0)
+						sub_remove_slots = lappend(sub_remove_slots, slotname);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
 				 * tablesync origin.
 				 */
-				if (state != SUBREL_STATE_READY)
+				if (state != SUBREL_STATE_READY &&
+					state != SUBREL_STATE_SYNCDONE)
 				{
 					char		originname[NAMEDATALEN];
 
@@ -1027,31 +1054,24 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
-		 * the database operations we won't be able to rollback dropped slots.
+		 * Drop the replication slots associated with tablesync workers for
+		 * removed tables. This has to be at the end because otherwise if
+		 * there is an error while doing the database operations we won't be
+		 * able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			memcpy(syncslotname, lfirst(lc), sizeof(NAMEDATALEN));
+
+			/*
+			 * There is no certainty, maybe the slot does not exist yet. Also,
+			 * if we fail after removing some of the slots, next time, it will
+			 * again try to drop already dropped slots and fail. For these
+			 * reasons, we allow missing_ok = true for the drop.
+			 */
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1474,6 +1494,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	char	   *subname;
 	char	   *conninfo;
 	char	   *slotname;
+	int64		lastusedid;
 	List	   *subworkers;
 	ListCell   *lc;
 	char		originname[NAMEDATALEN];
@@ -1546,6 +1567,14 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	else
 		slotname = NULL;
 
+	/* Get the last used identifier by the subscription */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID, tup,
+							Anum_pg_subscription_sublastusedid, &isnull);
+	if (!isnull)
+		lastusedid = DatumGetInt64(datum);
+	else
+		lastusedid = 0;
+
 	/*
 	 * Since dropping a replication slot is not transactional, the replication
 	 * slot stays dropped even if the transaction rolls back.  So we cannot
@@ -1595,6 +1624,8 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	}
 	list_free(subworkers);
 
+	rstates = GetSubscriptionRelations(subid, true);
+
 	/*
 	 * Remove the no-longer-useful entry in the launcher's table of apply
 	 * worker start times.
@@ -1606,36 +1637,26 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	ApplyLauncherForgetWorkerStartTime(subid);
 
 	/*
-	 * Cleanup of tablesync replication origins.
-	 *
-	 * Any READY-state relations would already have dealt with clean-ups.
+	 * Cleanup of tablesync replication origins associated with the
+	 * subscription, if exists. Try to drop origins by creating all origin
+	 * names created for this subscription.
 	 *
 	 * Note that the state can't change because we have already stopped both
 	 * the apply and tablesync workers and they can't restart because of
 	 * exclusive lock on the subscription.
+	 *
+	 * XXX: This can be handled better instead of looping through all possible
 	 */
-	rstates = GetSubscriptionRelations(subid, true);
-	foreach(lc, rstates)
+	for (int64 i = 1; i <= lastusedid; i++)
 	{
-		SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-		Oid			relid = rstate->relid;
-
-		/* Only cleanup resources of tablesync workers */
-		if (!OidIsValid(relid))
-			continue;
+		char		originname_to_drop[NAMEDATALEN] = {0};
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * It is possible that the origin is not yet created for tablesync
-		 * worker so passing missing_ok = true. This can happen for the states
-		 * before SUBREL_STATE_FINISHEDCOPY.
-		 */
-		ReplicationOriginNameForLogicalRep(subid, relid, originname,
-										   sizeof(originname));
-		replorigin_drop_by_name(originname, true, false);
+		snprintf(originname_to_drop, sizeof(originname_to_drop), "pg_%u_%lld", subid, (long long) i);
+		/* missing_ok = true, since the origin might be already dropped. */
+		replorigin_drop_by_name(originname_to_drop, true, false);
 	}
 
+
 	/* Clean up dependencies */
 	deleteSharedDependencyRecordsFor(SubscriptionRelationId, subid, 0);
 
@@ -1694,39 +1715,17 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
+		List	   *slots = NULL;
 
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
 
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char	   *syncslotname = (char *) lfirst(lc);
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1743,6 +1742,71 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * Get the replication slot names associated with the subscription.
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok)
+{
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+					 " FROM pg_replication_slots"
+					 " WHERE slot_name LIKE 'pg_%i_sync_%%';",
+					 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+					errmsg("could not receive list of slots associated with the subscription %u, error: %s",
+					subid, res->err));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();
+		return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -2155,6 +2219,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -2165,18 +2230,20 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
-			char		syncslotname[NAMEDATALEN] = {0};
-
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
-				 syncslotname);
+				 worker->slot_name);
 		}
 	}
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 1d4e83c4c1..d2c70dffbc 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -440,7 +440,8 @@ pa_launch_parallel_worker(void)
 										MySubscription->name,
 										MyLogicalRepWorker->userid,
 										InvalidOid,
-										dsm_segment_handle(winfo->dsm_seg));
+										dsm_segment_handle(winfo->dsm_seg),
+										InvalidRepSlotId);
 
 	if (launched)
 	{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 945619b603..9f378c311e 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -304,7 +304,7 @@ logicalrep_workers_find(Oid subid, bool only_running)
  */
 bool
 logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
-						 Oid relid, dsm_handle subworker_dsm)
+						 Oid relid, dsm_handle subworker_dsm, int64 slotid)
 {
 	BackgroundWorker bgw;
 	BackgroundWorkerHandle *bgw_handle;
@@ -430,6 +430,9 @@ retry:
 	worker->launch_time = now;
 	worker->in_use = true;
 	worker->generation++;
+	worker->created_slot = false;
+	worker->rep_slot_id = slotid;
+	worker->slot_name = (char *) palloc(NAMEDATALEN);
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -1184,7 +1187,8 @@ ApplyLauncherMain(Datum main_arg)
 				ApplyLauncherSetWorkerStartTime(sub->oid, now);
 				logicalrep_worker_launch(sub->dbid, sub->oid, sub->name,
 										 sub->owner, InvalidOid,
-										 DSM_HANDLE_INVALID);
+										 DSM_HANDLE_INVALID,
+										 InvalidRepSlotId);
 			}
 			else
 			{
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index d380abfa47..6bb9084720 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -317,40 +317,29 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 								   MyLogicalRepWorker->relid,
 								   MyLogicalRepWorker->relstate,
 								   MyLogicalRepWorker->relstate_lsn);
+		CommitTransactionCommand();
 
 		/*
-		 * End streaming so that LogRepWorkerWalRcvConn can be used to drop
-		 * the slot.
-		 */
-		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
-
-		/*
-		 * Cleanup the tablesync slot.
+		 * Cleanup the tablesync slot. If the slot name used by this worker is
+		 * different from the default slot name for the worker, this means the
+		 * current table had started to being synchronized by another worker
+		 * and replication slot. And this worker is reusing a replication slot
+		 * from a previous attempt. We do not need that replication slot
+		 * anymore.
 		 *
 		 * This has to be done after updating the state because otherwise if
 		 * there is an error while doing the database operations we won't be
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
+										MyLogicalRepWorker->rep_slot_id,
 										syncslotname,
 										sizeof(syncslotname));
 
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
-		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
-
-		CommitTransactionCommand();
-		pgstat_report_stat(false);
-
-		/*
-		 * Start a new transaction to clean up the tablesync origin tracking.
-		 * This transaction will be ended within the finish_sync_worker().
-		 * Now, even, if we fail to remove this here, the apply worker will
-		 * ensure to clean it up afterward.
+		 * We are safe to drop the replication tracking origin after this
+		 * point. Now, even, if we fail to remove this here, the apply worker
+		 * will ensure to clean it up afterward.
 		 *
 		 * We need to do this after the table state is set to SYNCDONE.
 		 * Otherwise, if an error occurs while performing the database
@@ -359,32 +348,73 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 * have been cleared before restart. So, the restarted worker will use
 		 * invalid replication progress state resulting in replay of
 		 * transactions that have already been applied.
+		 *
+		 * Firstly reset the origin session to remove the ownership of the
+		 * slot. This is needed to allow the origin to be dropped or reused
+		 * later.
 		 */
+		replorigin_session_reset();
+		replorigin_session_origin = InvalidRepOriginId;
+		replorigin_session_origin_lsn = InvalidXLogRecPtr;
+		replorigin_session_origin_timestamp = 0;
+
 		StartTransactionCommand();
+		if (MyLogicalRepWorker->slot_name && strcmp(syncslotname, MyLogicalRepWorker->slot_name) != 0)
+		{
+			/*
+			 * End streaming so that LogRepWorkerWalRcvConn can be used to
+			 * drop the slot.
+			 */
+			walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
 
-		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
+			ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+											   MyLogicalRepWorker->relid,
+											   originname,
+											   sizeof(originname));
+
+			/*
+			 * Drop replication origin
+			 *
+			 * There is a chance that the user is concurrently performing refresh
+			 * for the subscription where we remove the table state and its origin
+			 * or the apply worker would have removed this origin. So passing
+			 * missing_ok = true.
+			 */
+			replorigin_drop_by_name(originname, true, false);
+		}
 
 		/*
-		 * Resetting the origin session removes the ownership of the slot.
-		 * This is needed to allow the origin to be dropped.
+		 * We are safe to remove persisted replication slot and origin data,
+		 * since it's already in SYNCDONE state. They will not be needed
+		 * anymore.
 		 */
-		replorigin_session_reset();
-		replorigin_session_origin = InvalidRepOriginId;
-		replorigin_session_origin_lsn = InvalidXLogRecPtr;
-		replorigin_session_origin_timestamp = 0;
+		UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+							  MyLogicalRepWorker->relid,
+							  MyLogicalRepWorker->relstate,
+							  MyLogicalRepWorker->relstate_lsn,
+							  NULL,
+							  NULL);
+		ereport(DEBUG2,
+				(errmsg("process_syncing_tables_for_sync: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+						"NULL",
+						"NULL",
+						MyLogicalRepWorker->relstate,
+						MyLogicalRepWorker->relid,
+						MyLogicalRepWorker->subid)));
+		CommitTransactionCommand();
+		pgstat_report_stat(false);
 
 		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * There is a chance that the user is concurrently performing refresh
-		 * for the subscription where we remove the table state and its origin
-		 * or the apply worker would have removed this origin. So passing
-		 * missing_ok = true.
+		 * This should return the default origin name for the worker. Even if
+		 * the worker used a different origin for this table, it should be
+		 * dropped and removed from the catalog so far.
 		 */
-		replorigin_drop_by_name(originname, true, false);
+		StartTransactionCommand();
+		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+										   MyLogicalRepWorker->relid,
+										   originname,
+										   sizeof(originname));
 
 		/* Sync worker has completed synchronization of the current table. */
 		MyLogicalRepWorker->is_sync_completed = true;
@@ -481,6 +511,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 			if (current_lsn >= rstate->lsn)
 			{
 				char		originname[NAMEDATALEN];
+				bool		is_origin_null = true;
 
 				rstate->state = SUBREL_STATE_READY;
 				rstate->lsn = current_lsn;
@@ -501,18 +532,31 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				 * error while dropping we won't restart it to drop the
 				 * origin. So passing missing_ok = true.
 				 */
-				ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-												   rstate->relid,
-												   originname,
-												   sizeof(originname));
-				replorigin_drop_by_name(originname, true, false);
+				GetSubscriptionRelOrigin(MyLogicalRepWorker->subid,
+										 rstate->relid, originname,
+										 &is_origin_null);
+
+				if (!is_origin_null)
+				{
+					replorigin_drop_by_name(originname, true, false);
+				}
 
 				/*
 				 * Update the state to READY only after the origin cleanup.
 				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-										   rstate->relid, rstate->state,
-										   rstate->lsn);
+				UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+									  rstate->relid,
+									  rstate->state,
+									  rstate->lsn,
+									  NULL,
+									  NULL);
+				ereport(DEBUG2,
+					(errmsg("process_syncing_tables_for_apply: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+							"NULL", "NULL", rstate->state,
+							rstate->relid, MyLogicalRepWorker->subid)));
+
+				CommitTransactionCommand();
+				started_tx = false;
 			}
 		}
 		else
@@ -601,12 +645,25 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						TimestampDifferenceExceeds(hentry->last_start_time, now,
 												   wal_retrieve_retry_interval))
 					{
+						if (IsTransactionState())
+							CommitTransactionCommand();
+						StartTransactionCommand();
+						started_tx = true;
+
+						MySubscription->lastusedid++;
+						UpdateSubscriptionLastSlotId(MyLogicalRepWorker->subid,
+													 MySubscription->lastusedid);
+						ereport(DEBUG2,
+								(errmsg("process_syncing_tables_for_apply: incremented lastusedid to %lld for subscription %u",
+										(long long) MySubscription->lastusedid, MySubscription->oid)));
+
 						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
 												 MySubscription->oid,
 												 MySubscription->name,
 												 MyLogicalRepWorker->userid,
 												 rstate->relid,
-												 DSM_HANDLE_INVALID);
+												 DSM_HANDLE_INVALID,
+												 MySubscription->lastusedid);
 						hentry->last_start_time = now;
 					}
 				}
@@ -1229,8 +1286,8 @@ copy_table(Relation rel)
  * The name must not exceed NAMEDATALEN - 1 because of remote node constraints
  * on slot name length. We append system_identifier to avoid slot_name
  * collision with subscriptions in other clusters. With the current scheme
- * pg_%u_sync_%u_UINT64_FORMAT (3 + 10 + 6 + 10 + 20 + '\0'), the maximum
- * length of slot_name will be 50.
+ * pg_%u_sync_%lu_UINT64_FORMAT (3 + 10 + 6 + 20 + 20 + '\0'), the maximum
+ * length of slot_name will be 45.
  *
  * The returned slot name is stored in the supplied buffer (syncslotname) with
  * the given size.
@@ -1241,11 +1298,11 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
+ReplicationSlotNameForTablesync(Oid suboid, int64 slotid,
 								char *syncslotname, Size szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%lld_" UINT64_FORMAT, suboid,
+			(long long) slotid, GetSystemIdentifier());
 }
 
 /*
@@ -1289,6 +1346,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	UserContext ucxt;
 	bool		must_use_password;
 	bool		run_as_owner;
+	char	   *prev_slotname;
 
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
@@ -1323,7 +1381,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
+									MyLogicalRepWorker->rep_slot_id,
 									slotname,
 									NAMEDATALEN);
 
@@ -1355,12 +1413,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC ||
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY);
 
+	/*
+	 * See if tablesync of the current relation has been started with another
+	 * replication slot.
+	 *
+	 * Read previous slot name from the catalog, if exists.
+	 */
+	prev_slotname = (char *) palloc(NAMEDATALEN);
+	StartTransactionCommand();
+	GetSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+									  MyLogicalRepWorker->relid,
+									  prev_slotname);
+
 	/* Assign the origin tracking record name. */
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   sizeof(originname));
 
+	CommitTransactionCommand();
+
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC)
 	{
 		/*
@@ -1374,10 +1446,53 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		 * breakdown then it wouldn't have succeeded so trying it next time
 		 * seems like a better bet.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, slotname, true);
+		if (strlen(prev_slotname) > 0)
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);
+
+			StartTransactionCommand();
+			/* Replication origin might still exist. Try to drop */
+			replorigin_drop_by_name(originname, true, false);
+
+			/*
+			 * Remove replication slot and origin name from the relation's
+			 * catalog record
+			 */
+			UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+								  MyLogicalRepWorker->relid,
+								  MyLogicalRepWorker->relstate,
+								  MyLogicalRepWorker->relstate_lsn,
+								  NULL,
+								  NULL);
+			CommitTransactionCommand();
+			ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+						"NULL", "NULL", MyLogicalRepWorker->relstate,
+						MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+		}
 	}
 	else if (MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY)
 	{
+		/*
+		 * At this point, the table that is currently being synchronized
+		 * should have its replication slot name filled in the catalog. The
+		 * tablesync process was started with another sync worker and
+		 * replication slot. We need to continue using the same replication
+		 * slot in this worker too.
+		 */
+		if (strlen(prev_slotname) == 0)
+		{
+			elog(ERROR, "Replication slot could not be found for subscription %u, relation %u",
+				 MyLogicalRepWorker->subid,
+				 MyLogicalRepWorker->relid);
+		}
+
+		/*
+		 * Proceed with the correct replication slot. Use previously created
+		 * replication slot to sync this table.
+		 */
+		memcpy(slotname, prev_slotname, NAMEDATALEN);
+
 		/*
 		 * The COPY phase was previously done, but tablesync then crashed
 		 * before it was able to finish normally.
@@ -1397,7 +1512,9 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 		goto copy_table_done;
 	}
+	pfree(prev_slotname);
 
+	/* Preparing for table copy operation */
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 	MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
 	MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
@@ -1405,11 +1522,31 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   MyLogicalRepWorker->relstate,
-							   MyLogicalRepWorker->relstate_lsn);
+
+	/*
+	 * Refresh the originname in case of having non-existing origin
+	 * from previous failed sync attempts.
+	 * If that's the case, it should be removed from the catalog so far.
+	 * Then, we can continue by reusing the origin created by the current
+	 * worker instead of .
+	 */
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									MyLogicalRepWorker->relid,
+									originname,
+									sizeof(originname));
+
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  MyLogicalRepWorker->relstate,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					slotname, originname, MyLogicalRepWorker->relstate,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+
 	pgstat_report_stat(true);
 
 	StartTransactionCommand();
@@ -1437,48 +1574,96 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						res->err)));
 	walrcv_clear_result(res);
 
+	originid = replorigin_by_name(originname, true);
+
 	/*
 	 * Create a new permanent logical decoding slot. This slot will be used
 	 * for the catchup phase after COPY is done, so tell it to use the
 	 * snapshot to make the final data consistent.
+	 *
+	 * Replication slot will only be created if either this is the first run
+	 * of the worker or we're not using a previous replication slot.
 	 */
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-
-	/*
-	 * Setup replication origin tracking. The purpose of doing this before the
-	 * copy is to avoid doing the copy again due to any error in setting up
-	 * origin tracking.
-	 */
-	originid = replorigin_by_name(originname, true);
-	if (!OidIsValid(originid))
+	if (!MyLogicalRepWorker->created_slot)
 	{
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						   slotname, false /* permanent */ , false /* two_phase */ ,
+						   CRS_USE_SNAPSHOT, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: created replication slot %s for subscription %u",
+						slotname, MyLogicalRepWorker->subid)));
+
 		/*
-		 * Origin tracking does not exist, so create it now.
-		 *
-		 * Then advance to the LSN got from walrcv_create_slot. This is WAL
-		 * logged for the purpose of recovery. Locks are to prevent the
-		 * replication origin from vanishing while advancing.
+		 * Remember that we created the slot so that we will not try to create
+		 * it again.
 		 */
-		originid = replorigin_create(originname);
-
-		LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
-		replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
-						   true /* go backward */ , true /* WAL log */ );
-		UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->created_slot = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
+		/*
+		 * Setup replication origin tracking. The purpose of doing this before
+		 * the copy is to avoid doing the copy again due to any error in
+		 * setting up origin tracking.
+		 */
+		if (!OidIsValid(originid))
+		{
+			/*
+			 * Origin tracking does not exist, so create it now.
+			 */
+			originid = replorigin_create(originname);
+		}
+		else
+		{
+			/*
+			 * At this point, there shouldn't be any existing replication
+			 * origin with the same name.
+			 */
+			ereport(ERROR,
+					(errcode(ERRCODE_DUPLICATE_OBJECT),
+					 errmsg("replication origin \"%s\" already exists",
+							originname)));
+		}
 	}
 	else
 	{
-		ereport(ERROR,
-				(errcode(ERRCODE_DUPLICATE_OBJECT),
-				 errmsg("replication origin \"%s\" already exists",
-						originname)));
+		/*
+		 * Do not create a new replication slot, reuse the existing one
+		 * instead. Use a new snapshot for the replication slot to ensure that
+		 * tablesync and apply proceses are consistent with each other.
+		 */
+		WalRcvStreamOptions options;
+		int			server_version;
+
+		server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+		options.proto.logical.proto_version =
+			server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+			server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+			LOGICALREP_PROTO_VERSION_NUM;
+		options.proto.logical.publication_names = MySubscription->publications;
+
+		walrcv_slot_snapshot(LogRepWorkerWalRcvConn, slotname, &options, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: reusing replication slot %s for relation %u in subscription %u",
+						slotname, MyLogicalRepWorker->relid,
+						MyLogicalRepWorker->subid)));
 	}
 
+	/*
+	 * Advance to the LSN got from walrcv_create_slot or walrcv_slot_snapshot.
+	 * This is WAL logged for the purpose of recovery. Locks are to prevent
+	 * the replication origin from vanishing while advancing.
+	 *
+	 * Then setup replication origin tracking.
+	 */
+	LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+	replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
+					   true /* go backward */ , true /* WAL log */ );
+	UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+
 	/*
 	 * Make sure that the copy command runs as the table owner, unless the
 	 * user has opted out of that behaviour.
@@ -1537,12 +1722,18 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	 * Update the persisted state to indicate the COPY phase is done; make it
 	 * visible to others.
 	 */
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   SUBREL_STATE_FINISHEDCOPY,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  SUBREL_STATE_FINISHEDCOPY,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					originname, slotname, SUBREL_STATE_FINISHEDCOPY,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
 
 copy_table_done:
 
@@ -1668,6 +1859,9 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 
 	/* allocate slot name in long-lived context */
 	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+
+	/* Keep the replication slot name used for this sync. */
+	MyLogicalRepWorker->slot_name = *myslotname;
 	pfree(syncslotname);
 }
 
@@ -1686,13 +1880,25 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 {
 	MyLogicalRepWorker->is_sync_completed = false;
 
+	/*
+	 * If it's already connected to the publisher, end streaming before using
+	 * the same connection for another iteration
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		TimeLineID tli;
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	}
+
 	/* Start table synchronization. */
 	start_table_sync(origin_startpos, &slotname);
 
+	StartTransactionCommand();
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   originname_size);
+	CommitTransactionCommand();
 
 	set_apply_error_context_origin(originname);
 
@@ -1824,6 +2030,32 @@ TablesyncWorkerMain(Datum main_arg)
 				}
 				LWLockRelease(LogicalRepWorkerLock);
 			}
+
+			if (done)
+			{
+				TimeLineID	tli;
+
+				/*
+				 * It is important to give an error if we are unable to drop the
+				 * slot, otherwise, it won't be dropped till the corresponding
+				 * subscription is dropped. So passing missing_ok = false.
+				 */
+				if (MyLogicalRepWorker->created_slot)
+				{
+					walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+					ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
+				}
+
+				/*
+				 * Drop replication origin before exiting.
+				 *
+				 * There is a chance that the user is concurrently performing refresh
+				 * for the subscription where we remove the table state and its origin
+				 * or the apply worker would have removed this origin. So passing
+				 * missing_ok = true.
+				 */
+				replorigin_drop_by_name(originname, true, false);
+			}
 		}
 	}
 
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 10a356cc28..60c8a087fa 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -461,8 +461,16 @@ ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 {
 	if (OidIsValid(relid))
 	{
-		/* Replication origin name for tablesync workers. */
-		snprintf(originname, szoriginname, "pg_%u_%u", suboid, relid);
+		bool		is_null = true;
+
+		/*
+		 * Replication origin name for tablesync workers. First, look into the
+		 * catalog. If originname does not exist, then use the default name.
+		 */
+		GetSubscriptionRelOrigin(suboid, relid,
+								 originname, &is_null);
+		if (is_null)
+			snprintf(originname, szoriginname, "pg_%u_%lld", suboid, (long long) MyLogicalRepWorker->rep_slot_id);
 	}
 	else
 	{
@@ -4515,11 +4523,10 @@ run_apply_worker(WalRcvStreamOptions *options,
 				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
 					errmsg("subscription has no replication slot set")));
 
-	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-									   originname, originname_size);
-
 	/* Setup replication origin tracking. */
 	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
 	originid = replorigin_by_name(originname, true);
 	if (!OidIsValid(originid))
 		originid = replorigin_create(originname);
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 1d40eebc78..7e13f59847 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -108,6 +108,9 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
 
 	/* Only publish data originating from the specified origin */
 	text		suborigin BKI_DEFAULT(LOGICALREP_ORIGIN_ANY);
+
+	/* The last used ID to create a replication slot for tablesync */
+	int64		sublastusedid BKI_DEFAULT(0);
 #endif
 } FormData_pg_subscription;
 
@@ -144,6 +147,8 @@ typedef struct Subscription
 	List	   *publications;	/* List of publication names to subscribe to */
 	char	   *origin;			/* Only publish data originating from the
 								 * specified origin */
+	int64		lastusedid;		/* Last used unique ID to create replication
+								 * slots in tablesync */
 } Subscription;
 
 /* Disallow streaming in-progress transactions. */
@@ -164,6 +169,7 @@ typedef struct Subscription
 extern Subscription *GetSubscription(Oid subid, bool missing_ok);
 extern void FreeSubscription(Subscription *sub);
 extern void DisableSubscription(Oid subid);
+extern void UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid);
 
 extern int	CountDBSubscriptions(Oid dbid);
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 60a2bcca23..185164d75e 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -44,6 +44,12 @@ CATALOG(pg_subscription_rel,6102,SubscriptionRelRelationId)
 											 * used for synchronization
 											 * coordination, or NULL if not
 											 * valid */
+	NameData	srrelslotname BKI_FORCE_NULL;	/* name of the replication
+												 * slot for relation in
+												 * subscription */
+	NameData	srreloriginname BKI_FORCE_NULL; /* origin name for relation in
+												 * subscription */
+
 #endif
 } FormData_pg_subscription_rel;
 
@@ -81,10 +87,16 @@ typedef struct SubscriptionRelState
 } SubscriptionRelState;
 
 extern void AddSubscriptionRelState(Oid subid, Oid relid, char state,
-									XLogRecPtr sublsn);
+									XLogRecPtr sublsn, char *relslotname, char *reloriginname);
 extern void UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 									   XLogRecPtr sublsn);
+extern void UpdateSubscriptionRel(Oid subid, Oid relid, char state,
+								  XLogRecPtr sublsn, char *relslotname, char *reloriginname);
+
 extern char GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn);
+extern void GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname);
+extern void GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull);
+
 extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 
 extern bool HasSubscriptionRelations(Oid subid);
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index a8a89dc784..31b2c41893 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -237,8 +237,9 @@ extern bool InvalidateObsoleteReplicationSlots(ReplicationSlotInvalidationCause
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
 extern bool ReplicationSlotName(int index, Name name);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, Size szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, int64 slotid, char *syncslotname, Size szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List *GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 36c4ff3f59..873eca4f53 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -36,6 +36,23 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/*
+	 * Indicates if the sync worker created a replication slot for itself
+	 * in any point of its lifetime.
+	 * False means that the worker has not created a slot yet, and has been
+	 * reusing replication slots created by other workers so far.
+	 */
+	bool		created_slot;
+
+	/*
+	 * Unique identifier for replication slot to be created by tablesnync
+	 * workers, if needed.
+	 */
+	int64		rep_slot_id;
+
+	/* Replication slot name used by the worker. */
+	char	   *slot_name;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
@@ -243,7 +260,8 @@ extern LogicalRepWorker *logicalrep_worker_find(Oid subid, Oid relid,
 extern List *logicalrep_workers_find(Oid subid, bool only_running);
 extern bool logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname,
 									 Oid userid, Oid relid,
-									 dsm_handle subworker_dsm);
+									 dsm_handle subworker_dsm,
+									 int64 slotid);
 extern void logicalrep_worker_stop(Oid subid, Oid relid);
 extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
 extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
@@ -344,4 +362,7 @@ extern void start_apply(XLogRecPtr origin_startpos);
 extern void DisableSubscriptionAndExit(void);
 extern const char * get_worker_name(void);
 
+/* Invalid identifier to be used for naming replication slots */
+#define InvalidRepSlotId	0
+
 #endif							/* WORKER_INTERNAL_H */
-- 
2.25.1

#102Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#101)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Tue, Jul 18, 2023 at 1:54 AM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi,

PFA updated patches. Rebased 0003 with minor changes. Addressed Peter's reviews for 0001 and 0002 with some small comments below.

Thanks, I will take another look at these soon. FYI, the 0001 patch
does not apply cleanly. It needs to be rebased again because
get_worker_name() function was recently removed from HEAD.

replication/logical/worker.o: In function `InitializeLogRepWorker':
/home/postgres/oss_postgres_misc/src/backend/replication/logical/worker.c:4605:
undefined reference to `get_worker_name'

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#103Peter Smith
smithpb2250@gmail.com
In reply to: Peter Smith (#102)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Tue, Jul 18, 2023 at 11:25 AM Peter Smith <smithpb2250@gmail.com> wrote:

On Tue, Jul 18, 2023 at 1:54 AM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi,

PFA updated patches. Rebased 0003 with minor changes. Addressed Peter's reviews for 0001 and 0002 with some small comments below.

Thanks, I will take another look at these soon. FYI, the 0001 patch
does not apply cleanly. It needs to be rebased again because
get_worker_name() function was recently removed from HEAD.

Sorry, to be more correct -- it applied OK, but failed to build.

Show quoted text

replication/logical/worker.o: In function `InitializeLogRepWorker':
/home/postgres/oss_postgres_misc/src/backend/replication/logical/worker.c:4605:
undefined reference to `get_worker_name'

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#104Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Peter Smith (#103)
5 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Peter,

Peter Smith <smithpb2250@gmail.com>, 18 Tem 2023 Sal, 04:33 tarihinde şunu
yazdı:

On Tue, Jul 18, 2023 at 11:25 AM Peter Smith <smithpb2250@gmail.com>
wrote:

On Tue, Jul 18, 2023 at 1:54 AM Melih Mutlu <m.melihmutlu@gmail.com>

wrote:

Hi,

PFA updated patches. Rebased 0003 with minor changes. Addressed

Peter's reviews for 0001 and 0002 with some small comments below.

Thanks, I will take another look at these soon. FYI, the 0001 patch
does not apply cleanly. It needs to be rebased again because
get_worker_name() function was recently removed from HEAD.

Sorry, to be more correct -- it applied OK, but failed to build.

Attached the fixed patchset.

Thanks,
--
Melih Mutlu
Microsoft

Attachments:

v20-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchapplication/octet-stream; name=v20-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchDownload
From 8d2c69442aa002fd2162237a8f37a38afb67ae27 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Mon, 5 Jun 2023 15:04:41 +0300
Subject: [PATCH v20 1/5] Refactor to split Apply and Tablesync Workers

Both apply and tablesync workers were using ApplyWorkerMain() as entry
point. As the name implies, ApplyWorkerMain() should be considered as
the main function for apply workers. Tablesync worker's path was hidden
and does not have enough in common to share the same main function with
apply worker.

Also, most of the code shared by both worker types is already combined
in LogicalRepApplyLoop(). There is no need to combine the rest in
ApplyWorkerMain() anymore.

This patch introduces TablesyncWorkerMain() as a new entry point for
tablesync workers. This aims to increase code readability and help to
maintain logical replication workers separately.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/postmaster/bgworker.c             |   3 +
 .../replication/logical/applyparallelworker.c |   2 +-
 src/backend/replication/logical/launcher.c    |  32 +-
 src/backend/replication/logical/tablesync.c   | 132 +++++++
 src/backend/replication/logical/worker.c      | 364 ++++++++----------
 src/include/replication/logicalworker.h       |   1 +
 src/include/replication/worker_internal.h     |  10 +-
 7 files changed, 328 insertions(+), 216 deletions(-)

diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 5b4bd71694..505e38376c 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -131,6 +131,9 @@ static const struct
 	},
 	{
 		"ParallelApplyWorkerMain", ParallelApplyWorkerMain
+	},
+	{
+		"TablesyncWorkerMain", TablesyncWorkerMain
 	}
 };
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 6fb96148f4..1d4e83c4c1 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -942,7 +942,7 @@ ParallelApplyWorkerMain(Datum main_arg)
 	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
 		MyLogicalRepWorker->reply_time = 0;
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 542af7d863..e231fa7f95 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -459,24 +459,30 @@ retry:
 	snprintf(bgw.bgw_library_name, MAXPGPATH, "postgres");
 
 	if (is_parallel_apply_worker)
+	{
 		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ParallelApplyWorkerMain");
-	else
-		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
-
-	if (OidIsValid(relid))
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication worker for subscription %u sync %u", subid, relid);
-	else if (is_parallel_apply_worker)
+				 "logical replication parallel apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
+	}
+	else if (OidIsValid(relid))
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "TablesyncWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication parallel apply worker for subscription %u", subid);
+				 "logical replication tablesync worker for subscription %u sync %u",
+				 subid,
+				 relid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication tablesync worker");
+	}
 	else
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication apply worker for subscription %u", subid);
-
-	if (is_parallel_apply_worker)
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
-	else
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication worker");
+				 "logical replication apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication apply worker");
+	}
 
 	bgw.bgw_restart_time = BGW_NEVER_RESTART;
 	bgw.bgw_notify_pid = MyProcPid;
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 6d461654ab..a77d3e3032 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -104,17 +104,21 @@
 #include "nodes/makefuncs.h"
 #include "parser/parse_relation.h"
 #include "pgstat.h"
+#include "postmaster/interrupt.h"
 #include "replication/logicallauncher.h"
 #include "replication/logicalrelation.h"
+#include "replication/logicalworker.h"
 #include "replication/walreceiver.h"
 #include "replication/worker_internal.h"
 #include "replication/slot.h"
 #include "replication/origin.h"
 #include "storage/ipc.h"
 #include "storage/lmgr.h"
+#include "tcop/tcopprot.h"
 #include "utils/acl.h"
 #include "utils/array.h"
 #include "utils/builtins.h"
+#include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
 #include "utils/rls.h"
@@ -1584,6 +1588,134 @@ FetchTableStates(bool *started_tx)
 	return has_subrels;
 }
 
+/*
+ * Execute the initial sync with error handling. Disable the subscription,
+ * if it's required.
+ *
+ * Allocate the slot name in long-lived context on return. Note that we don't
+ * handle FATAL errors which are probably because of system resource error and
+ * are not repeatable.
+ */
+static void
+start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+{
+	char	   *syncslotname = NULL;
+
+	Assert(am_tablesync_worker());
+
+	PG_TRY();
+	{
+		/* Call initial sync. */
+		syncslotname = LogicalRepSyncTableStart(origin_startpos);
+	}
+	PG_CATCH();
+	{
+		if (MySubscription->disableonerr)
+			DisableSubscriptionAndExit();
+		else
+		{
+			/*
+			 * Report the worker failed during table synchronization. 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();
+
+	/* allocate slot name in long-lived context */
+	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+	pfree(syncslotname);
+}
+
+/*
+ * Runs the tablesync worker.
+ *
+ * It starts syncing tables. After a successful sync, sets streaming options
+ * and starts streaming to catchup.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Start table synchronization. */
+	start_table_sync(origin_startpos, &slotname);
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   originname_size);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	/* Start applying changes to catchup. */
+	start_apply(*origin_startpos);
+}
+
+/* Logical Replication Tablesync worker entry point */
+void
+TablesyncWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *myslotname = NULL;
+	WalRcvStreamOptions options;
+
+	/* Attach to slot */
+	logicalrep_worker_attach(worker_slot);
+
+	/* Setup signal handling */
+	pqsignal(SIGHUP, SignalHandlerForConfigReload);
+	pqsignal(SIGTERM, die);
+	BackgroundWorkerUnblockSignals();
+
+	/*
+	 * We don't currently need any ResourceOwner in a walreceiver process, but
+	 * if we did, we could call CreateAuxProcessResourceOwner here.
+	 */
+
+	/* Initialise stats to a sanish value */
+	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
+		MyLogicalRepWorker->reply_time = GetCurrentTimestamp();
+
+	/* Load the libpq-specific functions */
+	load_file("libpqwalreceiver", false);
+
+	InitializeLogRepWorker();
+
+	/* Connect to the origin and start the replication. */
+	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
+		 MySubscription->conninfo);
+
+	/*
+	 * Setup callback for syscache so that we know when something changes in
+	 * the subscription relation state.
+	 */
+	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
+								  invalidate_syncing_table_states,
+								  (Datum) 0);
+
+	run_tablesync_worker(&options,
+						 myslotname,
+						 originname,
+						 sizeof(originname),
+						 &origin_startpos);
+
+	finish_sync_worker();
+}
+
 /*
  * If the subscription has no tables then return false.
  *
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index dd353fd1cb..5f42a5ef40 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -395,8 +395,6 @@ static void stream_close_file(void);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
-static void DisableSubscriptionAndExit(void);
-
 static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
 static void apply_handle_insert_internal(ApplyExecutionData *edata,
 										 ResultRelInfo *relinfo,
@@ -4313,6 +4311,71 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+/*
+ * Sets streaming options including replication slot name and origin start
+ * position. Workers need these options for logical replication.
+ */
+void
+set_stream_options(WalRcvStreamOptions *options,
+				   char *slotname,
+				   XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (!am_tablesync_worker() &&
+		MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+		options->proto.logical.twophase = true;
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4347,24 +4410,18 @@ TwoPhaseTransactionGid(Oid subid, TransactionId xid, char *gid, int szgid)
 }
 
 /*
- * Execute the initial sync with error handling. Disable the subscription,
- * if it's required.
+ * Run the apply loop with error handling. Disable the subscription,
+ * if necessary.
  *
- * Allocate the slot name in long-lived context on return. Note that we don't
- * handle FATAL errors which are probably because of system resource error and
- * are not repeatable.
+ * Note that we don't handle FATAL errors which are probably because
+ * of system resource error and are not repeatable.
  */
-static void
-start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+void
+start_apply(XLogRecPtr origin_startpos)
 {
-	char	   *syncslotname = NULL;
-
-	Assert(am_tablesync_worker());
-
 	PG_TRY();
 	{
-		/* Call initial sync. */
-		syncslotname = LogicalRepSyncTableStart(origin_startpos);
+		LogicalRepApplyLoop(origin_startpos);
 	}
 	PG_CATCH();
 	{
@@ -4373,65 +4430,118 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 		else
 		{
 			/*
-			 * Report the worker failed during table synchronization. Abort
-			 * the current transaction so that the stats message is sent in an
+			 * Report the worker failed while applying changes. Abort the
+			 * current transaction so that the stats message is sent in an
 			 * idle state.
 			 */
 			AbortOutOfAnyTransaction();
-			pgstat_report_subscription_error(MySubscription->oid, false);
+			pgstat_report_subscription_error(MySubscription->oid, !am_tablesync_worker());
 
 			PG_RE_THROW();
 		}
 	}
 	PG_END_TRY();
-
-	/* allocate slot name in long-lived context */
-	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
-	pfree(syncslotname);
 }
 
 /*
- * Run the apply loop with error handling. Disable the subscription,
- * if necessary.
+ * Runs the leader apply worker.
  *
- * Note that we don't handle FATAL errors which are probably because
- * of system resource error and are not repeatable.
+ * It sets up replication origin, streaming options and then starts streaming.
  */
 static void
-start_apply(XLogRecPtr origin_startpos)
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
 {
-	PG_TRY();
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+	bool		must_use_password;
+
+	slotname = MySubscription->slotname;
+
+	/*
+	 * This shouldn't happen if the subscription is enabled, but guard
+	 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+	 * crash if slot is NULL.)
+	 */
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+
+	/* Is the use of a password mandatory? */
+	must_use_password = MySubscription->passwordrequired &&
+		!superuser_arg(MySubscription->owner);
+
+	/* Note that the superuser_arg call can access the DB */
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											must_use_password,
+											MySubscription->name, &err);
+
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+	 * We don't really use the output identify_system for anything but it
+	 * does some initializations on the upstream so let's still call it.
+	 */
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
 	{
-		LogicalRepApplyLoop(origin_startpos);
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
 	}
-	PG_CATCH();
-	{
-		if (MySubscription->disableonerr)
-			DisableSubscriptionAndExit();
-		else
-		{
-			/*
-			 * Report the worker failed while applying changes. Abort the
-			 * current transaction so that the stats message is sent in an
-			 * idle state.
-			 */
-			AbortOutOfAnyTransaction();
-			pgstat_report_subscription_error(MySubscription->oid, !am_tablesync_worker());
 
-			PG_RE_THROW();
-		}
-	}
-	PG_END_TRY();
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+
+	/* Run the main loop. */
+	start_apply(*origin_startpos);
 }
 
 /*
- * Common initialization for leader apply worker and parallel apply worker.
+ * Common initialization for leader apply worker, parallel apply worker and
+ * tablesync worker.
  *
  * Initialize the database connection, in-memory subscription and necessary
  * config options.
  */
 void
-InitializeApplyWorker(void)
+InitializeLogRepWorker(void)
 {
 	MemoryContext oldctx;
 
@@ -4493,7 +4603,7 @@ InitializeApplyWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has started",
+				(errmsg("logical replication worker for subscription \"%s\", table \"%s\" has started",
 						MySubscription->name,
 						get_rel_name(MyLogicalRepWorker->relid))));
 	else
@@ -4513,7 +4623,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	InitializingApplyWorker = true;
 
@@ -4537,7 +4646,7 @@ ApplyWorkerMain(Datum main_arg)
 	/* Load the libpq-specific functions */
 	load_file("libpqwalreceiver", false);
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
@@ -4545,71 +4654,6 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-		bool		must_use_password;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-
-		/* Is the use of a password mandatory? */
-		must_use_password = MySubscription->passwordrequired &&
-			!superuser_arg(MySubscription->owner);
-
-		/* Note that the superuser_arg call can access the DB */
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												must_use_password,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
 	 * the subscription relation state.
@@ -4618,90 +4662,8 @@ ApplyWorkerMain(Datum main_arg)
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
-
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
-
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-
-	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
-	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
-
-	if (!am_tablesync_worker())
-	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
-		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
-		}
-		else
-		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-		}
-
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-								 MySubscription->name,
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-								 "?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
-
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+	/* This is leader apply worker */
+	run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 
 	proc_exit(0);
 }
@@ -4710,7 +4672,7 @@ ApplyWorkerMain(Datum main_arg)
  * After error recovery, disable the subscription in a new transaction
  * and exit cleanly.
  */
-static void
+void
 DisableSubscriptionAndExit(void)
 {
 	/*
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 39588da79f..bbd71d0b42 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -18,6 +18,7 @@ extern PGDLLIMPORT volatile sig_atomic_t ParallelApplyMessagePending;
 
 extern void ApplyWorkerMain(Datum main_arg);
 extern void ParallelApplyWorkerMain(Datum main_arg);
+extern void TablesyncWorkerMain(Datum main_arg);
 
 extern bool IsLogicalWorker(void);
 extern bool IsLogicalParallelApplyWorker(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 343e781896..e7d03cb559 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -19,6 +19,7 @@
 #include "datatype/timestamp.h"
 #include "miscadmin.h"
 #include "replication/logicalrelation.h"
+#include "replication/walreceiver.h"
 #include "storage/buffile.h"
 #include "storage/fileset.h"
 #include "storage/lock.h"
@@ -265,7 +266,7 @@ extern void maybe_reread_subscription(void);
 
 extern void stream_cleanup_files(Oid subid, TransactionId xid);
 
-extern void InitializeApplyWorker(void);
+extern void InitializeLogRepWorker(void);
 
 extern void store_flush_position(XLogRecPtr remote_lsn, XLogRecPtr local_lsn);
 
@@ -326,4 +327,11 @@ am_parallel_apply_worker(void)
 	return isParallelApplyWorker(MyLogicalRepWorker);
 }
 
+
+extern void set_stream_options(WalRcvStreamOptions *options,
+							   char *slotname,
+							   XLogRecPtr *origin_startpos);
+extern void start_apply(XLogRecPtr origin_startpos);
+extern void DisableSubscriptionAndExit(void);
+
 #endif							/* WORKER_INTERNAL_H */
-- 
2.25.1

v20-0002-Reuse-Tablesync-Workers.patchapplication/octet-stream; name=v20-0002-Reuse-Tablesync-Workers.patchDownload
From 7b068ce75a2adf26aabf0fd52e7d09e0b9345ae1 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:04:46 +0300
Subject: [PATCH v20 2/5] Reuse Tablesync Workers

Before this patch, tablesync workers were capable of syncing only one
table. For each table, a new sync worker was launched and that worker would
exit when done processing the table.

Now, tablesync workers are not limited to processing only one
table. When done, they can move to processing another table in
the same subscription.

If there is a table that needs to be synced, an available tablesync
worker picks up that table and syncs it. Each tablesync worker
continues to pick new tables to sync until there are no tables left
requiring synchronization. If there was no available worker to
process the table, then a new tablesync worker will be launched,
provided the number of tablesync workers for the subscription does not
exceed max_sync_workers_per_subscription.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/replication/logical/launcher.c  |   1 +
 src/backend/replication/logical/tablesync.c | 133 +++++++++++++++++---
 src/backend/replication/logical/worker.c    |  36 +++++-
 src/include/replication/worker_internal.h   |   6 +
 4 files changed, 153 insertions(+), 23 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e231fa7f95..72e5ef8a78 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -440,6 +440,7 @@ retry:
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
+	worker->is_sync_completed = false;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index a77d3e3032..d3bb482994 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -134,10 +134,11 @@ static StringInfo copybuf = NULL;
 
 /*
  * Exit routine for synchronization worker.
+ *
+ * If reuse_worker is false, the worker will not be reused and exit.
  */
 static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+finish_sync_worker(bool reuse_worker)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -149,21 +150,33 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
+	/*
+	 * Disconnect from the publisher otherwise reusing the sync worker can
+	 * error due to exceeding max_wal_senders.
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		walrcv_disconnect(LogRepWorkerWalRcvConn);
+		LogRepWorkerWalRcvConn = NULL;
+	}
+
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
-	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
-	CommitTransactionCommand();
-
 	/* Find the leader apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
 
-	/* Stop gracefully */
-	proc_exit(0);
+	if (!reuse_worker)
+	{
+		StartTransactionCommand();
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\" has finished",
+						MySubscription->name)));
+		CommitTransactionCommand();
+
+		/* Stop gracefully */
+		proc_exit(0);
+	}
 }
 
 /*
@@ -383,7 +396,15 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 */
 		replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
+		/* Sync worker has completed synchronization of the current table. */
+		MyLogicalRepWorker->is_sync_completed = true;
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", relation \"%s\" with relid %u has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
+		CommitTransactionCommand();
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -1288,7 +1309,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		case SUBREL_STATE_SYNCDONE:
 		case SUBREL_STATE_READY:
 		case SUBREL_STATE_UNKNOWN:
-			finish_sync_worker();	/* doesn't return */
+			finish_sync_worker(false);	/* doesn't return */
 	}
 
 	/* Calculate the name of the tablesync slot. */
@@ -1645,6 +1666,8 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 					 int originname_size,
 					 XLogRecPtr *origin_startpos)
 {
+	MyLogicalRepWorker->is_sync_completed = false;
+
 	/* Start table synchronization. */
 	start_table_sync(origin_startpos, &slotname);
 
@@ -1672,6 +1695,7 @@ TablesyncWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
+	bool 		done = false;
 
 	/* Attach to slot */
 	logicalrep_worker_attach(worker_slot);
@@ -1707,13 +1731,84 @@ TablesyncWorkerMain(Datum main_arg)
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	run_tablesync_worker(&options,
-						 myslotname,
-						 originname,
-						 sizeof(originname),
-						 &origin_startpos);
+	/*
+	 * The loop where worker does its job. It loops until there is no relation
+	 * left to sync.
+	 */
+	for (;!done;)
+	{
+		List	   *rstates;
+		ListCell   *lc;
+
+		run_tablesync_worker(&options,
+							 myslotname,
+							 originname,
+							 sizeof(originname),
+							 &origin_startpos);
+
+		if (IsTransactionState())
+			CommitTransactionCommand();
+
+		if (MyLogicalRepWorker->is_sync_completed)
+		{
+			/* tablesync is done unless a table that needs syncning is found */
+			done = true;
+
+			/* This transaction will be committed by finish_sync_worker. */
+			StartTransactionCommand();
+
+			/*
+			 * Check if there is any table whose relation state is still INIT.
+			 * If a table in INIT state is found, the worker will not be
+			 * finished, it will be reused instead.
+			 */
+			rstates = GetSubscriptionRelations(MySubscription->oid, true);
+
+			foreach(lc, rstates)
+			{
+				SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
+
+				if (rstate->state == SUBREL_STATE_SYNCDONE)
+					continue;
+
+				/*
+				 * Take exclusive lock to prevent any other sync worker from
+				 * picking the same table.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
+
+				/*
+				 * Pick the table for the next run if it is not already picked
+				 * up by another worker.
+				 */
+				if (!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+				{
+					/* Update worker state for the next table */
+					MyLogicalRepWorker->relid = rstate->relid;
+					MyLogicalRepWorker->relstate = rstate->state;
+					MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+					LWLockRelease(LogicalRepWorkerLock);
+
+					/* Found a table for next iteration */
+					finish_sync_worker(true);
+
+					StartTransactionCommand();
+					ereport(LOG,
+							(errmsg("logical replication worker for subscription \"%s\" will be reused to sync table \"%s\" with relid %u.",
+									MySubscription->name,
+									get_rel_name(MyLogicalRepWorker->relid),
+									MyLogicalRepWorker->relid)));
+					CommitTransactionCommand();
+
+					done = false;
+					break;
+				}
+				LWLockRelease(LogicalRepWorkerLock);
+			}
+		}
+	}
 
-	finish_sync_worker();
+	finish_sync_worker(false);
 }
 
 /*
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 5f42a5ef40..ed8b20eb34 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3607,6 +3607,20 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can call process_syncing_tables_for_sync.
+				 *
+				 * process_syncing_tables_for_sync decides whether the sync of
+				 * the current table is completed. If it is completed,
+				 * streaming must be already ended. So, we can break the loop.
+				 */
+				if (MyLogicalRepWorker->is_sync_completed)
+				{
+					endofstream = true;
+					break;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -3626,6 +3640,15 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+
+			/*
+			 * If is_sync_completed is true, this means that the tablesync
+			 * worker is done with synchronization. Streaming has already been
+			 * ended by process_syncing_tables_for_sync. We should move to the
+			 * next table if needed, or exit.
+			 */
+			if (MyLogicalRepWorker->is_sync_completed)
+				endofstream = true;
 		}
 
 		/* Cleanup the memory. */
@@ -3728,8 +3751,12 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * End streaming here for only apply workers. Ending streaming for
+	 * tablesync workers is deferred until the worker exits its main loop.
+	 */
+	if (!am_tablesync_worker())
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
 }
 
 /*
@@ -4603,9 +4630,10 @@ InitializeLogRepWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("logical replication worker for subscription \"%s\", table \"%s\" has started",
+				(errmsg("logical replication worker for subscription \"%s\", table \"%s\" with relid %u has started",
 						MySubscription->name,
-						get_rel_name(MyLogicalRepWorker->relid))));
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
 	else
 		ereport(LOG,
 				(errmsg("logical replication apply worker for subscription \"%s\" has started",
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index e7d03cb559..891b020aff 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -57,6 +57,12 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/*
+	 * Indicates whether tablesync worker has completed syncing its assigned
+	 * table.
+	 */
+	bool		is_sync_completed;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction or
-- 
2.25.1

v20-0003-Reuse-connection-when-tablesync-workers-change-t.patchapplication/octet-stream; name=v20-0003-Reuse-connection-when-tablesync-workers-change-t.patchDownload
From c592a0175cf37c0033fc4cb1fc33d7d312cb83d6 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:13:52 +0300
Subject: [PATCH v20 3/5] Reuse connection when tablesync workers change the
 target

Previously tablesync workers establish new connections when it changes the syncing
table, but this might have additional overhead. This patch allows to reuse connections
instead.

As for the publisher node, this patch allows to reuse logical walsender processes
after the streaming is done once.
---
 src/backend/replication/logical/launcher.c  |  1 +
 src/backend/replication/logical/tablesync.c | 56 ++++++++++++++-------
 src/backend/replication/logical/worker.c    | 18 ++++---
 src/backend/replication/walsender.c         |  7 +++
 src/include/replication/worker_internal.h   |  3 ++
 5 files changed, 58 insertions(+), 27 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 72e5ef8a78..945619b603 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
 	worker->is_sync_completed = false;
+	worker->worker_slot = slot;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index d3bb482994..73b6fd77e2 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -150,16 +150,6 @@ finish_sync_worker(bool reuse_worker)
 		pgstat_report_stat(true);
 	}
 
-	/*
-	 * Disconnect from the publisher otherwise reusing the sync worker can
-	 * error due to exceeding max_wal_senders.
-	 */
-	if (LogRepWorkerWalRcvConn != NULL)
-	{
-		walrcv_disconnect(LogRepWorkerWalRcvConn);
-		LogRepWorkerWalRcvConn = NULL;
-	}
-
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
@@ -1258,6 +1248,24 @@ ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
 			 relid, GetSystemIdentifier());
 }
 
+/*
+ * Determine the application_name for tablesync workers.
+ *
+ * FIXME: set appropriate application_name. Previously, the slot name was used
+ * because the lifetime of the tablesync worker was same as that, but now the
+ * tablesync worker handles many slots during the synchronization so that it is
+ * not suitable. So what should be? Note that if the tablesync worker starts to
+ * reuse the replication slot during synchronization, we should use the slot
+ * name as application_name again.
+ */
+static void
+ApplicationNameForTablesync(Oid suboid, int worker_slot,
+							char *application_name, Size szapp)
+{
+	snprintf(application_name, szapp, "pg_%u_sync_%i_" UINT64_FORMAT, suboid,
+			 worker_slot, GetSystemIdentifier());
+}
+
 /*
  * Start syncing the table in the sync worker.
  *
@@ -1319,15 +1327,25 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 									slotname,
 									NAMEDATALEN);
 
-	/*
-	 * Here we use the slot name instead of the subscription name as the
-	 * application_name, so that it is different from the leader apply worker,
-	 * so that synchronous replication can distinguish them.
-	 */
-	LogRepWorkerWalRcvConn =
-		walrcv_connect(MySubscription->conninfo, true,
-					   must_use_password,
-					   slotname, &err);
+	/* Connect to the publisher if haven't done so already. */
+	if (LogRepWorkerWalRcvConn == NULL)
+	{
+		char application_name[NAMEDATALEN];
+
+		/*
+		 * The application_name must be also different from the leader apply
+		 * worker because synchronous replication must distinguish them.
+		 */
+		ApplicationNameForTablesync(MySubscription->oid,
+									MyLogicalRepWorker->worker_slot,
+									application_name,
+									NAMEDATALEN);
+		LogRepWorkerWalRcvConn =
+			walrcv_connect(MySubscription->conninfo, true,
+						   must_use_password,
+						   application_name, &err);
+	}
+
 	if (LogRepWorkerWalRcvConn == NULL)
 		ereport(ERROR,
 				(errcode(ERRCODE_CONNECTION_FAILURE),
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index ed8b20eb34..7ba58d0e67 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3480,20 +3480,22 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	ErrorContextCallback errcallback;
 
 	/*
-	 * Init the ApplyMessageContext which we clean up after each replication
-	 * protocol message.
+	 * Init the ApplyMessageContext if needed. This context is cleaned up
+	 * after each replication protocol message.
 	 */
-	ApplyMessageContext = AllocSetContextCreate(ApplyContext,
-												"ApplyMessageContext",
-												ALLOCSET_DEFAULT_SIZES);
+	if (!ApplyMessageContext)
+		ApplyMessageContext = AllocSetContextCreate(ApplyContext,
+													"ApplyMessageContext",
+													ALLOCSET_DEFAULT_SIZES);
 
 	/*
 	 * This memory context is used for per-stream data when the streaming mode
 	 * is enabled. This context is reset on each stream stop.
 	 */
-	LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
-													"LogicalStreamingContext",
-													ALLOCSET_DEFAULT_SIZES);
+	if (!LogicalStreamingContext)
+		LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
+														"LogicalStreamingContext",
+														ALLOCSET_DEFAULT_SIZES);
 
 	/* mark as idle, before starting to loop */
 	pgstat_report_activity(STATE_IDLE, NULL);
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index d27ef2985d..2f3e93cc40 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1830,7 +1830,14 @@ exec_replication_command(const char *cmd_string)
 				if (cmd->kind == REPLICATION_KIND_PHYSICAL)
 					StartReplication(cmd);
 				else
+				{
+					/*
+					 * Reset flags because reusing tablesync workers can mean
+					 * this is the second time here.
+					 */
+					streamingDoneSending = streamingDoneReceiving = false;
 					StartLogicalReplication(cmd);
+				}
 
 				/* dupe, but necessary per libpqrcv_endstreaming */
 				EndReplicationCommand(cmdtag);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 891b020aff..804c475746 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -63,6 +63,9 @@ typedef struct LogicalRepWorker
 	 */
 	bool		is_sync_completed;
 
+	/* Indicates the slot number which corresponds to this LogicalRepWorker. */
+	int			worker_slot;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction or
-- 
2.25.1

v20-0004-Add-replication-protocol-cmd-to-create-a-snapsho.patchapplication/octet-stream; name=v20-0004-Add-replication-protocol-cmd-to-create-a-snapsho.patchDownload
From 9e46a7e71f597de20d5119de994093a779c8f49b Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 13 Oct 2022 17:05:45 +0300
Subject: [PATCH v20 4/5] Add replication protocol cmd to create a snapshot

Introduced CREATE_REPLICATION_SNAPSHOT to be able to create and use a
snapshot without creating a new replication slot, but by using an
existing slot.

CREATE_REPLICATION_SNAPSHOT simply does what CREATE_REPLICATION_SLOT does
without creating a new replication slot.

CREATE_REPLICATION_SNAPSHOT command imports the snapshot into the current
transaction and returns consistent_point. The changes earlier than the
consistent_point will be applied by importing the snapshot. All changes
later than the consistent_point will be available to be consumed from
the replication slot.

This is useful for reusing replication slots in logical replication.
Otherwise, tablesync workers cannot start from a consistent point to copy
a relation and then apply changes by consuming from replication slot.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/protocol.sgml                    |  31 ++++++
 .../libpqwalreceiver/libpqwalreceiver.c       |  69 +++++++++++-
 src/backend/replication/logical/logical.c     |  40 ++++++-
 .../replication/logical/logicalfuncs.c        |   1 +
 src/backend/replication/repl_gram.y           |  18 ++-
 src/backend/replication/repl_scanner.l        |   2 +
 src/backend/replication/slotfuncs.c           |   1 +
 src/backend/replication/walsender.c           | 104 +++++++++++++++++-
 src/include/nodes/replnodes.h                 |  11 ++
 src/include/replication/logical.h             |   1 +
 src/include/replication/walreceiver.h         |  13 +++
 src/tools/pgindent/typedefs.list              |   2 +
 12 files changed, 289 insertions(+), 4 deletions(-)

diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index b11d9a6ba3..f7d5acee93 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2595,6 +2595,37 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
      </listitem>
     </varlistentry>
 
+    <varlistentry id="protocol-replication-replication-slot-snapshot">
+     <term><literal>CREATE_REPLICATION_SNAPSHOT</literal> <replaceable class="parameter">slot_name</replaceable> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
+      <indexterm><primary>CREATE_REPLICATION_SNAPSHOT</primary></indexterm>
+     </term>
+     <listitem>
+      <para>
+       Creates a snapshot including all the changes from the replication slot until
+       the point at which the replication slot becomes consistent. Then the snapshot
+       is used in the current transaction. This command is currently only supported
+       for logical replication slots.
+      </para>
+
+      <para>
+       In response to this command, the server will return a one-row result set,
+       containing the following field:
+       <variablelist>
+        <varlistentry>
+         <term><literal>consistent_point</literal> (<type>text</type>)</term>
+         <listitem>
+          <para>
+           The WAL location at which the slot became consistent.  This is the
+           earliest location from which streaming can start on this replication
+           slot.
+          </para>
+         </listitem>
+        </varlistentry>
+       </variablelist>
+      </para>
+     </listitem>
+    </varlistentry>
+
     <varlistentry id="protocol-replication-base-backup" xreflabel="BASE_BACKUP">
      <term><literal>BASE_BACKUP</literal> [ ( <replaceable class="parameter">option</replaceable> [, ...] ) ]
       <indexterm><primary>BASE_BACKUP</primary></indexterm>
diff --git a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
index 60d5c1fc40..ac929fe41e 100644
--- a/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
+++ b/src/backend/replication/libpqwalreceiver/libpqwalreceiver.c
@@ -82,6 +82,8 @@ static WalRcvExecResult *libpqrcv_exec(WalReceiverConn *conn,
 									   const int nRetTypes,
 									   const Oid *retTypes);
 static void libpqrcv_disconnect(WalReceiverConn *conn);
+static void libpqrcv_slot_snapshot(WalReceiverConn *conn, char *slotname,
+								   const WalRcvStreamOptions *options, XLogRecPtr *lsn);
 
 static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_connect = libpqrcv_connect,
@@ -98,7 +100,8 @@ static WalReceiverFunctionsType PQWalReceiverFunctions = {
 	.walrcv_create_slot = libpqrcv_create_slot,
 	.walrcv_get_backend_pid = libpqrcv_get_backend_pid,
 	.walrcv_exec = libpqrcv_exec,
-	.walrcv_disconnect = libpqrcv_disconnect
+	.walrcv_disconnect = libpqrcv_disconnect,
+	.walrcv_slot_snapshot = libpqrcv_slot_snapshot
 };
 
 /* Prototypes for private functions */
@@ -982,6 +985,70 @@ libpqrcv_create_slot(WalReceiverConn *conn, const char *slotname,
 	return snapshot;
 }
 
+/*
+ * Create a new snapshot by using an existing replication slot.
+ */
+static void
+libpqrcv_slot_snapshot(WalReceiverConn *conn,
+					   char *slotname,
+					   const WalRcvStreamOptions *options,
+					   XLogRecPtr *lsn)
+{
+	StringInfoData cmd;
+	PGresult   *res;
+	char	   *pubnames_str;
+	List	   *pubnames;
+	char	   *pubnames_literal;
+
+	initStringInfo(&cmd);
+
+	/* Build the command. */
+	appendStringInfo(&cmd, "CREATE_REPLICATION_SNAPSHOT \"%s\"", slotname);
+	appendStringInfoString(&cmd, " (");
+	appendStringInfo(&cmd, " proto_version '%u'",
+					 options->proto.logical.proto_version);
+
+	/* Add publication names. */
+	pubnames = options->proto.logical.publication_names;
+	pubnames_str = stringlist_to_identifierstr(conn->streamConn, pubnames);
+	if (!pubnames_str)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("Could not create a snapshot by replication slot %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	pubnames_literal = PQescapeLiteral(conn->streamConn, pubnames_str,
+									   strlen(pubnames_str));
+	if (!pubnames_literal)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),	/* likely guess */
+				 errmsg("Could not create a snapshot by replication slot %s",
+						pchomp(PQerrorMessage(conn->streamConn)))));
+	appendStringInfo(&cmd, ", publication_names %s", pubnames_literal);
+	PQfreemem(pubnames_literal);
+	pfree(pubnames_str);
+
+	appendStringInfoString(&cmd, " )");
+
+	/* Execute the command. */
+	res = libpqrcv_PQexec(conn->streamConn, cmd.data);
+	pfree(cmd.data);
+
+	if (PQresultStatus(res) != PGRES_TUPLES_OK)
+	{
+		PQclear(res);
+		ereport(ERROR,
+				(errcode(ERRCODE_PROTOCOL_VIOLATION),
+				 errmsg("Could not create a snapshot by replication slot \"%s\": %s",
+						slotname, pchomp(PQerrorMessage(conn->streamConn)))));
+	}
+
+	if (lsn)
+		*lsn = DatumGetLSN(DirectFunctionCall1Coll(pg_lsn_in, InvalidOid,
+												   CStringGetDatum(PQgetvalue(res, 0, 0))));
+
+	PQclear(res);
+}
+
 /*
  * Return PID of remote backend process.
  */
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 41243d0187..3c11f8e1dd 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -476,6 +476,10 @@ CreateInitDecodingContext(const char *plugin,
  * fast_forward
  *		bypass the generation of logical changes.
  *
+ * need_full_snapshot
+ * 		if true, must obtain a snapshot able to read all tables;
+ *  	if false, one that can read only catalogs is acceptable.
+ *
  * xl_routine
  *		XLogReaderRoutine used by underlying xlogreader
  *
@@ -494,6 +498,7 @@ LogicalDecodingContext *
 CreateDecodingContext(XLogRecPtr start_lsn,
 					  List *output_plugin_options,
 					  bool fast_forward,
+					  bool need_full_snapshot,
 					  XLogReaderRoutine *xl_routine,
 					  LogicalOutputPluginWriterPrepareWrite prepare_write,
 					  LogicalOutputPluginWriterWrite do_write,
@@ -502,6 +507,7 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 	LogicalDecodingContext *ctx;
 	ReplicationSlot *slot;
 	MemoryContext old_context;
+	TransactionId xmin_horizon = InvalidTransactionId;
 
 	/* shorter lines... */
 	slot = MyReplicationSlot;
@@ -571,8 +577,40 @@ CreateDecodingContext(XLogRecPtr start_lsn,
 		start_lsn = slot->data.confirmed_flush;
 	}
 
+
+	/*
+	 * We need to determine a safe xmin horizon to start decoding from if we
+	 * want to create a snapshot too. Otherwise we would end up with a
+	 * snapshot that cannot be imported since xmin value from the snapshot may
+	 * be less than the oldest safe xmin. To avoid this call
+	 * GetOldestSafeDecodingTransactionId() to return a safe xmin value, which
+	 * can be used while exporting/importing the snapshot.
+	 *
+	 * So we have to acquire the ProcArrayLock to prevent computation of new
+	 * xmin horizons by other backends, get the safe decoding xid, and inform
+	 * the slot machinery about the new limit. Once that's done the
+	 * ProcArrayLock can be released as the slot machinery now is protecting
+	 * against vacuum.
+	 */
+	if (need_full_snapshot)
+	{
+		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+		xmin_horizon = GetOldestSafeDecodingTransactionId(!need_full_snapshot);
+
+		SpinLockAcquire(&slot->mutex);
+		slot->effective_catalog_xmin = xmin_horizon;
+		slot->data.catalog_xmin = xmin_horizon;
+		slot->effective_xmin = xmin_horizon;
+		SpinLockRelease(&slot->mutex);
+
+		ReplicationSlotsComputeRequiredXmin(true);
+
+		LWLockRelease(ProcArrayLock);
+	}
+
 	ctx = StartupDecodingContext(output_plugin_options,
-								 start_lsn, InvalidTransactionId, false,
+								 start_lsn, xmin_horizon, need_full_snapshot,
 								 fast_forward, xl_routine, prepare_write,
 								 do_write, update_progress);
 
diff --git a/src/backend/replication/logical/logicalfuncs.c b/src/backend/replication/logical/logicalfuncs.c
index 55a24c02c9..85c5cdb633 100644
--- a/src/backend/replication/logical/logicalfuncs.c
+++ b/src/backend/replication/logical/logicalfuncs.c
@@ -208,6 +208,7 @@ pg_logical_slot_get_changes_guts(FunctionCallInfo fcinfo, bool confirm, bool bin
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									options,
 									false,
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/repl_gram.y b/src/backend/replication/repl_gram.y
index 0c874e33cf..957a5cc653 100644
--- a/src/backend/replication/repl_gram.y
+++ b/src/backend/replication/repl_gram.y
@@ -65,6 +65,7 @@ Node *replication_parse_result;
 %token K_CREATE_REPLICATION_SLOT
 %token K_DROP_REPLICATION_SLOT
 %token K_TIMELINE_HISTORY
+%token K_CREATE_REPLICATION_SNAPSHOT
 %token K_WAIT
 %token K_TIMELINE
 %token K_PHYSICAL
@@ -80,7 +81,7 @@ Node *replication_parse_result;
 %type <node>	command
 %type <node>	base_backup start_replication start_logical_replication
 				create_replication_slot drop_replication_slot identify_system
-				read_replication_slot timeline_history show
+				read_replication_slot timeline_history show create_replication_snapshot
 %type <list>	generic_option_list
 %type <defelt>	generic_option
 %type <uintval>	opt_timeline
@@ -114,6 +115,7 @@ command:
 			| read_replication_slot
 			| timeline_history
 			| show
+			| create_replication_snapshot
 			;
 
 /*
@@ -307,6 +309,19 @@ timeline_history:
 				}
 			;
 
+/*
+ * CREATE_REPLICATION_SNAPSHOT %s options
+ */
+create_replication_snapshot:
+			K_CREATE_REPLICATION_SNAPSHOT var_name plugin_options
+				{
+					CreateReplicationSnapshotCmd *n = makeNode(CreateReplicationSnapshotCmd);
+					n->slotname = $2;
+					n->options = $3;
+					$$ = (Node *) n;
+				}
+			;
+
 opt_physical:
 			K_PHYSICAL
 			| /* EMPTY */
@@ -400,6 +415,7 @@ ident_or_keyword:
 			| K_CREATE_REPLICATION_SLOT	{ $$ = "create_replication_slot"; }
 			| K_DROP_REPLICATION_SLOT		{ $$ = "drop_replication_slot"; }
 			| K_TIMELINE_HISTORY			{ $$ = "timeline_history"; }
+			| K_CREATE_REPLICATION_SNAPSHOT	{ $$ = "create_replication_snapshot"; }
 			| K_WAIT						{ $$ = "wait"; }
 			| K_TIMELINE					{ $$ = "timeline"; }
 			| K_PHYSICAL					{ $$ = "physical"; }
diff --git a/src/backend/replication/repl_scanner.l b/src/backend/replication/repl_scanner.l
index 1cc7fb858c..0cc2cdd0be 100644
--- a/src/backend/replication/repl_scanner.l
+++ b/src/backend/replication/repl_scanner.l
@@ -126,6 +126,7 @@ START_REPLICATION	{ return K_START_REPLICATION; }
 CREATE_REPLICATION_SLOT		{ return K_CREATE_REPLICATION_SLOT; }
 DROP_REPLICATION_SLOT		{ return K_DROP_REPLICATION_SLOT; }
 TIMELINE_HISTORY	{ return K_TIMELINE_HISTORY; }
+CREATE_REPLICATION_SNAPSHOT	{ return K_CREATE_REPLICATION_SNAPSHOT; }
 PHYSICAL			{ return K_PHYSICAL; }
 RESERVE_WAL			{ return K_RESERVE_WAL; }
 LOGICAL				{ return K_LOGICAL; }
@@ -303,6 +304,7 @@ replication_scanner_is_replication_command(void)
 		case K_DROP_REPLICATION_SLOT:
 		case K_READ_REPLICATION_SLOT:
 		case K_TIMELINE_HISTORY:
+		case K_CREATE_REPLICATION_SNAPSHOT:
 		case K_SHOW:
 			/* Yes; push back the first token so we can parse later. */
 			repl_pushed_back_token = first_token;
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 6035cf4816..c5b2d5b61f 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -486,6 +486,7 @@ pg_logical_replication_slot_advance(XLogRecPtr moveto)
 		ctx = CreateDecodingContext(InvalidXLogRecPtr,
 									NIL,
 									true,	/* fast_forward */
+									false,
 									XL_ROUTINE(.page_read = read_local_xlog_page,
 											   .segment_open = wal_segment_open,
 											   .segment_close = wal_segment_close),
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 2f3e93cc40..88f74fb76a 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -238,6 +238,7 @@ static void CreateReplicationSlot(CreateReplicationSlotCmd *cmd);
 static void DropReplicationSlot(DropReplicationSlotCmd *cmd);
 static void StartReplication(StartReplicationCmd *cmd);
 static void StartLogicalReplication(StartReplicationCmd *cmd);
+static void CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd);
 static void ProcessStandbyMessage(void);
 static void ProcessStandbyReplyMessage(void);
 static void ProcessStandbyHSFeedbackMessage(void);
@@ -1283,7 +1284,7 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	 * are reported early.
 	 */
 	logical_decoding_ctx =
-		CreateDecodingContext(cmd->startpoint, cmd->options, false,
+		CreateDecodingContext(cmd->startpoint, cmd->options, false, false,
 							  XL_ROUTINE(.page_read = logical_read_xlog_page,
 										 .segment_open = WalSndSegmentOpen,
 										 .segment_close = wal_segment_close),
@@ -1335,6 +1336,98 @@ StartLogicalReplication(StartReplicationCmd *cmd)
 	EndCommand(&qc, DestRemote, false);
 }
 
+/*
+ * Create a snapshot from an existing replication slot.
+ */
+static void
+CreateReplicationSnapshot(CreateReplicationSnapshotCmd *cmd)
+{
+	Snapshot	snap;
+	LogicalDecodingContext *ctx;
+	char		xloc[MAXFNAMELEN];
+	DestReceiver *dest;
+	TupOutputState *tstate;
+	TupleDesc	tupdesc;
+	Datum		values[1];
+	bool		nulls[1] = {0};
+
+	Assert(!MyReplicationSlot);
+
+	CheckLogicalDecodingRequirements();
+
+	if (!IsTransactionBlock())
+		ereport(ERROR,
+				(errmsg("%s must be called inside a transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (XactIsoLevel != XACT_REPEATABLE_READ)
+		ereport(ERROR,
+				(errmsg("%s must be called in REPEATABLE READ isolation mode transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (!XactReadOnly)
+		ereport(ERROR,
+				(errmsg("%s must be called in a read only transaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (FirstSnapshotSet)
+		ereport(ERROR,
+				(errmsg("%s must be called before any query",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	if (IsSubTransaction())
+		ereport(ERROR,
+				(errmsg("%s must not be called in a subtransaction",
+						"CREATE_REPLICATION_SNAPSHOT ...")));
+
+	ReplicationSlotAcquire(cmd->slotname, false);
+
+	ctx = CreateDecodingContext(MyReplicationSlot->data.restart_lsn,
+								cmd->options,
+								false,
+								true,
+								XL_ROUTINE(.page_read = logical_read_xlog_page,
+										   .segment_open = WalSndSegmentOpen,
+										   .segment_close = wal_segment_close),
+								WalSndPrepareWrite, WalSndWriteData,
+								WalSndUpdateProgress);
+
+	/*
+	 * Signal that we don't need the timeout mechanism. We're just creating
+	 * the snapshot with the replication slot and don't yet accept feedback
+	 * messages or send keepalives. As we possibly need to wait for further
+	 * WAL the walsender would otherwise possibly be killed too soon.
+	 */
+	last_reply_timestamp = 0;
+
+	/* build initial snapshot, might take a while */
+	DecodingContextFindStartpoint(ctx);
+
+	snap = SnapBuildInitialSnapshot(ctx->snapshot_builder);
+	RestoreTransactionSnapshot(snap, MyProc);
+
+	/* Don't need the decoding context anymore */
+	FreeDecodingContext(ctx);
+
+	/* Create a tuple to send consistent WAL location */
+	snprintf(xloc, sizeof(xloc), "%X/%X",
+			 LSN_FORMAT_ARGS(MyReplicationSlot->data.confirmed_flush));
+
+	dest = CreateDestReceiver(DestRemoteSimple);
+	tupdesc = CreateTemplateTupleDesc(1);
+	TupleDescInitBuiltinEntry(tupdesc, (AttrNumber) 1, "consistent_point",
+							  TEXTOID, -1, 0);
+	tstate = begin_tup_output_tupdesc(dest, tupdesc, &TTSOpsVirtual);
+
+	/* consistent wal location */
+	values[0] = CStringGetTextDatum(xloc);
+
+	do_tup_output(tstate, values, nulls);
+	end_tup_output(tstate);
+
+	ReplicationSlotRelease();
+}
+
 /*
  * LogicalDecodingContext 'prepare_write' callback.
  *
@@ -1870,6 +1963,15 @@ exec_replication_command(const char *cmd_string)
 			}
 			break;
 
+		case T_CreateReplicationSnapshotCmd:
+			{
+				cmdtag = "CREATE_REPLICATION_SNAPSHOT";
+				set_ps_display(cmdtag);
+				CreateReplicationSnapshot((CreateReplicationSnapshotCmd *) cmd_node);
+				EndReplicationCommand(cmdtag);
+				break;
+			}
+
 		default:
 			elog(ERROR, "unrecognized replication command node tag: %u",
 				 cmd_node->type);
diff --git a/src/include/nodes/replnodes.h b/src/include/nodes/replnodes.h
index 4321ba8f86..154ab74f33 100644
--- a/src/include/nodes/replnodes.h
+++ b/src/include/nodes/replnodes.h
@@ -108,4 +108,15 @@ typedef struct TimeLineHistoryCmd
 	TimeLineID	timeline;
 } TimeLineHistoryCmd;
 
+/* ----------------------
+ *		CREATE_REPLICATION_SNAPSHOT command
+ * ----------------------
+ */
+typedef struct CreateReplicationSnapshotCmd
+{
+	NodeTag		type;
+	char	   *slotname;
+	List	   *options;
+} CreateReplicationSnapshotCmd;
+
 #endif							/* REPLNODES_H */
diff --git a/src/include/replication/logical.h b/src/include/replication/logical.h
index 5f49554ea0..6535786a0e 100644
--- a/src/include/replication/logical.h
+++ b/src/include/replication/logical.h
@@ -125,6 +125,7 @@ extern LogicalDecodingContext *CreateInitDecodingContext(const char *plugin,
 extern LogicalDecodingContext *CreateDecodingContext(XLogRecPtr start_lsn,
 													 List *output_plugin_options,
 													 bool fast_forward,
+													 bool need_full_snapshot,
 													 XLogReaderRoutine *xl_routine,
 													 LogicalOutputPluginWriterPrepareWrite prepare_write,
 													 LogicalOutputPluginWriterWrite do_write,
diff --git a/src/include/replication/walreceiver.h b/src/include/replication/walreceiver.h
index 281626fa6f..f01e5f23a7 100644
--- a/src/include/replication/walreceiver.h
+++ b/src/include/replication/walreceiver.h
@@ -386,6 +386,16 @@ typedef WalRcvExecResult *(*walrcv_exec_fn) (WalReceiverConn *conn,
  */
 typedef void (*walrcv_disconnect_fn) (WalReceiverConn *conn);
 
+/*
+ * walrcv_slot_snapshot_fn
+ *
+ * Create a snapshot by an existing replication slot
+ */
+typedef void (*walrcv_slot_snapshot_fn) (WalReceiverConn *conn,
+										 char *slotname,
+										 const WalRcvStreamOptions *options,
+										 XLogRecPtr *lsn);
+
 typedef struct WalReceiverFunctionsType
 {
 	walrcv_connect_fn walrcv_connect;
@@ -403,6 +413,7 @@ typedef struct WalReceiverFunctionsType
 	walrcv_get_backend_pid_fn walrcv_get_backend_pid;
 	walrcv_exec_fn walrcv_exec;
 	walrcv_disconnect_fn walrcv_disconnect;
+	walrcv_slot_snapshot_fn walrcv_slot_snapshot;
 } WalReceiverFunctionsType;
 
 extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
@@ -437,6 +448,8 @@ extern PGDLLIMPORT WalReceiverFunctionsType *WalReceiverFunctions;
 	WalReceiverFunctions->walrcv_exec(conn, exec, nRetTypes, retTypes)
 #define walrcv_disconnect(conn) \
 	WalReceiverFunctions->walrcv_disconnect(conn)
+#define walrcv_slot_snapshot(conn, slotname, options, lsn) \
+	WalReceiverFunctions->walrcv_slot_snapshot(conn, slotname, options, lsn)
 
 static inline void
 walrcv_clear_result(WalRcvExecResult *walres)
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index e941fb6c82..cc023b10fa 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2348,6 +2348,7 @@ ReplicationSlotInvalidationCause
 ReplicationSlotOnDisk
 ReplicationSlotPersistency
 ReplicationSlotPersistentData
+CreateReplicationSnapshotCmd
 ReplicationState
 ReplicationStateCtl
 ReplicationStateOnDisk
@@ -3855,6 +3856,7 @@ walrcv_receive_fn
 walrcv_send_fn
 walrcv_server_version_fn
 walrcv_startstreaming_fn
+walrcv_slot_snapshot_fn
 wchar2mb_with_len_converter
 wchar_t
 win32_deadchild_waitinfo
-- 
2.25.1

v20-0005-Reuse-Replication-Slot-and-Origin-in-Tablesync.patchapplication/octet-stream; name=v20-0005-Reuse-Replication-Slot-and-Origin-in-Tablesync.patchDownload
From 6af61e78faa5b60cbcdbe0ae074fa51876d7636f Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Thu, 2 Jun 2022 17:39:37 +0300
Subject: [PATCH v20 5/5] Reuse Replication Slot and Origin in Tablesync

This commit allows reusing replication slots and origins during tablesync.

Earlier, a tablesync worker was creating a new replication slot and origin
each time it syncs a new table. With this patch, replication
slots/origins can be reusable for tablesync.

This reduces the overhead of creating/dropping replication slots and origins
and improves tablesync speed significantly especially for empty or small tables.

If the state of the current table is INIT or DATASYNC, tablesync worker needs a
repliation slot/origin. If the worker has not created slot and origin in
its previous runs, it will create those first. Otherwise the worker reuses
slot and origin created by the same worker in previous iterations earlier.
Tables in FINISHEDCOPY are expected to have a replication slot and origin.
Slot and origin names for such tables are persisted in
pg_subscription_rel catalog. Tablesync worker can fetch them and proceed
with existing slot and origin of FINISHEDCOPY tables and does not need to
create new ones.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 doc/src/sgml/catalogs.sgml                    |  31 ++
 src/backend/catalog/pg_subscription.c         | 247 ++++++++++-
 src/backend/commands/subscriptioncmds.c       | 229 ++++++----
 .../replication/logical/applyparallelworker.c |   3 +-
 src/backend/replication/logical/launcher.c    |   8 +-
 src/backend/replication/logical/tablesync.c   | 414 ++++++++++++++----
 src/backend/replication/logical/worker.c      |  17 +-
 src/include/catalog/pg_subscription.h         |   6 +
 src/include/catalog/pg_subscription_rel.h     |  14 +-
 src/include/replication/slot.h                |   3 +-
 src/include/replication/worker_internal.h     |  23 +-
 11 files changed, 804 insertions(+), 191 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 307ad88b50..200a6b2ae9 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8031,6 +8031,19 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        origin.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>sublastusedid</structfield> <type>int8</type>
+      </para>
+      <para>
+      The last used ID for tablesync workers. It acts as an unique identifier
+      for replication slots which are created by tablesync workers.
+      The last used ID needs to be persisted to make logical replication safely
+      proceed after any interruption. If sublastusedid is 0, then no table has
+      been synced yet.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
@@ -8115,6 +8128,24 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        otherwise null
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srrelslotname</structfield> <type>name</type>
+      </para>
+      <para>
+       Replication slot name that is used for synchronization of relation
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>srreloriginname</structfield> <type>name</type>
+      </para>
+      <para>
+       Origin name that is used for tracking synchronization of relation
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index d07f88ce28..152fdaa310 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -108,6 +108,14 @@ GetSubscription(Oid subid, bool missing_ok)
 								   Anum_pg_subscription_suborigin);
 	sub->origin = TextDatumGetCString(datum);
 
+	/* Get last used id */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID,
+							tup,
+							Anum_pg_subscription_sublastusedid,
+							&isnull);
+	Assert(!isnull);
+	sub->lastusedid = DatumGetInt64(datum);
+
 	ReleaseSysCache(tup);
 
 	return sub;
@@ -199,6 +207,44 @@ DisableSubscription(Oid subid)
 	table_close(rel, NoLock);
 }
 
+/*
+ * Update the last used replication slot ID for the given subscription.
+ */
+void
+UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid)
+{
+	Relation	rel;
+	bool		nulls[Natts_pg_subscription];
+	bool		replaces[Natts_pg_subscription];
+	Datum		values[Natts_pg_subscription];
+	HeapTuple	tup;
+
+	/* Look up the subscription in the catalog */
+	rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+	tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	/* Form a new tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	replaces[Anum_pg_subscription_sublastusedid - 1] = true;
+	values[Anum_pg_subscription_sublastusedid- 1] = Int64GetDatum(lastusedid);
+
+	/* Update the catalog */
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+	heap_freetuple(tup);
+
+	table_close(rel, NoLock);
+}
+
 /*
  * Convert text array to list of strings.
  *
@@ -228,7 +274,7 @@ textarray_to_stringlist(ArrayType *textarray)
  */
 void
 AddSubscriptionRelState(Oid subid, Oid relid, char state,
-						XLogRecPtr sublsn)
+						XLogRecPtr sublsn, char *relslotname, char *reloriginname)
 {
 	Relation	rel;
 	HeapTuple	tup;
@@ -257,6 +303,16 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
 	else
 		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -269,6 +325,60 @@ AddSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Internal function to modify columns for relation state update
+ */
+static void
+UpdateSubscriptionRelState_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char state,
+									XLogRecPtr sublsn)
+{
+	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
+	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+
+	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	if (sublsn != InvalidXLogRecPtr)
+		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
+	else
+		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication slot update
+ */
+static void
+UpdateSubscriptionRelReplicationSlot_internal(Datum *values,
+											bool *nulls,
+											bool *replaces,
+											char *relslotname)
+{
+	replaces[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+	if (relslotname)
+		values[Anum_pg_subscription_rel_srrelslotname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(relslotname));
+	else
+		nulls[Anum_pg_subscription_rel_srrelslotname - 1] = true;
+}
+
+/*
+ * Internal function to modify columns for replication origin update
+ */
+static void
+UpdateSubscriptionRelOrigin_internal(Datum *values,
+									bool *nulls,
+									bool *replaces,
+									char *reloriginname)
+{
+	replaces[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+	if (reloriginname)
+		values[Anum_pg_subscription_rel_srreloriginname - 1] =
+			DirectFunctionCall1(namein, CStringGetDatum(reloriginname));
+	else
+		nulls[Anum_pg_subscription_rel_srreloriginname - 1] = true;
+}
+
 /*
  * Update the state of a subscription table.
  */
@@ -299,14 +409,56 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	memset(nulls, false, sizeof(nulls));
 	memset(replaces, false, sizeof(replaces));
 
-	replaces[Anum_pg_subscription_rel_srsubstate - 1] = true;
-	values[Anum_pg_subscription_rel_srsubstate - 1] = CharGetDatum(state);
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
 
-	replaces[Anum_pg_subscription_rel_srsublsn - 1] = true;
-	if (sublsn != InvalidXLogRecPtr)
-		values[Anum_pg_subscription_rel_srsublsn - 1] = LSNGetDatum(sublsn);
-	else
-		nulls[Anum_pg_subscription_rel_srsublsn - 1] = true;
+	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+							replaces);
+
+	/* Update the catalog. */
+	CatalogTupleUpdate(rel, &tup->t_self, tup);
+
+	/* Cleanup. */
+	table_close(rel, NoLock);
+}
+
+/*
+ * Update replication slot name, origin name and state of
+ * a subscription table in one transaction.
+ */
+void
+UpdateSubscriptionRel(Oid subid,
+					  Oid relid,
+					  char state,
+					  XLogRecPtr sublsn,
+					  char *relslotname,
+					  char *reloriginname)
+{
+	Relation	rel;
+	HeapTuple	tup;
+	bool		nulls[Natts_pg_subscription_rel];
+	Datum		values[Natts_pg_subscription_rel];
+	bool		replaces[Natts_pg_subscription_rel];
+
+	LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+	rel = table_open(SubscriptionRelRelationId, RowExclusiveLock);
+
+	/* Try finding existing mapping. */
+	tup = SearchSysCacheCopy2(SUBSCRIPTIONRELMAP,
+							  ObjectIdGetDatum(relid),
+							  ObjectIdGetDatum(subid));
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "subscription table %u in subscription %u does not exist",
+			 relid, subid);
+
+	/* Update the tuple. */
+	memset(values, 0, sizeof(values));
+	memset(nulls, false, sizeof(nulls));
+	memset(replaces, false, sizeof(replaces));
+
+	UpdateSubscriptionRelState_internal(values, nulls, replaces, state, sublsn);
+	UpdateSubscriptionRelReplicationSlot_internal(values, nulls, replaces, relslotname);
+	UpdateSubscriptionRelOrigin_internal(values, nulls, replaces, reloriginname);
 
 	tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
 							replaces);
@@ -318,6 +470,85 @@ UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 	table_close(rel, NoLock);
 }
 
+/*
+ * Get origin name of subscription table.
+ *
+ * reloriginname's value has the replication origin name if the origin exists.
+ */
+void
+GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*originname;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srreloriginname, isnull);
+	if (!*isnull)
+	{
+		originname = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(reloriginname, originname, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
+/*
+ * Get replication slot name of subscription table.
+ *
+ * slotname's value has the replication slot name if the subscription has any.
+ */
+void
+GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname)
+{
+	HeapTuple	tup;
+	Relation	rel;
+	Datum 		d;
+	char		*relrepslot;
+	bool		isnull;
+
+	rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+	/* Try finding the mapping. */
+	tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
+						  ObjectIdGetDatum(relid),
+						  ObjectIdGetDatum(subid));
+
+	if (!HeapTupleIsValid(tup))
+	{
+		table_close(rel, AccessShareLock);
+	}
+
+	d = SysCacheGetAttr(SUBSCRIPTIONRELMAP, tup,
+						Anum_pg_subscription_rel_srrelslotname, &isnull);
+	if (!isnull)
+	{
+		relrepslot = DatumGetCString(DirectFunctionCall1(nameout, d));
+		memcpy(slotname, relrepslot, NAMEDATALEN);
+	}
+
+	/* Cleanup */
+	ReleaseSysCache(tup);
+
+	table_close(rel, AccessShareLock);
+}
+
 /*
  * Get state of subscription table.
  *
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index d4e798baeb..4f5b3c572c 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -710,6 +710,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 		publicationListToArray(publications);
 	values[Anum_pg_subscription_suborigin - 1] =
 		CStringGetTextDatum(opts.origin);
+	values[Anum_pg_subscription_sublastusedid - 1] = Int64GetDatum(0);
 
 	tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
 
@@ -773,7 +774,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 										 rv->schemaname, rv->relname);
 
 				AddSubscriptionRelState(subid, relid, table_state,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 			}
 
 			/*
@@ -864,6 +865,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 	SubRemoveRels *sub_remove_rels;
 	WalReceiverConn *wrconn;
 	bool		must_use_password;
+	List	   *sub_remove_slots = NIL;
+	LogicalRepWorker *worker;
 
 	/* Load the library providing us libpq calls. */
 	load_file("libpqwalreceiver", false);
@@ -943,7 +946,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				AddSubscriptionRelState(sub->oid, relid,
 										copy_data ? SUBREL_STATE_INIT : SUBREL_STATE_READY,
-										InvalidXLogRecPtr);
+										InvalidXLogRecPtr, NULL, NULL);
 				ereport(DEBUG1,
 						(errmsg_internal("table \"%s.%s\" added to subscription \"%s\"",
 										 rv->schemaname, rv->relname, sub->name)));
@@ -967,6 +970,7 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 			{
 				char		state;
 				XLogRecPtr	statelsn;
+				char		slotname[NAMEDATALEN] = {0};
 
 				/*
 				 * Lock pg_subscription_rel with AccessExclusiveLock to
@@ -993,13 +997,36 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 
 				RemoveSubscriptionRel(sub->oid, relid);
 
-				logicalrep_worker_stop(sub->oid, relid);
+				/*
+				 * Find the logical replication sync worker. If exists, store
+				 * the slot number for dropping associated replication slots
+				 * later.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+				worker = logicalrep_worker_find(sub->oid, relid, false);
+				if (worker)
+				{
+					logicalrep_worker_stop(sub->oid, relid);
+					sub_remove_slots = lappend(sub_remove_slots, &worker->slot_name);
+				}
+				else
+				{
+					/*
+					 * Sync of this relation might be failed in an earlier
+					 * attempt, but the replication slot might still exist.
+					 */
+					GetSubscriptionRelReplicationSlot(sub->oid, relid, slotname);
+					if (strlen(slotname) > 0)
+						sub_remove_slots = lappend(sub_remove_slots, slotname);
+				}
+				LWLockRelease(LogicalRepWorkerLock);
 
 				/*
 				 * For READY state, we would have already dropped the
 				 * tablesync origin.
 				 */
-				if (state != SUBREL_STATE_READY)
+				if (state != SUBREL_STATE_READY &&
+					state != SUBREL_STATE_SYNCDONE)
 				{
 					char		originname[NAMEDATALEN];
 
@@ -1027,31 +1054,24 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		}
 
 		/*
-		 * Drop the tablesync slots associated with removed tables. This has
-		 * to be at the end because otherwise if there is an error while doing
-		 * the database operations we won't be able to rollback dropped slots.
+		 * Drop the replication slots associated with tablesync workers for
+		 * removed tables. This has to be at the end because otherwise if
+		 * there is an error while doing the database operations we won't be
+		 * able to rollback dropped slots.
 		 */
-		for (off = 0; off < remove_rel_len; off++)
+		foreach(lc, sub_remove_slots)
 		{
-			if (sub_remove_rels[off].state != SUBREL_STATE_READY &&
-				sub_remove_rels[off].state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+			char		syncslotname[NAMEDATALEN] = {0};
 
-				/*
-				 * For READY/SYNCDONE states we know the tablesync slot has
-				 * already been dropped by the tablesync worker.
-				 *
-				 * For other states, there is no certainty, maybe the slot
-				 * does not exist yet. Also, if we fail after removing some of
-				 * the slots, next time, it will again try to drop already
-				 * dropped slots and fail. For these reasons, we allow
-				 * missing_ok = true for the drop.
-				 */
-				ReplicationSlotNameForTablesync(sub->oid, sub_remove_rels[off].relid,
-												syncslotname, sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			memcpy(syncslotname, lfirst(lc), sizeof(NAMEDATALEN));
+
+			/*
+			 * There is no certainty, maybe the slot does not exist yet. Also,
+			 * if we fail after removing some of the slots, next time, it will
+			 * again try to drop already dropped slots and fail. For these
+			 * reasons, we allow missing_ok = true for the drop.
+			 */
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 	}
 	PG_FINALLY();
@@ -1474,6 +1494,7 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	char	   *subname;
 	char	   *conninfo;
 	char	   *slotname;
+	int64		lastusedid;
 	List	   *subworkers;
 	ListCell   *lc;
 	char		originname[NAMEDATALEN];
@@ -1546,6 +1567,14 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	else
 		slotname = NULL;
 
+	/* Get the last used identifier by the subscription */
+	datum = SysCacheGetAttr(SUBSCRIPTIONOID, tup,
+							Anum_pg_subscription_sublastusedid, &isnull);
+	if (!isnull)
+		lastusedid = DatumGetInt64(datum);
+	else
+		lastusedid = 0;
+
 	/*
 	 * Since dropping a replication slot is not transactional, the replication
 	 * slot stays dropped even if the transaction rolls back.  So we cannot
@@ -1595,6 +1624,8 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	}
 	list_free(subworkers);
 
+	rstates = GetSubscriptionRelations(subid, true);
+
 	/*
 	 * Remove the no-longer-useful entry in the launcher's table of apply
 	 * worker start times.
@@ -1606,36 +1637,26 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	ApplyLauncherForgetWorkerStartTime(subid);
 
 	/*
-	 * Cleanup of tablesync replication origins.
-	 *
-	 * Any READY-state relations would already have dealt with clean-ups.
+	 * Cleanup of tablesync replication origins associated with the
+	 * subscription, if exists. Try to drop origins by creating all origin
+	 * names created for this subscription.
 	 *
 	 * Note that the state can't change because we have already stopped both
 	 * the apply and tablesync workers and they can't restart because of
 	 * exclusive lock on the subscription.
+	 *
+	 * XXX: This can be handled better instead of looping through all possible
 	 */
-	rstates = GetSubscriptionRelations(subid, true);
-	foreach(lc, rstates)
+	for (int64 i = 1; i <= lastusedid; i++)
 	{
-		SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-		Oid			relid = rstate->relid;
-
-		/* Only cleanup resources of tablesync workers */
-		if (!OidIsValid(relid))
-			continue;
+		char		originname_to_drop[NAMEDATALEN] = {0};
 
-		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * It is possible that the origin is not yet created for tablesync
-		 * worker so passing missing_ok = true. This can happen for the states
-		 * before SUBREL_STATE_FINISHEDCOPY.
-		 */
-		ReplicationOriginNameForLogicalRep(subid, relid, originname,
-										   sizeof(originname));
-		replorigin_drop_by_name(originname, true, false);
+		snprintf(originname_to_drop, sizeof(originname_to_drop), "pg_%u_%lld", subid, (long long) i);
+		/* missing_ok = true, since the origin might be already dropped. */
+		replorigin_drop_by_name(originname_to_drop, true, false);
 	}
 
+
 	/* Clean up dependencies */
 	deleteSharedDependencyRecordsFor(SubscriptionRelationId, subid, 0);
 
@@ -1694,39 +1715,17 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 
 	PG_TRY();
 	{
-		foreach(lc, rstates)
-		{
-			SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-			Oid			relid = rstate->relid;
+		List	   *slots = NULL;
 
-			/* Only cleanup resources of tablesync workers */
-			if (!OidIsValid(relid))
-				continue;
 
-			/*
-			 * Drop the tablesync slots associated with removed tables.
-			 *
-			 * For SYNCDONE/READY states, the tablesync slot is known to have
-			 * already been dropped by the tablesync worker.
-			 *
-			 * For other states, there is no certainty, maybe the slot does
-			 * not exist yet. Also, if we fail after removing some of the
-			 * slots, next time, it will again try to drop already dropped
-			 * slots and fail. For these reasons, we allow missing_ok = true
-			 * for the drop.
-			 */
-			if (rstate->state != SUBREL_STATE_SYNCDONE)
-			{
-				char		syncslotname[NAMEDATALEN] = {0};
+		slots = GetReplicationSlotNamesBySubId(wrconn, subid, true);
+		foreach(lc, slots)
+		{
+			char	   *syncslotname = (char *) lfirst(lc);
 
-				ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-												sizeof(syncslotname));
-				ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
-			}
+			ReplicationSlotDropAtPubNode(wrconn, syncslotname, true);
 		}
 
-		list_free(rstates);
-
 		/*
 		 * If there is a slot associated with the subscription, then drop the
 		 * replication slot at the publisher.
@@ -1743,6 +1742,71 @@ DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel)
 	table_close(rel, NoLock);
 }
 
+/*
+ * GetReplicationSlotNamesBySubId
+ *
+ * Get the replication slot names associated with the subscription.
+ */
+List *
+GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok)
+{
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {NAMEOID};
+	List	   *tablelist = NIL;
+
+	Assert(wrconn);
+
+	load_file("libpqwalreceiver", false);
+
+	initStringInfo(&cmd);
+	appendStringInfo(&cmd, "SELECT slot_name"
+					 " FROM pg_replication_slots"
+					 " WHERE slot_name LIKE 'pg_%i_sync_%%';",
+					 subid);
+	PG_TRY();
+	{
+		WalRcvExecResult *res;
+
+		res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+
+		if (res->status != WALRCV_OK_TUPLES)
+		{
+			ereport(ERROR,
+					errmsg("could not receive list of slots associated with the subscription %u, error: %s",
+					subid, res->err));
+		}
+
+		/* Process tables. */
+		slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+		while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+		{
+			char	   *repslotname;
+			char	   *slotattr;
+			bool		isnull;
+
+			slotattr = NameStr(*DatumGetName(slot_getattr(slot, 1, &isnull)));
+			Assert(!isnull);
+
+			repslotname = palloc(sizeof(char) * strlen(slotattr) + 1);
+			memcpy(repslotname, slotattr, sizeof(char) * strlen(slotattr));
+			repslotname[strlen(slotattr)] = '\0';
+			tablelist = lappend(tablelist, repslotname);
+
+			ExecClearTuple(slot);
+		}
+		ExecDropSingleTupleTableSlot(slot);
+
+		walrcv_clear_result(res);
+	}
+	PG_FINALLY();
+	{
+		pfree(cmd.data);
+	}
+	PG_END_TRY();
+		return tablelist;
+}
+
 /*
  * Drop the replication slot at the publisher node using the replication
  * connection.
@@ -2155,6 +2219,7 @@ static void
 ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 {
 	ListCell   *lc;
+	LogicalRepWorker *worker;
 
 	foreach(lc, rstates)
 	{
@@ -2165,18 +2230,20 @@ ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err)
 		if (!OidIsValid(relid))
 			continue;
 
+		/* Check if there is a sync worker for the relation */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		worker = logicalrep_worker_find(subid, relid, false);
+		LWLockRelease(LogicalRepWorkerLock);
+
 		/*
 		 * Caller needs to ensure that relstate doesn't change underneath us.
 		 * See DropSubscription where we get the relstates.
 		 */
-		if (rstate->state != SUBREL_STATE_SYNCDONE)
+		if (worker &&
+			rstate->state != SUBREL_STATE_SYNCDONE)
 		{
-			char		syncslotname[NAMEDATALEN] = {0};
-
-			ReplicationSlotNameForTablesync(subid, relid, syncslotname,
-											sizeof(syncslotname));
 			elog(WARNING, "could not drop tablesync replication slot \"%s\"",
-				 syncslotname);
+				 worker->slot_name);
 		}
 	}
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 1d4e83c4c1..d2c70dffbc 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -440,7 +440,8 @@ pa_launch_parallel_worker(void)
 										MySubscription->name,
 										MyLogicalRepWorker->userid,
 										InvalidOid,
-										dsm_segment_handle(winfo->dsm_seg));
+										dsm_segment_handle(winfo->dsm_seg),
+										InvalidRepSlotId);
 
 	if (launched)
 	{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 945619b603..9f378c311e 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -304,7 +304,7 @@ logicalrep_workers_find(Oid subid, bool only_running)
  */
 bool
 logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
-						 Oid relid, dsm_handle subworker_dsm)
+						 Oid relid, dsm_handle subworker_dsm, int64 slotid)
 {
 	BackgroundWorker bgw;
 	BackgroundWorkerHandle *bgw_handle;
@@ -430,6 +430,9 @@ retry:
 	worker->launch_time = now;
 	worker->in_use = true;
 	worker->generation++;
+	worker->created_slot = false;
+	worker->rep_slot_id = slotid;
+	worker->slot_name = (char *) palloc(NAMEDATALEN);
 	worker->proc = NULL;
 	worker->dbid = dbid;
 	worker->userid = userid;
@@ -1184,7 +1187,8 @@ ApplyLauncherMain(Datum main_arg)
 				ApplyLauncherSetWorkerStartTime(sub->oid, now);
 				logicalrep_worker_launch(sub->dbid, sub->oid, sub->name,
 										 sub->owner, InvalidOid,
-										 DSM_HANDLE_INVALID);
+										 DSM_HANDLE_INVALID,
+										 InvalidRepSlotId);
 			}
 			else
 			{
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 73b6fd77e2..bfb201ce69 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -317,40 +317,29 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 								   MyLogicalRepWorker->relid,
 								   MyLogicalRepWorker->relstate,
 								   MyLogicalRepWorker->relstate_lsn);
+		CommitTransactionCommand();
 
 		/*
-		 * End streaming so that LogRepWorkerWalRcvConn can be used to drop
-		 * the slot.
-		 */
-		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
-
-		/*
-		 * Cleanup the tablesync slot.
+		 * Cleanup the tablesync slot. If the slot name used by this worker is
+		 * different from the default slot name for the worker, this means the
+		 * current table had started to being synchronized by another worker
+		 * and replication slot. And this worker is reusing a replication slot
+		 * from a previous attempt. We do not need that replication slot
+		 * anymore.
 		 *
 		 * This has to be done after updating the state because otherwise if
 		 * there is an error while doing the database operations we won't be
 		 * able to rollback dropped slot.
 		 */
 		ReplicationSlotNameForTablesync(MyLogicalRepWorker->subid,
-										MyLogicalRepWorker->relid,
+										MyLogicalRepWorker->rep_slot_id,
 										syncslotname,
 										sizeof(syncslotname));
 
 		/*
-		 * It is important to give an error if we are unable to drop the slot,
-		 * otherwise, it won't be dropped till the corresponding subscription
-		 * is dropped. So passing missing_ok = false.
-		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
-
-		CommitTransactionCommand();
-		pgstat_report_stat(false);
-
-		/*
-		 * Start a new transaction to clean up the tablesync origin tracking.
-		 * This transaction will be ended within the finish_sync_worker().
-		 * Now, even, if we fail to remove this here, the apply worker will
-		 * ensure to clean it up afterward.
+		 * We are safe to drop the replication tracking origin after this
+		 * point. Now, even, if we fail to remove this here, the apply worker
+		 * will ensure to clean it up afterward.
 		 *
 		 * We need to do this after the table state is set to SYNCDONE.
 		 * Otherwise, if an error occurs while performing the database
@@ -359,32 +348,73 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 * have been cleared before restart. So, the restarted worker will use
 		 * invalid replication progress state resulting in replay of
 		 * transactions that have already been applied.
+		 *
+		 * Firstly reset the origin session to remove the ownership of the
+		 * slot. This is needed to allow the origin to be dropped or reused
+		 * later.
 		 */
+		replorigin_session_reset();
+		replorigin_session_origin = InvalidRepOriginId;
+		replorigin_session_origin_lsn = InvalidXLogRecPtr;
+		replorigin_session_origin_timestamp = 0;
+
 		StartTransactionCommand();
+		if (MyLogicalRepWorker->slot_name && strcmp(syncslotname, MyLogicalRepWorker->slot_name) != 0)
+		{
+			/*
+			 * End streaming so that LogRepWorkerWalRcvConn can be used to
+			 * drop the slot.
+			 */
+			walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
 
-		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
+			ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+											   MyLogicalRepWorker->relid,
+											   originname,
+											   sizeof(originname));
+
+			/*
+			 * Drop replication origin
+			 *
+			 * There is a chance that the user is concurrently performing refresh
+			 * for the subscription where we remove the table state and its origin
+			 * or the apply worker would have removed this origin. So passing
+			 * missing_ok = true.
+			 */
+			replorigin_drop_by_name(originname, true, false);
+		}
 
 		/*
-		 * Resetting the origin session removes the ownership of the slot.
-		 * This is needed to allow the origin to be dropped.
+		 * We are safe to remove persisted replication slot and origin data,
+		 * since it's already in SYNCDONE state. They will not be needed
+		 * anymore.
 		 */
-		replorigin_session_reset();
-		replorigin_session_origin = InvalidRepOriginId;
-		replorigin_session_origin_lsn = InvalidXLogRecPtr;
-		replorigin_session_origin_timestamp = 0;
+		UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+							  MyLogicalRepWorker->relid,
+							  MyLogicalRepWorker->relstate,
+							  MyLogicalRepWorker->relstate_lsn,
+							  NULL,
+							  NULL);
+		ereport(DEBUG2,
+				(errmsg("process_syncing_tables_for_sync: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+						"NULL",
+						"NULL",
+						MyLogicalRepWorker->relstate,
+						MyLogicalRepWorker->relid,
+						MyLogicalRepWorker->subid)));
+		CommitTransactionCommand();
+		pgstat_report_stat(false);
 
 		/*
-		 * Drop the tablesync's origin tracking if exists.
-		 *
-		 * There is a chance that the user is concurrently performing refresh
-		 * for the subscription where we remove the table state and its origin
-		 * or the apply worker would have removed this origin. So passing
-		 * missing_ok = true.
+		 * This should return the default origin name for the worker. Even if
+		 * the worker used a different origin for this table, it should be
+		 * dropped and removed from the catalog so far.
 		 */
-		replorigin_drop_by_name(originname, true, false);
+		StartTransactionCommand();
+		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+										   MyLogicalRepWorker->relid,
+										   originname,
+										   sizeof(originname));
 
 		/* Sync worker has completed synchronization of the current table. */
 		MyLogicalRepWorker->is_sync_completed = true;
@@ -481,6 +511,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 			if (current_lsn >= rstate->lsn)
 			{
 				char		originname[NAMEDATALEN];
+				bool		is_origin_null = true;
 
 				rstate->state = SUBREL_STATE_READY;
 				rstate->lsn = current_lsn;
@@ -501,18 +532,31 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				 * error while dropping we won't restart it to drop the
 				 * origin. So passing missing_ok = true.
 				 */
-				ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-												   rstate->relid,
-												   originname,
-												   sizeof(originname));
-				replorigin_drop_by_name(originname, true, false);
+				GetSubscriptionRelOrigin(MyLogicalRepWorker->subid,
+										 rstate->relid, originname,
+										 &is_origin_null);
+
+				if (!is_origin_null)
+				{
+					replorigin_drop_by_name(originname, true, false);
+				}
 
 				/*
 				 * Update the state to READY only after the origin cleanup.
 				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-										   rstate->relid, rstate->state,
-										   rstate->lsn);
+				UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+									  rstate->relid,
+									  rstate->state,
+									  rstate->lsn,
+									  NULL,
+									  NULL);
+				ereport(DEBUG2,
+					(errmsg("process_syncing_tables_for_apply: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+							"NULL", "NULL", rstate->state,
+							rstate->relid, MyLogicalRepWorker->subid)));
+
+				CommitTransactionCommand();
+				started_tx = false;
 			}
 		}
 		else
@@ -601,12 +645,25 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						TimestampDifferenceExceeds(hentry->last_start_time, now,
 												   wal_retrieve_retry_interval))
 					{
+						if (IsTransactionState())
+							CommitTransactionCommand();
+						StartTransactionCommand();
+						started_tx = true;
+
+						MySubscription->lastusedid++;
+						UpdateSubscriptionLastSlotId(MyLogicalRepWorker->subid,
+													 MySubscription->lastusedid);
+						ereport(DEBUG2,
+								(errmsg("process_syncing_tables_for_apply: incremented lastusedid to %lld for subscription %u",
+										(long long) MySubscription->lastusedid, MySubscription->oid)));
+
 						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
 												 MySubscription->oid,
 												 MySubscription->name,
 												 MyLogicalRepWorker->userid,
 												 rstate->relid,
-												 DSM_HANDLE_INVALID);
+												 DSM_HANDLE_INVALID,
+												 MySubscription->lastusedid);
 						hentry->last_start_time = now;
 					}
 				}
@@ -1229,8 +1286,8 @@ copy_table(Relation rel)
  * The name must not exceed NAMEDATALEN - 1 because of remote node constraints
  * on slot name length. We append system_identifier to avoid slot_name
  * collision with subscriptions in other clusters. With the current scheme
- * pg_%u_sync_%u_UINT64_FORMAT (3 + 10 + 6 + 10 + 20 + '\0'), the maximum
- * length of slot_name will be 50.
+ * pg_%u_sync_%lu_UINT64_FORMAT (3 + 10 + 6 + 20 + 20 + '\0'), the maximum
+ * length of slot_name will be 45.
  *
  * The returned slot name is stored in the supplied buffer (syncslotname) with
  * the given size.
@@ -1241,11 +1298,11 @@ copy_table(Relation rel)
  * had changed.
  */
 void
-ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
+ReplicationSlotNameForTablesync(Oid suboid, int64 slotid,
 								char *syncslotname, Size szslot)
 {
-	snprintf(syncslotname, szslot, "pg_%u_sync_%u_" UINT64_FORMAT, suboid,
-			 relid, GetSystemIdentifier());
+	snprintf(syncslotname, szslot, "pg_%u_sync_%lld_" UINT64_FORMAT, suboid,
+			(long long) slotid, GetSystemIdentifier());
 }
 
 /*
@@ -1289,6 +1346,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	UserContext ucxt;
 	bool		must_use_password;
 	bool		run_as_owner;
+	char	   *prev_slotname;
 
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
@@ -1323,7 +1381,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	/* Calculate the name of the tablesync slot. */
 	slotname = (char *) palloc(NAMEDATALEN);
 	ReplicationSlotNameForTablesync(MySubscription->oid,
-									MyLogicalRepWorker->relid,
+									MyLogicalRepWorker->rep_slot_id,
 									slotname,
 									NAMEDATALEN);
 
@@ -1355,12 +1413,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC ||
 		   MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY);
 
+	/*
+	 * See if tablesync of the current relation has been started with another
+	 * replication slot.
+	 *
+	 * Read previous slot name from the catalog, if exists.
+	 */
+	prev_slotname = (char *) palloc(NAMEDATALEN);
+	StartTransactionCommand();
+	GetSubscriptionRelReplicationSlot(MyLogicalRepWorker->subid,
+									  MyLogicalRepWorker->relid,
+									  prev_slotname);
+
 	/* Assign the origin tracking record name. */
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   sizeof(originname));
 
+	CommitTransactionCommand();
+
 	if (MyLogicalRepWorker->relstate == SUBREL_STATE_DATASYNC)
 	{
 		/*
@@ -1374,10 +1446,53 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		 * breakdown then it wouldn't have succeeded so trying it next time
 		 * seems like a better bet.
 		 */
-		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, slotname, true);
+		if (strlen(prev_slotname) > 0)
+		{
+			ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, prev_slotname, true);
+
+			StartTransactionCommand();
+			/* Replication origin might still exist. Try to drop */
+			replorigin_drop_by_name(originname, true, false);
+
+			/*
+			 * Remove replication slot and origin name from the relation's
+			 * catalog record
+			 */
+			UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+								  MyLogicalRepWorker->relid,
+								  MyLogicalRepWorker->relstate,
+								  MyLogicalRepWorker->relstate_lsn,
+								  NULL,
+								  NULL);
+			CommitTransactionCommand();
+			ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+						"NULL", "NULL", MyLogicalRepWorker->relstate,
+						MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+		}
 	}
 	else if (MyLogicalRepWorker->relstate == SUBREL_STATE_FINISHEDCOPY)
 	{
+		/*
+		 * At this point, the table that is currently being synchronized
+		 * should have its replication slot name filled in the catalog. The
+		 * tablesync process was started with another sync worker and
+		 * replication slot. We need to continue using the same replication
+		 * slot in this worker too.
+		 */
+		if (strlen(prev_slotname) == 0)
+		{
+			elog(ERROR, "Replication slot could not be found for subscription %u, relation %u",
+				 MyLogicalRepWorker->subid,
+				 MyLogicalRepWorker->relid);
+		}
+
+		/*
+		 * Proceed with the correct replication slot. Use previously created
+		 * replication slot to sync this table.
+		 */
+		memcpy(slotname, prev_slotname, NAMEDATALEN);
+
 		/*
 		 * The COPY phase was previously done, but tablesync then crashed
 		 * before it was able to finish normally.
@@ -1397,7 +1512,9 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 		goto copy_table_done;
 	}
+	pfree(prev_slotname);
 
+	/* Preparing for table copy operation */
 	SpinLockAcquire(&MyLogicalRepWorker->relmutex);
 	MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
 	MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
@@ -1405,11 +1522,31 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   MyLogicalRepWorker->relstate,
-							   MyLogicalRepWorker->relstate_lsn);
+
+	/*
+	 * Refresh the originname in case of having non-existing origin
+	 * from previous failed sync attempts.
+	 * If that's the case, it should be removed from the catalog so far.
+	 * Then, we can continue by reusing the origin created by the current
+	 * worker instead of .
+	 */
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									MyLogicalRepWorker->relid,
+									originname,
+									sizeof(originname));
+
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  MyLogicalRepWorker->relstate,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					slotname, originname, MyLogicalRepWorker->relstate,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
+
 	pgstat_report_stat(true);
 
 	StartTransactionCommand();
@@ -1437,48 +1574,96 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 						res->err)));
 	walrcv_clear_result(res);
 
+	originid = replorigin_by_name(originname, true);
+
 	/*
 	 * Create a new permanent logical decoding slot. This slot will be used
 	 * for the catchup phase after COPY is done, so tell it to use the
 	 * snapshot to make the final data consistent.
+	 *
+	 * Replication slot will only be created if either this is the first run
+	 * of the worker or we're not using a previous replication slot.
 	 */
-	walrcv_create_slot(LogRepWorkerWalRcvConn,
-					   slotname, false /* permanent */ , false /* two_phase */ ,
-					   CRS_USE_SNAPSHOT, origin_startpos);
-
-	/*
-	 * Setup replication origin tracking. The purpose of doing this before the
-	 * copy is to avoid doing the copy again due to any error in setting up
-	 * origin tracking.
-	 */
-	originid = replorigin_by_name(originname, true);
-	if (!OidIsValid(originid))
+	if (!MyLogicalRepWorker->created_slot)
 	{
+		walrcv_create_slot(LogRepWorkerWalRcvConn,
+						   slotname, false /* permanent */ , false /* two_phase */ ,
+						   CRS_USE_SNAPSHOT, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: created replication slot %s for subscription %u",
+						slotname, MyLogicalRepWorker->subid)));
+
 		/*
-		 * Origin tracking does not exist, so create it now.
-		 *
-		 * Then advance to the LSN got from walrcv_create_slot. This is WAL
-		 * logged for the purpose of recovery. Locks are to prevent the
-		 * replication origin from vanishing while advancing.
+		 * Remember that we created the slot so that we will not try to create
+		 * it again.
 		 */
-		originid = replorigin_create(originname);
-
-		LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
-		replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
-						   true /* go backward */ , true /* WAL log */ );
-		UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+		SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+		MyLogicalRepWorker->created_slot = true;
+		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
+		/*
+		 * Setup replication origin tracking. The purpose of doing this before
+		 * the copy is to avoid doing the copy again due to any error in
+		 * setting up origin tracking.
+		 */
+		if (!OidIsValid(originid))
+		{
+			/*
+			 * Origin tracking does not exist, so create it now.
+			 */
+			originid = replorigin_create(originname);
+		}
+		else
+		{
+			/*
+			 * At this point, there shouldn't be any existing replication
+			 * origin with the same name.
+			 */
+			ereport(ERROR,
+					(errcode(ERRCODE_DUPLICATE_OBJECT),
+					 errmsg("replication origin \"%s\" already exists",
+							originname)));
+		}
 	}
 	else
 	{
-		ereport(ERROR,
-				(errcode(ERRCODE_DUPLICATE_OBJECT),
-				 errmsg("replication origin \"%s\" already exists",
-						originname)));
+		/*
+		 * Do not create a new replication slot, reuse the existing one
+		 * instead. Use a new snapshot for the replication slot to ensure that
+		 * tablesync and apply proceses are consistent with each other.
+		 */
+		WalRcvStreamOptions options;
+		int			server_version;
+
+		server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+		options.proto.logical.proto_version =
+			server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+			server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+			LOGICALREP_PROTO_VERSION_NUM;
+		options.proto.logical.publication_names = MySubscription->publications;
+
+		walrcv_slot_snapshot(LogRepWorkerWalRcvConn, slotname, &options, origin_startpos);
+		ereport(DEBUG2,
+				(errmsg("LogicalRepSyncTableStart: reusing replication slot %s for relation %u in subscription %u",
+						slotname, MyLogicalRepWorker->relid,
+						MyLogicalRepWorker->subid)));
 	}
 
+	/*
+	 * Advance to the LSN got from walrcv_create_slot or walrcv_slot_snapshot.
+	 * This is WAL logged for the purpose of recovery. Locks are to prevent
+	 * the replication origin from vanishing while advancing.
+	 *
+	 * Then setup replication origin tracking.
+	 */
+	LockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+	replorigin_advance(originid, *origin_startpos, InvalidXLogRecPtr,
+					   true /* go backward */ , true /* WAL log */ );
+	UnlockRelationOid(ReplicationOriginRelationId, RowExclusiveLock);
+
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+
 	/*
 	 * Make sure that the copy command runs as the table owner, unless the
 	 * user has opted out of that behaviour.
@@ -1537,12 +1722,18 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	 * Update the persisted state to indicate the COPY phase is done; make it
 	 * visible to others.
 	 */
-	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-							   MyLogicalRepWorker->relid,
-							   SUBREL_STATE_FINISHEDCOPY,
-							   MyLogicalRepWorker->relstate_lsn);
+	UpdateSubscriptionRel(MyLogicalRepWorker->subid,
+						  MyLogicalRepWorker->relid,
+						  SUBREL_STATE_FINISHEDCOPY,
+						  MyLogicalRepWorker->relstate_lsn,
+						  slotname,
+						  originname);
 
 	CommitTransactionCommand();
+	ereport(DEBUG2,
+			(errmsg("LogicalRepSyncTableStart: updated originname: %s, slotname: %s, state: %c for relation \"%u\" in subscription \"%u\".",
+					originname, slotname, SUBREL_STATE_FINISHEDCOPY,
+					MyLogicalRepWorker->relid, MyLogicalRepWorker->subid)));
 
 copy_table_done:
 
@@ -1668,6 +1859,9 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 
 	/* allocate slot name in long-lived context */
 	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+
+	/* Keep the replication slot name used for this sync. */
+	MyLogicalRepWorker->slot_name = *myslotname;
 	pfree(syncslotname);
 }
 
@@ -1686,13 +1880,25 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 {
 	MyLogicalRepWorker->is_sync_completed = false;
 
+	/*
+	 * If it's already connected to the publisher, end streaming before using
+	 * the same connection for another iteration
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		TimeLineID tli;
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	}
+
 	/* Start table synchronization. */
 	start_table_sync(origin_startpos, &slotname);
 
+	StartTransactionCommand();
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
 									   originname,
 									   originname_size);
+	CommitTransactionCommand();
 
 	set_apply_error_context_origin(originname);
 
@@ -1823,6 +2029,32 @@ TablesyncWorkerMain(Datum main_arg)
 				}
 				LWLockRelease(LogicalRepWorkerLock);
 			}
+
+			if (done)
+			{
+				TimeLineID	tli;
+
+				/*
+				 * It is important to give an error if we are unable to drop the
+				 * slot, otherwise, it won't be dropped till the corresponding
+				 * subscription is dropped. So passing missing_ok = false.
+				 */
+				if (MyLogicalRepWorker->created_slot)
+				{
+					walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+					ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, MyLogicalRepWorker->slot_name, false);
+				}
+
+				/*
+				 * Drop replication origin before exiting.
+				 *
+				 * There is a chance that the user is concurrently performing refresh
+				 * for the subscription where we remove the table state and its origin
+				 * or the apply worker would have removed this origin. So passing
+				 * missing_ok = true.
+				 */
+				replorigin_drop_by_name(originname, true, false);
+			}
 		}
 	}
 
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 7ba58d0e67..c10b3d0656 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -447,8 +447,16 @@ ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 {
 	if (OidIsValid(relid))
 	{
-		/* Replication origin name for tablesync workers. */
-		snprintf(originname, szoriginname, "pg_%u_%u", suboid, relid);
+		bool		is_null = true;
+
+		/*
+		 * Replication origin name for tablesync workers. First, look into the
+		 * catalog. If originname does not exist, then use the default name.
+		 */
+		GetSubscriptionRelOrigin(suboid, relid,
+								 originname, &is_null);
+		if (is_null)
+			snprintf(originname, szoriginname, "pg_%u_%lld", suboid, (long long) MyLogicalRepWorker->rep_slot_id);
 	}
 	else
 	{
@@ -4501,11 +4509,10 @@ run_apply_worker(WalRcvStreamOptions *options,
 				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
 					errmsg("subscription has no replication slot set")));
 
-	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-									   originname, originname_size);
-
 	/* Setup replication origin tracking. */
 	StartTransactionCommand();
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
 	originid = replorigin_by_name(originname, true);
 	if (!OidIsValid(originid))
 		originid = replorigin_create(originname);
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 1d40eebc78..7e13f59847 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -108,6 +108,9 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
 
 	/* Only publish data originating from the specified origin */
 	text		suborigin BKI_DEFAULT(LOGICALREP_ORIGIN_ANY);
+
+	/* The last used ID to create a replication slot for tablesync */
+	int64		sublastusedid BKI_DEFAULT(0);
 #endif
 } FormData_pg_subscription;
 
@@ -144,6 +147,8 @@ typedef struct Subscription
 	List	   *publications;	/* List of publication names to subscribe to */
 	char	   *origin;			/* Only publish data originating from the
 								 * specified origin */
+	int64		lastusedid;		/* Last used unique ID to create replication
+								 * slots in tablesync */
 } Subscription;
 
 /* Disallow streaming in-progress transactions. */
@@ -164,6 +169,7 @@ typedef struct Subscription
 extern Subscription *GetSubscription(Oid subid, bool missing_ok);
 extern void FreeSubscription(Subscription *sub);
 extern void DisableSubscription(Oid subid);
+extern void UpdateSubscriptionLastSlotId(Oid subid, int64 lastusedid);
 
 extern int	CountDBSubscriptions(Oid dbid);
 
diff --git a/src/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index 60a2bcca23..185164d75e 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -44,6 +44,12 @@ CATALOG(pg_subscription_rel,6102,SubscriptionRelRelationId)
 											 * used for synchronization
 											 * coordination, or NULL if not
 											 * valid */
+	NameData	srrelslotname BKI_FORCE_NULL;	/* name of the replication
+												 * slot for relation in
+												 * subscription */
+	NameData	srreloriginname BKI_FORCE_NULL; /* origin name for relation in
+												 * subscription */
+
 #endif
 } FormData_pg_subscription_rel;
 
@@ -81,10 +87,16 @@ typedef struct SubscriptionRelState
 } SubscriptionRelState;
 
 extern void AddSubscriptionRelState(Oid subid, Oid relid, char state,
-									XLogRecPtr sublsn);
+									XLogRecPtr sublsn, char *relslotname, char *reloriginname);
 extern void UpdateSubscriptionRelState(Oid subid, Oid relid, char state,
 									   XLogRecPtr sublsn);
+extern void UpdateSubscriptionRel(Oid subid, Oid relid, char state,
+								  XLogRecPtr sublsn, char *relslotname, char *reloriginname);
+
 extern char GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn);
+extern void GetSubscriptionRelReplicationSlot(Oid subid, Oid relid, char *slotname);
+extern void GetSubscriptionRelOrigin(Oid subid, Oid relid, char *reloriginname, bool *isnull);
+
 extern void RemoveSubscriptionRel(Oid subid, Oid relid);
 
 extern bool HasSubscriptionRelations(Oid subid);
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index a8a89dc784..31b2c41893 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -237,8 +237,9 @@ extern bool InvalidateObsoleteReplicationSlots(ReplicationSlotInvalidationCause
 extern ReplicationSlot *SearchNamedReplicationSlot(const char *name, bool need_lock);
 extern int	ReplicationSlotIndex(ReplicationSlot *slot);
 extern bool ReplicationSlotName(int index, Name name);
-extern void ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname, Size szslot);
+extern void ReplicationSlotNameForTablesync(Oid suboid, int64 slotid, char *syncslotname, Size szslot);
 extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
+extern List *GetReplicationSlotNamesBySubId(WalReceiverConn *wrconn, Oid subid, bool missing_ok);
 
 extern void StartupReplicationSlots(void);
 extern void CheckPointReplicationSlots(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 804c475746..92976f9f90 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -36,6 +36,23 @@ typedef struct LogicalRepWorker
 	/* Indicates if this slot is used or free. */
 	bool		in_use;
 
+	/*
+	 * Indicates if the sync worker created a replication slot for itself
+	 * in any point of its lifetime.
+	 * False means that the worker has not created a slot yet, and has been
+	 * reusing replication slots created by other workers so far.
+	 */
+	bool		created_slot;
+
+	/*
+	 * Unique identifier for replication slot to be created by tablesnync
+	 * workers, if needed.
+	 */
+	int64		rep_slot_id;
+
+	/* Replication slot name used by the worker. */
+	char	   *slot_name;
+
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
@@ -243,7 +260,8 @@ extern LogicalRepWorker *logicalrep_worker_find(Oid subid, Oid relid,
 extern List *logicalrep_workers_find(Oid subid, bool only_running);
 extern bool logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname,
 									 Oid userid, Oid relid,
-									 dsm_handle subworker_dsm);
+									 dsm_handle subworker_dsm,
+									 int64 slotid);
 extern void logicalrep_worker_stop(Oid subid, Oid relid);
 extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
 extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
@@ -343,4 +361,7 @@ extern void set_stream_options(WalRcvStreamOptions *options,
 extern void start_apply(XLogRecPtr origin_startpos);
 extern void DisableSubscriptionAndExit(void);
 
+/* Invalid identifier to be used for naming replication slots */
+#define InvalidRepSlotId	0
+
 #endif							/* WORKER_INTERNAL_H */
-- 
2.25.1

#105Amit Kapila
amit.kapila16@gmail.com
In reply to: Melih Mutlu (#104)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Tue, Jul 18, 2023 at 2:33 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Attached the fixed patchset.

Few comments on 0001
====================
1.
+ logicalrep_worker_attach(worker_slot);
+
+ /* Setup signal handling */
+ pqsignal(SIGHUP, SignalHandlerForConfigReload);
+ pqsignal(SIGTERM, die);
+ BackgroundWorkerUnblockSignals();
+
+ /*
+ * We don't currently need any ResourceOwner in a walreceiver process, but
+ * if we did, we could call CreateAuxProcessResourceOwner here.
+ */
+
+ /* Initialise stats to a sanish value */
+ MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
+ MyLogicalRepWorker->reply_time = GetCurrentTimestamp();
+
+ /* Load the libpq-specific functions */
+ load_file("libpqwalreceiver", false);
+
+ InitializeLogRepWorker();
+
+ /* Connect to the origin and start the replication. */
+ elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
+ MySubscription->conninfo);
+
+ /*
+ * Setup callback for syscache so that we know when something changes in
+ * the subscription relation state.
+ */
+ CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
+   invalidate_syncing_table_states,
+   (Datum) 0);

It seems this part of the code is the same for ApplyWorkerMain() and
TablesyncWorkerMain(). So, won't it be better to move it into a common
function?

2. Can LogicalRepSyncTableStart() be static function?

3. I think you don't need to send 0004, 0005 each time till we are
able to finish patches till 0003.

4. In 0001's commit message, you can say that it will help the
upcoming reuse tablesync worker patch.

--
With Regards,
Amit Kapila.

#106vignesh C
vignesh21@gmail.com
In reply to: Peter Smith (#92)
5 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Tue, 11 Jul 2023 at 08:30, Peter Smith <smithpb2250@gmail.com> wrote:

On Tue, Jul 11, 2023 at 12:31 AM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi,

Hayato Kuroda (Fujitsu) <kuroda.hayato@fujitsu.com>, 6 Tem 2023 Per,
12:47 tarihinde şunu yazdı:

Dear Melih,

Thanks for the 0003 patch. But it did not work for me. Can you create
a subscription successfully with patch 0003 applied?
I get the following error: " ERROR: table copy could not start
transaction on publisher: another command is already in progress".

You got the ERROR when all the patches (0001-0005) were applied, right?
I have focused on 0001 and 0002 only, so I missed something.
If it was not correct, please attach the logfile and test script what you did.

Yes, I did get an error with all patches applied. But with only 0001
and 0002, your version seems like working and mine does not.
What do you think about combining 0002 and 0003? Or should those stay separate?

Even if patches 0003 and 0002 are to be combined, I think that should
not happen until after the "reuse" design is confirmed which way is
best.

e.g. IMO it might be easier to compare the different PoC designs for
patch 0002 if there is no extra logic involved.

PoC design#1 -- each tablesync decides for itself what to do next
after it finishes
PoC design#2 -- reuse tablesync using a "pool" of available workers

I did a POC for design#2 for implementing a worker pool to synchronize
the tables for a subscriber. The core design is the same as what Melih
had implemented at [1]/messages/by-id/CAGPVpCSk4v-V1WbFDy8a5dL7Es5z8da6hoQbuVyrqP5s3Yh6Cg@mail.gmail.com. I had already started the implementation of
POC based on one of the earlier e-mail [2]/messages/by-id/CAHut+Ps8gWP9tCPK9gdMnxyshRKgVP3pJnAnaJto_T07uR9xUA@mail.gmail.com Peter had shared.
The POC has been implemented like:
a) Apply worker will check the tablesync pool and see if any tablesync
worker is free:
i) If there are no free workers in the pool, start a table sync
worker and add it to the table sync pool.
ii) If there are free workers in the pool, re-use the tablesync
worker for synchronizing another table.
b) Apply worker will check if the tables are synchronized, if all the
tables are synchronized apply worker will release all the workers from
the tablesync pool
c) Apply worker and tablesync worker has shared memory to share the
following relation data and execution state between the apply worker
and the tablesync worker
d) The apply worker and tablesync worker's pid are also stored in the
shared memory so that we need not take a lock on LogicalRepWorkerLock
and loop on max_logical_replication_workers every time. We use the pid
stored in shared memory to wake up the apply worker and tablesync
worker whenever needed.

While I was implementing the POC I found one issue in the POC
patch(there is no problem with the HEAD code, issue was only with the
POC):
1) Apply worker was waiting for the table to be set to SYNCDONE.
2) Mean time tablesync worker sets the table to SYNCDONE and sets
apply worker's latch.
3) Apply worker will reset the latch set by tablesync and go to main
loop and wait in main loop latch(since tablesync worker's latch was
already reset, apply worker will wait for 1 second)
To fix this I had to set apply worker's latch once in 1ms in this case
alone which is not a good solution as it will consume a lot of cpu
cycles. A better fix for this would be to introduce a new subscription
relation state.

Attached patch has the changes for the same. 001, 0002 and 0003 are
the patches shared by Melih and Kuroda-san earlier. 0004 patch has the
changes for the POC of Tablesync worker pool implementation.
POC design 1: Tablesync worker identifies the tables that should be
synced and reuses the connection.
POC design 2: Tablesync worker pool with apply worker scheduling the
work to tablesync workers in the tablesync pool and reusing the
connection.
Performance results for 10 empty tables:
+-------------------+--------------------+--------------------+----------------------+----------------+
|                        | 2 sync workers | 4 sync workers | 8 sync
workers  | 16 sync workers|
+-------------------+--------------------+--------------------+----------------------+----------------+
| HEAD              | 128.4685 ms    | 121.271 ms     | 136.5455 ms
 |   N/A          |
+-------------------+--------------------+--------------------+----------------------+----------------+
| POC design#1|  70.7095 ms    |  80.9805 ms    | 102.773  ms       |
 N/A          |
+-------------------+--------------------+--------------------+----------------------+----------------+
| POC design#2|  70.858 ms      |  83.0845 ms    | 112.505 ms        |
  N/A          |
+-------------------+--------------------+--------------------+----------------------+----------------+
Performance results for 100 empty tables:
+-------------------+--------------------+--------------------+----------------------+----------------+
|                         | 2 sync workers | 4 sync workers | 8 sync
workers | 16 sync workers|
+-------------------+--------------------+--------------------+----------------------+----------------+
| HEAD              | 1039.89 ms      | 860.88 ms       | 1112.312 ms
    |  1122.52 ms    |
+-------------------+--------------------+--------------------+----------------------+----------------+
| POC design#1|  310.920 ms    | 293.14 ms       |  385.698  ms      |
  456.64 ms    |
+-------------------+--------------------+--------------------+----------------------+----------------+
| POC design#2 |  318.464 ms    | 313.98 ms      |  352.316 ms       |
  441.53 ms    |
+-------------------+--------------------+--------------------+----------------------+----------------+
Performance results for 1000 empty tables:
+-------------------+--------------------+--------------------+----------------------+----------------+
|                         | 2 sync workers | 4 sync workers | 8 sync
workers | 16 sync workers|
+------------------+---------------------+---------------------+---------------------+----------------+
| HEAD              | 16327.96 ms    | 10253.65 ms    | 9741.986 ms
|  10278.98 ms   |
+-------------------+--------------------+---------------------+---------------------+----------------+
| POC design#1|  3598.21 ms    |  3099.54 ms     | 2944.386 ms     |
2588.20 ms   |
+-------------------+--------------------+---------------------+---------------------+----------------+
| POC design#2|  4131.72 ms    |  2840.36 ms     | 3001.159 ms    |
5461.82 ms   |
+-------------------+--------------------+---------------------+--------------------+----------------+
Performance results for 2000 empty tables:
+-------------------+--------------------+--------------------+----------------------+----------------+
|                         | 2 sync workers | 4 sync workers | 8 sync
workers | 16 sync workers|
+-------------------+--------------------+--------------------+----------------------+----------------+
| HEAD              | 47210.92 ms    | 25239.90 ms   | 19171.48 ms
|   19556.46 ms  |
+-------------------+--------------------+--------------------+---------------------+----------------+
| POC design#1| 10598.32 ms   |  6995.61 ms    |  6507.53 ms      |
5295.72 ms  |
+-------------------+--------------------+--------------------+-------------------------------------+
| POC design#2| 11121.00 ms   |  6659.74 ms    |  6253.66 ms      |
15433.81 ms  |
+-------------------+--------------------+--------------------+-------------------------------------+

The performance result execution for the same is attached in
Perftest_Results.xlsx.
Also testing with a) table having data and b) apply worker applying
changes while table sync in progress is not done. One of us will do
and try to share the results for these too.
It is noticed that performance of POC design #1 and POC design #2 are
good but POC design #2's performance degrades when there are a greater
number of workers and more tables. In POC design #2, when there are a
greater number of workers and tables, apply worker is becoming a
bottleneck as it must allocate work for all the workers.
Based on the test results, POC design #1 is better.

Thanks to Kuroda-san for helping me in running the performance tests.

[1]: /messages/by-id/CAGPVpCSk4v-V1WbFDy8a5dL7Es5z8da6hoQbuVyrqP5s3Yh6Cg@mail.gmail.com
[2]: /messages/by-id/CAHut+Ps8gWP9tCPK9gdMnxyshRKgVP3pJnAnaJto_T07uR9xUA@mail.gmail.com

Regards,
Vignesh

Attachments:

0003-reuse-connection-when-tablesync-workers-change-the-t.patchapplication/octet-stream; name=0003-reuse-connection-when-tablesync-workers-change-the-t.patchDownload
From ac8da0bca98f24c1498346b85e2b2751e0aa9ef6 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:13:52 +0300
Subject: [PATCH 3/4] reuse connection when tablesync workers change the target

---
 src/backend/replication/logical/tablesync.c | 53 ++++++++++++++-------
 src/backend/replication/logical/worker.c    | 30 +++++++-----
 src/backend/replication/walsender.c         |  6 +++
 src/include/replication/worker_internal.h   |  3 +-
 4 files changed, 61 insertions(+), 31 deletions(-)

diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 605c5bd4ec..f042d9ae00 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -144,16 +144,6 @@ clean_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
-	/*
-	 * Disconnect from publisher. Otherwise reused sync workers causes
-	 * exceeding max_wal_senders
-	 */
-	if (LogRepWorkerWalRcvConn != NULL)
-	{
-		walrcv_disconnect(LogRepWorkerWalRcvConn);
-		LogRepWorkerWalRcvConn = NULL;
-	}
-
 	/* Find the leader apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
 }
@@ -167,6 +157,16 @@ finish_sync_worker(void)
 {
 	clean_sync_worker();
 
+	/*
+	 * Disconnect from publisher. Otherwise reused sync workers causes
+	 * exceeding max_wal_senders.
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		walrcv_disconnect(LogRepWorkerWalRcvConn);
+		LogRepWorkerWalRcvConn = NULL;
+	}
+
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
@@ -1268,7 +1268,7 @@ ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
  * The returned slot name is palloc'ed in current memory context.
  */
 char *
-LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
+LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 {
 	char	   *slotname;
 	char	   *err;
@@ -1321,14 +1321,31 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 									NAMEDATALEN);
 
 	/*
-	 * Here we use the slot name instead of the subscription name as the
-	 * application_name, so that it is different from the leader apply worker,
-	 * so that synchronous replication can distinguish them.
+	 * Connect to publisher if not yet. The application_name must be also
+	 * different from the leader apply worker because synchronous replication
+	 * must distinguish them.
 	 */
-	LogRepWorkerWalRcvConn =
-		walrcv_connect(MySubscription->conninfo, true,
-					   must_use_password,
-					   slotname, &err);
+	if (LogRepWorkerWalRcvConn == NULL)
+	{
+		char application_name[NAMEDATALEN];
+
+		/*
+		 * FIXME: set appropriate application_name. Previously, the slot name
+		 * was used becasue the lifetime of the tablesync worker was same as
+		 * that, but now the tablesync worker handles many slots during the
+		 * synchronization so that it is not suitable. So what should be?
+		 * Note that if the tablesync worker starts to reuse the replication
+		 * slot during synchronization, we should use the slot name as
+		 * application_name again.
+		 */
+		snprintf(application_name, NAMEDATALEN, "pg_%u_sync_%i",
+				 MySubscription->oid, worker_slot);
+		LogRepWorkerWalRcvConn =
+			walrcv_connect(MySubscription->conninfo, true,
+						   must_use_password,
+						   application_name, &err);
+	}
+
 	if (LogRepWorkerWalRcvConn == NULL)
 		ereport(ERROR,
 				(errcode(ERRCODE_CONNECTION_FAILURE),
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 81f7a6de66..f77bc55e34 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3486,19 +3486,21 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 	/*
 	 * Init the ApplyMessageContext which we clean up after each replication
-	 * protocol message.
+	 * protocol message, if needed.
 	 */
-	ApplyMessageContext = AllocSetContextCreate(ApplyContext,
-												"ApplyMessageContext",
-												ALLOCSET_DEFAULT_SIZES);
+	if (!ApplyMessageContext)
+		ApplyMessageContext = AllocSetContextCreate(ApplyContext,
+													"ApplyMessageContext",
+													ALLOCSET_DEFAULT_SIZES);
 
 	/*
 	 * This memory context is used for per-stream data when the streaming mode
 	 * is enabled. This context is reset on each stream stop.
 	 */
-	LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
-													"LogicalStreamingContext",
-													ALLOCSET_DEFAULT_SIZES);
+	if (!LogicalStreamingContext)
+		LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
+														"LogicalStreamingContext",
+														ALLOCSET_DEFAULT_SIZES);
 
 	/* mark as idle, before starting to loop */
 	pgstat_report_activity(STATE_IDLE, NULL);
@@ -4451,7 +4453,9 @@ TwoPhaseTransactionGid(Oid subid, TransactionId xid, char *gid, int szgid)
  * are not repeatable.
  */
 static void
-start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+start_table_sync(XLogRecPtr *origin_startpos,
+				 char **myslotname,
+				 int worker_slot)
 {
 	char	   *syncslotname = NULL;
 
@@ -4460,7 +4464,7 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 	PG_TRY();
 	{
 		/* Call initial sync. */
-		syncslotname = LogicalRepSyncTableStart(origin_startpos);
+		syncslotname = LogicalRepSyncTableStart(origin_startpos, worker_slot);
 	}
 	PG_CATCH();
 	{
@@ -4531,12 +4535,13 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 					 char *slotname,
 					 char *originname,
 					 int originname_size,
-					 XLogRecPtr *origin_startpos)
+					 XLogRecPtr *origin_startpos,
+					 int worker_slot)
 {
 	MyLogicalRepWorker->is_sync_completed = false;
 
 	/* Start table synchronization. */
-	start_table_sync(origin_startpos, &slotname);
+	start_table_sync(origin_startpos, &slotname, worker_slot);
 
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
 									   MyLogicalRepWorker->relid,
@@ -4837,7 +4842,8 @@ TablesyncWorkerMain(Datum main_arg)
 							 myslotname,
 							 originname,
 							 sizeof(originname),
-							 &origin_startpos);
+							 &origin_startpos,
+							 worker_slot);
 
 		if (IsTransactionState())
 			CommitTransactionCommand();
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index d27ef2985d..6e878e7bf5 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1827,6 +1827,12 @@ exec_replication_command(const char *cmd_string)
 				set_ps_display(cmdtag);
 				PreventInTransactionBlock(true, cmdtag);
 
+				/*
+				 * Initialize the flag again because this streaming may be
+				 * second time.
+				 */
+				streamingDoneSending = streamingDoneReceiving = false;
+
 				if (cmd->kind == REPLICATION_KIND_PHYSICAL)
 					StartReplication(cmd);
 				else
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 1e9f8e6e72..af6fd339f7 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -249,7 +249,8 @@ extern int	logicalrep_sync_worker_count(Oid subid);
 
 extern void ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 											   char *originname, Size szoriginname);
-extern char *LogicalRepSyncTableStart(XLogRecPtr *origin_startpos);
+extern char *LogicalRepSyncTableStart(XLogRecPtr *origin_startpos,
+									  int worker_slot);
 
 extern bool AllTablesyncsReady(void);
 extern void UpdateTwoPhaseState(Oid suboid, char new_state);
-- 
2.34.1

0002-Reuse-Tablesync-Workers.patchapplication/octet-stream; name=0002-Reuse-Tablesync-Workers.patchDownload
From 533aa256916a83ac9907c513bed765e8d8e13c6f Mon Sep 17 00:00:00 2001
From: Vignesh C <vignesh21@gmail.com>
Date: Fri, 14 Jul 2023 09:58:04 +0530
Subject: [PATCH 2/4] Reuse Tablesync Workers

This commit allows reusing tablesync workers for syncing more than one
table sequentially during their lifetime, instead of exiting after
only syncing one table.

Before this commit, tablesync workers were capable of syncing only one
table. For each table, a new sync worker was launched and that worker would
exit when done processing the table.

Now, tablesync workers are not limited to processing only one
table. When done, they can move to processing another table in
the same subscription.

If there is a table that needs to be synced, an available tablesync
worker picks up that table and syncs it. Each tablesync worker
continues to pick new tables to sync until there are no tables left
requiring synchronization. If there was no available worker to
process the table, then a new tablesync worker will be launched,
provided the number of tablesync workers for the subscription does not
exceed max_sync_workers_per_subscription.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/replication/logical/launcher.c  |   1 +
 src/backend/replication/logical/tablesync.c |  50 ++++++--
 src/backend/replication/logical/worker.c    | 121 ++++++++++++++++++--
 src/include/replication/worker_internal.h   |   7 ++
 4 files changed, 158 insertions(+), 21 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e231fa7f95..72e5ef8a78 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -440,6 +440,7 @@ retry:
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
+	worker->is_sync_completed = false;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 8125bbd170..605c5bd4ec 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -129,15 +129,14 @@ static bool FetchTableStates(bool *started_tx);
 static StringInfo copybuf = NULL;
 
 /*
- * Exit routine for synchronization worker.
+ * Prepares the synchronization worker for reuse or exit.
  */
 void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+clean_sync_worker(void)
 {
 	/*
-	 * Commit any outstanding transaction. This is the usual case, unless
-	 * there was nothing to do for the table.
+	 * Commit any outstanding transaction. This is the usual case, unless there
+	 * was nothing to do for the table.
 	 */
 	if (IsTransactionState())
 	{
@@ -145,19 +144,38 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
+	/*
+	 * Disconnect from publisher. Otherwise reused sync workers causes
+	 * exceeding max_wal_senders
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		walrcv_disconnect(LogRepWorkerWalRcvConn);
+		LogRepWorkerWalRcvConn = NULL;
+	}
+
+	/* Find the leader apply worker and signal it. */
+	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
+}
+
+/*
+ * Exit routine for synchronization worker.
+ */
+void
+pg_attribute_noreturn()
+finish_sync_worker(void)
+{
+	clean_sync_worker();
+
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
 	StartTransactionCommand();
 	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
+			(errmsg("logical replication table synchronization worker for subscription \"%s\" has finished",
+					MySubscription->name)));
 	CommitTransactionCommand();
 
-	/* Find the leader apply worker and signal it. */
-	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
-
 	/* Stop gracefully */
 	proc_exit(0);
 }
@@ -379,7 +397,15 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 */
 		replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
+		/* Sync worker has completed synchronization of the current table. */
+		MyLogicalRepWorker->is_sync_completed = true;
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", relation \"%s\" with relid %u has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
+		CommitTransactionCommand();
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 1a530d3bb1..81f7a6de66 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3612,6 +3612,20 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can call process_syncing_tables_for_sync.
+				 *
+				 * process_syncing_tables_for_sync decides whether the sync of
+				 * the current table is completed. If it is completed,
+				 * streaming must be already ended. So, we can break the loop.
+				 */
+				if (MyLogicalRepWorker->is_sync_completed)
+				{
+					endofstream = true;
+					break;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -3631,6 +3645,15 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+
+			/*
+			 * If is_sync_completed is true, this means that the tablesync
+			 * worker is done with synchronization. Streaming has already been
+			 * ended by process_syncing_tables_for_sync. We should move to the
+			 * next table if needed, or exit.
+			 */
+			if (MyLogicalRepWorker->is_sync_completed)
+				endofstream = true;
 		}
 
 		/* Cleanup the memory. */
@@ -3733,8 +3756,12 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * Tablesync workers should end streaming before exiting the main loop to
+	 * drop replication slot. Only end streaming here for apply workers.
+	 */
+	if (!am_tablesync_worker())
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
 }
 
 /*
@@ -4506,6 +4533,8 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 					 int originname_size,
 					 XLogRecPtr *origin_startpos)
 {
+	MyLogicalRepWorker->is_sync_completed = false;
+
 	/* Start table synchronization. */
 	start_table_sync(origin_startpos, &slotname);
 
@@ -4683,10 +4712,11 @@ InitializeLogRepWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("%s for subscription \"%s\", table \"%s\" has started",
+				(errmsg("%s for subscription \"%s\", table \"%s\" with relid %u has started",
 						get_worker_name(),
 						MySubscription->name,
-						get_rel_name(MyLogicalRepWorker->relid))));
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
 	else
 		ereport(LOG,
 				(errmsg("logical replication apply worker for subscription \"%s\" has started",
@@ -4793,11 +4823,84 @@ TablesyncWorkerMain(Datum main_arg)
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	run_tablesync_worker(&options,
-						 myslotname,
-						 originname,
-						 sizeof(originname),
-						 &origin_startpos);
+	/*
+	 * The loop where worker does its job. It loops until there is no relation
+	 * left to sync.
+	 */
+	for (;;)
+	{
+		List	   *rstates;
+		ListCell   *lc;
+		bool 	is_table_found = false;
+
+		run_tablesync_worker(&options,
+							 myslotname,
+							 originname,
+							 sizeof(originname),
+							 &origin_startpos);
+
+		if (IsTransactionState())
+			CommitTransactionCommand();
+
+		if (MyLogicalRepWorker->is_sync_completed)
+		{
+			/* This transaction will be committed by clean_sync_worker. */
+			StartTransactionCommand();
+
+			/*
+			 * Check if there is any table whose relation state is still INIT.
+			 * If a table in INIT state is found, the worker will not be
+			 * finished, it will be reused instead.
+			 */
+			rstates = GetSubscriptionRelations(MySubscription->oid, true);
+
+			foreach(lc, rstates)
+			{
+				SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
+
+				if (rstate->state == SUBREL_STATE_SYNCDONE)
+					continue;
+
+				/*
+				 * Take exclusive lock to prevent any other sync worker from
+				 * picking the same table.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
+
+				/*
+				 * Pick the table for the next run if it is not already picked
+				 * up by another worker.
+				 */
+				if (!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+				{
+					/* Update worker state for the next table */
+					MyLogicalRepWorker->relid = rstate->relid;
+					MyLogicalRepWorker->relstate = rstate->state;
+					MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+					LWLockRelease(LogicalRepWorkerLock);
+
+					/* Found a table for next iteration */
+					is_table_found = true;
+					clean_sync_worker();
+
+					StartTransactionCommand();
+					ereport(LOG,
+							(errmsg("%s for subscription \"%s\" has moved to sync table \"%s\" with relid %u.",
+									get_worker_name(),
+									MySubscription->name,
+									get_rel_name(MyLogicalRepWorker->relid),
+									MyLogicalRepWorker->relid)));
+					CommitTransactionCommand();
+
+					break;
+				}
+				LWLockRelease(LogicalRepWorkerLock);
+			}
+
+			if (!is_table_found)
+				break;
+		}
+	}
 
 	finish_sync_worker();
 }
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 7aba034774..1e9f8e6e72 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -56,6 +56,12 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/*
+	 * Indicates whether tablesync worker has completed sycning its assigned
+	 * table. If true, no need to continue with that table.
+	 */
+	bool		is_sync_completed;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction or
@@ -308,6 +314,7 @@ extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
 #define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)
 
 extern void finish_sync_worker(void);
+extern void clean_sync_worker(void);
 
 static inline bool
 am_tablesync_worker(void)
-- 
2.34.1

0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchapplication/octet-stream; name=0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchDownload
From 6085e260aac441392c658b61c08d86779ecc4690 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Mon, 5 Jun 2023 15:04:41 +0300
Subject: [PATCH 1/4] Refactor to split Apply and Tablesync Workers

Both apply and tablesync workers were using ApplyWorkerMain() as entry
point. As the name implies, ApplyWorkerMain() should be considered as
the main function for apply workers. Tablesync worker's path was hidden
and does not have enough in common to share the same main function with
apply worker.

Also; most of the code shared by both worker types are already combined
in LogicalRepApplyLoop(). There is no need to combine the rest in
ApplyWorkerMain() anymore.

This commit introduces TablesyncWorkerMain() as a new entry point for
tablesync workers and separates both type of workers from each other.
This aims to increase code readability and help to maintain logical
replication workers separately.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/postmaster/bgworker.c             |   3 +
 .../replication/logical/applyparallelworker.c |   2 +-
 src/backend/replication/logical/launcher.c    |  32 +-
 src/backend/replication/logical/tablesync.c   |   2 +-
 src/backend/replication/logical/worker.c      | 388 +++++++++++-------
 src/include/replication/logicalworker.h       |   1 +
 src/include/replication/worker_internal.h     |   4 +-
 7 files changed, 270 insertions(+), 162 deletions(-)

diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 5b4bd71694..505e38376c 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -131,6 +131,9 @@ static const struct
 	},
 	{
 		"ParallelApplyWorkerMain", ParallelApplyWorkerMain
+	},
+	{
+		"TablesyncWorkerMain", TablesyncWorkerMain
 	}
 };
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 6fb96148f4..1d4e83c4c1 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -942,7 +942,7 @@ ParallelApplyWorkerMain(Datum main_arg)
 	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
 		MyLogicalRepWorker->reply_time = 0;
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 542af7d863..e231fa7f95 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -459,24 +459,30 @@ retry:
 	snprintf(bgw.bgw_library_name, MAXPGPATH, "postgres");
 
 	if (is_parallel_apply_worker)
+	{
 		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ParallelApplyWorkerMain");
-	else
-		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
-
-	if (OidIsValid(relid))
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication worker for subscription %u sync %u", subid, relid);
-	else if (is_parallel_apply_worker)
+				 "logical replication parallel apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
+	}
+	else if (OidIsValid(relid))
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "TablesyncWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication parallel apply worker for subscription %u", subid);
+				 "logical replication tablesync worker for subscription %u sync %u",
+				 subid,
+				 relid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication tablesync worker");
+	}
 	else
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication apply worker for subscription %u", subid);
-
-	if (is_parallel_apply_worker)
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
-	else
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication worker");
+				 "logical replication apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication apply worker");
+	}
 
 	bgw.bgw_restart_time = BGW_NEVER_RESTART;
 	bgw.bgw_notify_pid = MyProcPid;
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 6d461654ab..8125bbd170 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -131,7 +131,7 @@ static StringInfo copybuf = NULL;
 /*
  * Exit routine for synchronization worker.
  */
-static void
+void
 pg_attribute_noreturn()
 finish_sync_worker(void)
 {
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index dd353fd1cb..1a530d3bb1 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -392,6 +392,9 @@ static void stream_open_file(Oid subid, TransactionId xid,
 static void stream_write_change(char action, StringInfo s);
 static void stream_open_and_write_change(TransactionId xid, char action, StringInfo s);
 static void stream_close_file(void);
+static void set_stream_options(WalRcvStreamOptions *options,
+							   char *slotname,
+							   XLogRecPtr *origin_startpos);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
@@ -4313,6 +4316,72 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+ /* set_stream_options
+  * 	Set logical replication streaming options.
+  *
+  * This function sets streaming options including replication slot name and
+  * origin start position. Workers need these options for logical replication.
+  */
+static void
+set_stream_options(WalRcvStreamOptions *options,
+				   char *slotname,
+				   XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+		options->proto.logical.twophase = true;
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4416,7 +4485,8 @@ start_apply(XLogRecPtr origin_startpos)
 			 * idle state.
 			 */
 			AbortOutOfAnyTransaction();
-			pgstat_report_subscription_error(MySubscription->oid, !am_tablesync_worker());
+			pgstat_report_subscription_error(MySubscription->oid,
+											 !am_tablesync_worker());
 
 			PG_RE_THROW();
 		}
@@ -4425,13 +4495,133 @@ start_apply(XLogRecPtr origin_startpos)
 }
 
 /*
- * Common initialization for leader apply worker and parallel apply worker.
+ * Runs the tablesync worker.
+ * It starts syncing tables. After a successful sync, sets streaming options
+ * and starts streaming to catchup.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	/* Start table synchronization. */
+	start_table_sync(origin_startpos, &slotname);
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   originname_size);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	/* Start applying changes to catchup. */
+	start_apply(*origin_startpos);
+}
+
+/*
+ * Runs the leader apply worker.
+ * It sets up replication origin, streaming options and then starts streaming.
+ */
+static void
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
+{
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+	bool		must_use_password;
+
+	slotname = MySubscription->slotname;
+
+	/*
+	 * This shouldn't happen if the subscription is enabled, but guard
+	 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+	 * crash if slot is NULL.)
+	 */
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
+
+	/* Setup replication origin tracking. */
+	StartTransactionCommand();
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+
+	/* Is the use of a password mandatory? */
+	must_use_password = MySubscription->passwordrequired &&
+		!superuser_arg(MySubscription->owner);
+
+	/* Note that the superuser_arg call can access the DB */
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											must_use_password,
+											MySubscription->name, &err);
+
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+	 * We don't really use the output identify_system for anything but it
+	 * does some initializations on the upstream so let's still call it.
+	 */
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+	{
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
+	}
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+
+	/* Run the main loop. */
+	start_apply(*origin_startpos);
+}
+
+/*
+ * Common initialization for logical replication workers; leader apply worker,
+ * parallel apply worker and tablesync worker.
  *
  * Initialize the database connection, in-memory subscription and necessary
  * config options.
  */
 void
-InitializeApplyWorker(void)
+InitializeLogRepWorker(void)
 {
 	MemoryContext oldctx;
 
@@ -4493,7 +4683,8 @@ InitializeApplyWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has started",
+				(errmsg("%s for subscription \"%s\", table \"%s\" has started",
+						get_worker_name(),
 						MySubscription->name,
 						get_rel_name(MyLogicalRepWorker->relid))));
 	else
@@ -4513,7 +4704,6 @@ ApplyWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
-	int			server_version;
 
 	InitializingApplyWorker = true;
 
@@ -4537,7 +4727,7 @@ ApplyWorkerMain(Datum main_arg)
 	/* Load the libpq-specific functions */
 	load_file("libpqwalreceiver", false);
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
@@ -4545,71 +4735,6 @@ ApplyWorkerMain(Datum main_arg)
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-		bool		must_use_password;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-
-		/* Is the use of a password mandatory? */
-		must_use_password = MySubscription->passwordrequired &&
-			!superuser_arg(MySubscription->owner);
-
-		/* Note that the superuser_arg call can access the DB */
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												must_use_password,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
 	 * the subscription relation state.
@@ -4618,92 +4743,63 @@ ApplyWorkerMain(Datum main_arg)
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
+	/* This is leader apply worker */
+	run_apply_worker(&options, myslotname, originname, sizeof(originname), &origin_startpos);
 
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
+	proc_exit(0);
+}
 
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
+/* Logical Replication Tablesync worker entry point */
+void
+TablesyncWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *myslotname = NULL;
+	WalRcvStreamOptions options;
+
+	/* Attach to slot */
+	logicalrep_worker_attach(worker_slot);
+
+	/* Setup signal handling */
+	pqsignal(SIGHUP, SignalHandlerForConfigReload);
+	pqsignal(SIGTERM, die);
+	BackgroundWorkerUnblockSignals();
 
 	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 * We don't currently need any ResourceOwner in a walreceiver process, but
+	 * if we did, we could call CreateAuxProcessResourceOwner here.
 	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
 
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
+	/* Initialise stats to a sanish value */
+	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
+		MyLogicalRepWorker->reply_time = GetCurrentTimestamp();
 
-	if (!am_tablesync_worker())
-	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
-		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+	/* Load the libpq-specific functions */
+	load_file("libpqwalreceiver", false);
 
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
-		}
-		else
-		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-		}
+	InitializeLogRepWorker();
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-								 MySubscription->name,
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-								 "?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
+	/* Connect to the origin and start the replication. */
+	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
+		 MySubscription->conninfo);
 
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+	/*
+	 * Setup callback for syscache so that we know when something changes in
+	 * the subscription relation state.
+	 */
+	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
+								  invalidate_syncing_table_states,
+								  (Datum) 0);
 
-	proc_exit(0);
+	run_tablesync_worker(&options,
+						 myslotname,
+						 originname,
+						 sizeof(originname),
+						 &origin_startpos);
+
+	finish_sync_worker();
 }
 
 /*
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 39588da79f..bbd71d0b42 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -18,6 +18,7 @@ extern PGDLLIMPORT volatile sig_atomic_t ParallelApplyMessagePending;
 
 extern void ApplyWorkerMain(Datum main_arg);
 extern void ParallelApplyWorkerMain(Datum main_arg);
+extern void TablesyncWorkerMain(Datum main_arg);
 
 extern bool IsLogicalWorker(void);
 extern bool IsLogicalParallelApplyWorker(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 343e781896..7aba034774 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -265,7 +265,7 @@ extern void maybe_reread_subscription(void);
 
 extern void stream_cleanup_files(Oid subid, TransactionId xid);
 
-extern void InitializeApplyWorker(void);
+extern void InitializeLogRepWorker(void);
 
 extern void store_flush_position(XLogRecPtr remote_lsn, XLogRecPtr local_lsn);
 
@@ -307,6 +307,8 @@ extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
 
 #define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)
 
+extern void finish_sync_worker(void);
+
 static inline bool
 am_tablesync_worker(void)
 {
-- 
2.34.1

0004-Support-worker-pool-for-table-sync.patchapplication/octet-stream; name=0004-Support-worker-pool-for-table-sync.patchDownload
From a4cb759a9f053fe66b8d1632a89d7880c2c6dfc2 Mon Sep 17 00:00:00 2001
From: Vignesh C <vignesh21@gmail.com>
Date: Fri, 14 Jul 2023 09:59:53 +0530
Subject: [PATCH 4/4] Support worker pool for table sync.

Support worker pool for table sync.
---
 src/backend/replication/logical/Makefile      |   1 +
 src/backend/replication/logical/launcher.c    |   4 +-
 src/backend/replication/logical/meson.build   |   1 +
 src/backend/replication/logical/tablesync.c   | 185 +++++----
 .../replication/logical/tablesyncpool.c       | 382 ++++++++++++++++++
 src/backend/replication/logical/worker.c      | 154 +++----
 src/include/replication/worker_internal.h     |  71 +++-
 src/include/storage/procsignal.h              |   1 +
 8 files changed, 641 insertions(+), 158 deletions(-)
 create mode 100644 src/backend/replication/logical/tablesyncpool.c

diff --git a/src/backend/replication/logical/Makefile b/src/backend/replication/logical/Makefile
index 2dc25e37bb..251a28c4bd 100644
--- a/src/backend/replication/logical/Makefile
+++ b/src/backend/replication/logical/Makefile
@@ -27,6 +27,7 @@ OBJS = \
 	reorderbuffer.o \
 	snapbuild.o \
 	tablesync.o \
+	tablesyncpool.o \
 	worker.o
 
 include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 72e5ef8a78..6ef38fc68d 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -315,7 +315,7 @@ logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname, Oid userid,
 	int			nsyncworkers;
 	int			nparallelapplyworkers;
 	TimestampTz now;
-	bool		is_parallel_apply_worker = (subworker_dsm != DSM_HANDLE_INVALID);
+	bool		is_parallel_apply_worker = (subworker_dsm != DSM_HANDLE_INVALID) && !OidIsValid(relid);
 
 	/* Sanity check - tablesync worker cannot be a subworker */
 	Assert(!(is_parallel_apply_worker && OidIsValid(relid)));
@@ -489,7 +489,7 @@ retry:
 	bgw.bgw_notify_pid = MyProcPid;
 	bgw.bgw_main_arg = Int32GetDatum(slot);
 
-	if (is_parallel_apply_worker)
+	if (is_parallel_apply_worker || OidIsValid(relid))
 		memcpy(bgw.bgw_extra, &subworker_dsm, sizeof(dsm_handle));
 
 	if (!RegisterDynamicBackgroundWorker(&bgw, &bgw_handle))
diff --git a/src/backend/replication/logical/meson.build b/src/backend/replication/logical/meson.build
index d48cd4c590..99726fe504 100644
--- a/src/backend/replication/logical/meson.build
+++ b/src/backend/replication/logical/meson.build
@@ -13,5 +13,6 @@ backend_sources += files(
   'reorderbuffer.c',
   'snapbuild.c',
   'tablesync.c',
+  'tablesyncpool.c',
   'worker.c',
 )
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index f042d9ae00..28113108df 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -152,10 +152,17 @@ clean_sync_worker(void)
  * Exit routine for synchronization worker.
  */
 void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+finish_sync_worker(bool reuse_worker)
 {
-	clean_sync_worker();
+	/*
+	 * Commit any outstanding transaction. This is the usual case, unless there
+	 * was nothing to do for the table.
+	 */
+	if (IsTransactionState())
+	{
+		CommitTransactionCommand();
+		pgstat_report_stat(true);
+	}
 
 	/*
 	 * Disconnect from publisher. Otherwise reused sync workers causes
@@ -167,17 +174,22 @@ finish_sync_worker(void)
 		LogRepWorkerWalRcvConn = NULL;
 	}
 
-	/* And flush all writes. */
-	XLogFlush(GetXLogWriteRecPtr());
+	if (!reuse_worker)
+	{
+		tsp_worker_cleanup();
 
-	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\" has finished",
-					MySubscription->name)));
-	CommitTransactionCommand();
+		/* And flush all writes. */
+		XLogFlush(GetXLogWriteRecPtr());
+
+		StartTransactionCommand();
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\" has finished",
+						MySubscription->name)));
+		CommitTransactionCommand();
 
-	/* Stop gracefully */
-	proc_exit(0);
+		/* Stop gracefully */
+		proc_exit(0);
+	}
 }
 
 /*
@@ -406,11 +418,59 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 						get_rel_name(MyLogicalRepWorker->relid),
 						MyLogicalRepWorker->relid)));
 		CommitTransactionCommand();
+		tsp_send_signal(tsp_get_apply_worker_pid());
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
 }
 
+static void
+check_caughtup_update_state(XLogRecPtr current_lsn,
+							SubscriptionRelState *rstate, bool *started_tx)
+{
+	/*
+	 * Apply has caught up to the position where the table sync has
+	 * finished.  Mark the table as ready so that the apply will just
+	 * continue to replicate it normally.
+	 */
+	if (current_lsn >= rstate->lsn)
+	{
+		char		originname[NAMEDATALEN];
+
+		rstate->state = SUBREL_STATE_READY;
+		rstate->lsn = current_lsn;
+		if (!(*started_tx))
+		{
+			StartTransactionCommand();
+			*started_tx = true;
+		}
+
+		/*
+		 * Remove the tablesync origin tracking if exists.
+		 *
+		 * There is a chance that the user is concurrently performing
+		 * refresh for the subscription where we remove the table
+		 * state and its origin or the tablesync worker would have
+		 * already removed this origin. We can't rely on tablesync
+		 * worker to remove the origin tracking as if there is any
+		 * error while dropping we won't restart it to drop the
+		 * origin. So passing missing_ok = true.
+		 */
+		ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+											rstate->relid,
+											originname,
+											sizeof(originname));
+		replorigin_drop_by_name(originname, true, false);
+
+		/*
+		 * Update the state to READY only after the origin cleanup.
+		 */
+		UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
+									rstate->relid, rstate->state,
+									rstate->lsn);
+	}
+}
+
 /*
  * Handle table synchronization cooperation from the apply worker.
  *
@@ -469,10 +529,15 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 	 * Clean up the hash table when we're done with all tables (just to
 	 * release the bit of memory).
 	 */
-	else if (table_states_not_ready == NIL && last_start_times)
+	else if (table_states_not_ready == NIL)
 	{
-		hash_destroy(last_start_times);
-		last_start_times = NULL;
+		if (last_start_times)
+		{
+			hash_destroy(last_start_times);
+			last_start_times = NULL;
+		}
+
+		tsp_free_all_workers();
 	}
 
 	/*
@@ -483,49 +548,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 		SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
 
 		if (rstate->state == SUBREL_STATE_SYNCDONE)
-		{
-			/*
-			 * Apply has caught up to the position where the table sync has
-			 * finished.  Mark the table as ready so that the apply will just
-			 * continue to replicate it normally.
-			 */
-			if (current_lsn >= rstate->lsn)
-			{
-				char		originname[NAMEDATALEN];
-
-				rstate->state = SUBREL_STATE_READY;
-				rstate->lsn = current_lsn;
-				if (!started_tx)
-				{
-					StartTransactionCommand();
-					started_tx = true;
-				}
-
-				/*
-				 * Remove the tablesync origin tracking if exists.
-				 *
-				 * There is a chance that the user is concurrently performing
-				 * refresh for the subscription where we remove the table
-				 * state and its origin or the tablesync worker would have
-				 * already removed this origin. We can't rely on tablesync
-				 * worker to remove the origin tracking as if there is any
-				 * error while dropping we won't restart it to drop the
-				 * origin. So passing missing_ok = true.
-				 */
-				ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
-												   rstate->relid,
-												   originname,
-												   sizeof(originname));
-				replorigin_drop_by_name(originname, true, false);
-
-				/*
-				 * Update the state to READY only after the origin cleanup.
-				 */
-				UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
-										   rstate->relid, rstate->state,
-										   rstate->lsn);
-			}
-		}
+			check_caughtup_update_state(current_lsn, rstate, &started_tx);
 		else
 		{
 			LogicalRepWorker *syncworker;
@@ -578,6 +601,8 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 
 					wait_for_relation_state_change(rstate->relid,
 												   SUBREL_STATE_SYNCDONE);
+					check_caughtup_update_state(current_lsn, rstate,
+												&started_tx);
 				}
 				else
 					LWLockRelease(LogicalRepWorkerLock);
@@ -589,38 +614,36 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				 * running sync workers for this subscription, while we have
 				 * the lock.
 				 */
-				int			nsyncworkers =
-					logicalrep_sync_worker_count(MyLogicalRepWorker->subid);
+				int nsyncworkers;
+				TimestampTz now = GetCurrentTimestamp();
+				struct tablesync_start_time_mapping *hentry;
+				bool		found;
+
+				nsyncworkers = logicalrep_sync_worker_count(MyLogicalRepWorker->subid);
 
 				/* Now safe to release the LWLock */
 				LWLockRelease(LogicalRepWorkerLock);
 
+				hentry = hash_search(last_start_times, &rstate->relid,
+										HASH_ENTER, &found);
+
 				/*
-				 * If there are free sync worker slot(s), start a new sync
-				 * worker for the table.
+				 * If free any workers are available don't launch it,
+				 * reuse the worker.
 				 */
-				if (nsyncworkers < max_sync_workers_per_subscription)
+				if (!tsp_allocate_free_worker(rstate->relid, rstate->state,
+											  rstate->lsn))
 				{
-					TimestampTz now = GetCurrentTimestamp();
-					struct tablesync_start_time_mapping *hentry;
-					bool		found;
-
-					hentry = hash_search(last_start_times, &rstate->relid,
-										 HASH_ENTER, &found);
-
-					if (!found ||
-						TimestampDifferenceExceeds(hentry->last_start_time, now,
-												   wal_retrieve_retry_interval))
+					if (nsyncworkers < max_sync_workers_per_subscription)
 					{
-						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
-												 MySubscription->oid,
-												 MySubscription->name,
-												 MyLogicalRepWorker->userid,
-												 rstate->relid,
-												 DSM_HANDLE_INVALID);
-						hentry->last_start_time = now;
+						if (!found ||
+							TimestampDifferenceExceeds(hentry->last_start_time, now,
+													wal_retrieve_retry_interval))
+							tsp_launch_worker(rstate->relid);
 					}
 				}
+
+				hentry->last_start_time = now;
 			}
 		}
 	}
@@ -1310,7 +1333,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos, int worker_slot)
 		case SUBREL_STATE_SYNCDONE:
 		case SUBREL_STATE_READY:
 		case SUBREL_STATE_UNKNOWN:
-			finish_sync_worker();	/* doesn't return */
+			finish_sync_worker(false);	/* doesn't return */
 	}
 
 	/* Calculate the name of the tablesync slot. */
diff --git a/src/backend/replication/logical/tablesyncpool.c b/src/backend/replication/logical/tablesyncpool.c
new file mode 100644
index 0000000000..24acfc9a99
--- /dev/null
+++ b/src/backend/replication/logical/tablesyncpool.c
@@ -0,0 +1,382 @@
+/*-------------------------------------------------------------------------
+ * tablesyncpool.c
+ *	   Support routines for syncronizing table using worker pool
+ *
+ * Copyright (c) 2023, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *	  src/backend/replication/logical/tablesyncpool.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "pgstat.h"
+#include "postmaster/interrupt.h"
+#include "replication/logicalworker.h"
+#include "replication/worker_internal.h"
+#include "tcop/tcopprot.h"
+#include "utils/inval.h"
+#include "utils/resowner.h"
+#include "utils/syscache.h"
+
+/*
+ * A list (pool) of table sync workers. The information for
+ * the new worker is added to the list after successfully launching it.
+ */
+static List *TableSyncPool = NIL;
+TablesyncPoolShared *MyTablesyncPoolShared = NULL;
+
+static void tsp_free_worker_info(TablesyncPoolInfo *winfo);
+
+/*
+ * Set up a dynamic shared memory segment.
+ *
+ * We set up a control region that contains a fixed-size worker info of
+ * TablesyncPoolShared.
+ *
+ * Returns true on success, false on failure.
+ */
+static bool
+tsp_setup_dsm(TablesyncPoolInfo *winfo)
+{
+	shm_toc_estimator e;
+	Size		segsize;
+	dsm_segment *seg;
+	shm_toc    *toc;
+	TablesyncPoolShared *shared;
+
+	/*
+	 * 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.
+	 */
+	shm_toc_initialize_estimator(&e);
+	shm_toc_estimate_chunk(&e, sizeof(TablesyncPoolShared));
+
+	shm_toc_estimate_keys(&e, 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);
+	if (!seg)
+		return false;
+
+	toc = shm_toc_create(PG_LOGICAL_TABLESYNC_POOL_SHM_MAGIC, dsm_segment_address(seg),
+						 segsize);
+
+	/* Set up the header region. */
+	shared = shm_toc_allocate(toc, sizeof(TablesyncPoolShared));
+	SpinLockInit(&shared->mutex);
+
+	shared->exec_state = TS_FREE;
+	shared->parent_pid = MyProcPid;
+	shared->relstate = 'i';
+	shared->relstate_lsn = 0;
+
+	shm_toc_insert(toc, TABLESYNC_POOL_KEY_SHARED, shared);
+
+	/* Return results to caller. */
+	winfo->dsm_seg = seg;
+	winfo->shared = shared;
+
+	return true;
+}
+
+/*
+ * Start a new tablesync worker and add it to the tablesync pool.
+ */
+TablesyncPoolInfo *
+tsp_launch_worker(Oid relid)
+{
+	MemoryContext oldcontext;
+	bool launched;
+	TablesyncPoolInfo *winfo;
+	ResourceOwner saveResourceOwner;
+
+	/*
+	 * The worker info can be used for the lifetime of the worker process, so
+	 * create it in a permanent context.
+	 */
+	oldcontext = MemoryContextSwitchTo(ApplyContext);
+
+	winfo = (TablesyncPoolInfo *)palloc0(sizeof(TablesyncPoolInfo));
+
+	saveResourceOwner = CurrentResourceOwner;
+	CurrentResourceOwner = NULL;
+
+	/* Setup shared memory. */
+	if (!tsp_setup_dsm(winfo))
+	{
+		CurrentResourceOwner = saveResourceOwner;
+		MemoryContextSwitchTo(oldcontext);
+		pfree(winfo);
+		return NULL;
+	}
+
+	CurrentResourceOwner = saveResourceOwner;
+	winfo->shared->relid = relid;
+
+	/* Start a new table sync worker. */
+	launched = logicalrep_worker_launch(MyLogicalRepWorker->dbid,
+										MySubscription->oid,
+										MySubscription->name,
+										MyLogicalRepWorker->userid,
+										relid,
+										dsm_segment_handle(winfo->dsm_seg));
+
+	if (launched)
+		TableSyncPool = lappend(TableSyncPool, winfo);
+	else
+	{
+		tsp_free_worker_info(winfo);
+		winfo = NULL;
+	}
+
+	MemoryContextSwitchTo(oldcontext);
+
+	return winfo;
+}
+
+/*
+ * Wait until the tablesync worker/apply worker changes the state to the
+ * expected one.
+ *
+ * Returns false if the tablesync worker/apply worker has disappeared.
+ */
+bool
+wait_for_tsp_worker_state_change(TablesyncPoolState expected_state, int pid,
+								 Oid relid, int wait_time,
+								 TablesyncPoolShared *shared)
+{
+	int			rc;
+	bool	proc_exited = false;
+
+	for (;;)
+	{
+		LogicalRepWorker *worker;
+		TablesyncPoolState state = tsp_get_exec_state(shared);
+
+		CHECK_FOR_INTERRUPTS();
+
+		if (shared->exited)
+			break;
+
+		/* It is applicable only for apply worker. */
+		if (!OidIsValid(relid) && state == TS_INIT_STOP)
+			break;
+
+		/* Done if already in correct state. */
+		if (state == expected_state)
+			return true;
+
+		if (tsp_send_signal(pid))
+		{
+			/* Bail out if the tablesync worker/apply worker has died. */
+			LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+			worker = logicalrep_worker_find(MyLogicalRepWorker->subid,
+											relid, false);
+			if (!worker || !worker->proc || (worker->in_use && pid != worker->proc->pid))
+				proc_exited = true;
+
+			LWLockRelease(LogicalRepWorkerLock);
+
+			if (proc_exited)
+				break;
+		}
+
+		/*
+		 * Wait. We expect to get a latch signal back from the tablesync
+		 * worker/apply worker, but use a timeout in case it dies without
+		 * sending one.
+		 */
+		rc = WaitLatch(MyLatch,
+					   WL_LATCH_SET | WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
+					   wait_time, WAIT_EVENT_LOGICAL_SYNC_STATE_CHANGE);
+
+		if (rc & WL_LATCH_SET)
+			ResetLatch(MyLatch);
+	}
+
+	return false;
+}
+
+/*
+ * Stop the given logical replication parallel tablesync worker.
+ */
+static void
+logicalrep_tsp_worker_stop(TablesyncPoolInfo *winfo)
+
+{
+	TablesyncPoolShared *shared = winfo->shared;
+
+	tsp_set_exec_state(shared, TS_INIT_STOP);
+	wait_for_tsp_worker_state_change(TS_STOP, shared->child_pid,
+									 shared->relid, 1000L, shared);
+}
+
+/*
+ * Allocate a free tablesync worker which is available.
+ */
+bool
+tsp_allocate_free_worker(Oid relid, char state, XLogRecPtr lsn)
+{
+	TablesyncPoolInfo *winfo;
+	ListCell *lc;
+	MemoryContext oldcontext;
+
+	if (!list_length(TableSyncPool))
+		return false;
+
+	oldcontext = MemoryContextSwitchTo(ApplyContext);
+
+retry:
+
+	/* Try to get an available table sync worker from the worker pool. */
+	foreach (lc, TableSyncPool)
+	{
+		TablesyncPoolState exec_state;
+
+		winfo = (TablesyncPoolInfo *)lfirst(lc);
+
+		/* worker might have exited due to an error */
+		if (winfo->shared->exited)
+		{
+			tsp_free_worker_info(winfo);
+
+			/* TableSyncPool has changed, repeate the loop again */
+			goto retry;
+		}
+
+		exec_state = tsp_get_exec_state(winfo->shared);
+
+		if (exec_state != TS_DONE)
+			continue;
+
+		winfo->shared->relid = relid;
+		winfo->shared->relstate = state;
+		winfo->shared->relstate_lsn = lsn;
+
+		tsp_set_exec_state(winfo->shared, TS_INIT);
+		tsp_send_signal(winfo->shared->child_pid);
+
+		MemoryContextSwitchTo(oldcontext);
+		return true;
+	}
+
+	MemoryContextSwitchTo(oldcontext);
+
+	return false;
+}
+
+/*
+ * Set the execution state for a given table sync worker.
+ */
+void
+tsp_set_exec_state(TablesyncPoolShared *wshared,
+				   TablesyncPoolState exec_state)
+{
+	SpinLockAcquire(&wshared->mutex);
+	wshared->exec_state = exec_state;
+	SpinLockRelease(&wshared->mutex);
+}
+
+/*
+ * Get the execution state of a given table sync worker.
+ */
+TablesyncPoolState
+tsp_get_exec_state(TablesyncPoolShared *wshared)
+{
+	TablesyncPoolState exec_state;
+
+	SpinLockAcquire(&wshared->mutex);
+	exec_state = wshared->exec_state;
+	SpinLockRelease(&wshared->mutex);
+
+	return exec_state;
+}
+
+/*
+ * Free the table sync worker.
+ */
+static void
+tsp_free_worker(TablesyncPoolInfo *winfo)
+{
+	logicalrep_tsp_worker_stop(winfo);
+	tsp_free_worker_info(winfo);
+}
+
+/*
+ * Free the table sync worker information.
+ */
+static void
+tsp_free_worker_info(TablesyncPoolInfo *winfo)
+{
+	Assert(winfo);
+
+	if (winfo->dsm_seg)
+		dsm_detach(winfo->dsm_seg);
+
+	/* Remove from the worker pool. */
+	TableSyncPool = list_delete_ptr(TableSyncPool, winfo);
+
+	pfree(winfo);
+}
+
+/*
+ * Free all the table sync workers as there is no more work to allocate.
+ */
+void
+tsp_free_all_workers()
+{
+	TablesyncPoolInfo *winfo = NULL;
+
+	while (list_length(TableSyncPool) > 0)
+	{
+		winfo = (TablesyncPoolInfo *)lfirst(list_head(TableSyncPool));
+		tsp_free_worker(winfo);
+	}
+
+	TableSyncPool = NIL;
+}
+
+/*
+ * Stop the tablesync worker.
+ */
+void
+tsp_worker_cleanup(void)
+{
+	tsp_set_exec_state(MyTablesyncPoolShared, TS_STOP);
+	tsp_send_signal(MyTablesyncPoolShared->parent_pid);
+}
+
+/*
+ * Set the tablesync shared information global variable.
+ */
+void
+tsp_set_shared(TablesyncPoolShared *shared)
+{
+	MyTablesyncPoolShared = shared;
+}
+
+/*
+ * Get the apply worker pid.
+ */
+int
+tsp_get_apply_worker_pid(void)
+{
+	return MyTablesyncPoolShared->parent_pid;
+}
+
+/*
+ * Send signal to tsp worker or apply worker.
+ */
+int
+tsp_send_signal(int pid)
+{
+	return SendProcSignal(pid, PROCSIG_TABLESYNC_WORKER_POOL, InvalidBackendId);
+}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index f77bc55e34..6c2bcfe584 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -438,6 +438,20 @@ static inline void reset_apply_error_context_info(void);
 static TransApplyAction get_transaction_apply_action(TransactionId xid,
 													 ParallelApplyWorkerInfo **winfo);
 
+/*
+ * Return the name of the logical replication worker.
+ */
+const char *
+get_worker_name(void)
+{
+	if (am_tablesync_worker())
+		return _("logical replication table synchronization worker");
+	else if (am_parallel_apply_worker())
+		return _("logical replication parallel apply worker");
+	else
+		return _("logical replication apply worker");
+}
+
 /*
  * Form the origin name for the subscription.
  *
@@ -4530,7 +4544,7 @@ start_apply(XLogRecPtr origin_startpos)
  * It starts syncing tables. After a successful sync, sets streaming options
  * and starts streaming to catchup.
  */
-static void
+void
 run_tablesync_worker(WalRcvStreamOptions *options,
 					 char *slotname,
 					 char *originname,
@@ -4788,17 +4802,47 @@ ApplyWorkerMain(Datum main_arg)
 void
 TablesyncWorkerMain(Datum main_arg)
 {
-	int			worker_slot = DatumGetInt32(main_arg);
-	char		originname[NAMEDATALEN];
-	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
-	char	   *myslotname = NULL;
+	int worker_slot = DatumGetInt32(main_arg);
+	char originname[NAMEDATALEN];
+	dsm_handle	handle;
+	dsm_segment *seg;
+	shm_toc    *toc;
+	TablesyncPoolShared *shared;
+
+	XLogRecPtr origin_startpos = InvalidXLogRecPtr;
+	char *myslotname = NULL;
 	WalRcvStreamOptions options;
 
+	/*
+	 * Attach to the dynamic shared memory segment for the table sync pool, and
+	 * find its table of contents.
+	 */
+	memcpy(&handle, MyBgworkerEntry->bgw_extra, sizeof(dsm_handle));
+	seg = dsm_attach(handle);
+	if (!seg)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+				 errmsg("could not map dynamic shared memory segment")));
+
+	toc = shm_toc_attach(PG_LOGICAL_TABLESYNC_POOL_SHM_MAGIC, dsm_segment_address(seg));
+	if (!toc)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+				 errmsg("invalid magic number in dynamic shared memory segment")));
+
+	/* Look up the shared information. */
+	shared = shm_toc_lookup(toc, TABLESYNC_POOL_KEY_SHARED, false);
+	tsp_set_shared(shared);
+
+	shared->child_pid = MyProcPid;
+	tsp_set_exec_state(shared, TS_INIT);
+
 	/* Attach to slot */
 	logicalrep_worker_attach(worker_slot);
 
 	/* Setup signal handling */
 	pqsignal(SIGHUP, SignalHandlerForConfigReload);
+	pqsignal(SIGINT, SignalHandlerForShutdownRequest);
 	pqsignal(SIGTERM, die);
 	BackgroundWorkerUnblockSignals();
 
@@ -4816,17 +4860,13 @@ TablesyncWorkerMain(Datum main_arg)
 
 	InitializeLogRepWorker();
 
-	/* Connect to the origin and start the replication. */
-	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
-		 MySubscription->conninfo);
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
 	 * the subscription relation state.
 	 */
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
 								  invalidate_syncing_table_states,
-								  (Datum) 0);
+								  (Datum)0);
 
 	/*
 	 * The loop where worker does its job. It loops until there is no relation
@@ -4834,81 +4874,47 @@ TablesyncWorkerMain(Datum main_arg)
 	 */
 	for (;;)
 	{
-		List	   *rstates;
-		ListCell   *lc;
-		bool 	is_table_found = false;
+		/* Update worker state for the next table */
+		MyLogicalRepWorker->relid = shared->relid;
+		MyLogicalRepWorker->relstate = shared->relstate;
+		MyLogicalRepWorker->relstate_lsn = shared->relstate_lsn;
+
+		tsp_set_exec_state(shared, TS_IN_PROGRESS);
 
-		run_tablesync_worker(&options,
-							 myslotname,
-							 originname,
-							 sizeof(originname),
-							 &origin_startpos,
-							 worker_slot);
+		PG_TRY();
+		{
+			run_tablesync_worker(&options,
+								myslotname,
+								originname,
+								sizeof(originname),
+								&origin_startpos,
+								worker_slot);
+		}
+		PG_CATCH();
+		{
+			shared->exited = true;
+			PG_RE_THROW();
+		}
+
+		PG_END_TRY();
 
 		if (IsTransactionState())
+		{
 			CommitTransactionCommand();
+			pgstat_report_stat(true);
+		}
 
 		if (MyLogicalRepWorker->is_sync_completed)
 		{
-			/* This transaction will be committed by clean_sync_worker. */
-			StartTransactionCommand();
-
-			/*
-			 * Check if there is any table whose relation state is still INIT.
-			 * If a table in INIT state is found, the worker will not be
-			 * finished, it will be reused instead.
-			 */
-			rstates = GetSubscriptionRelations(MySubscription->oid, true);
-
-			foreach(lc, rstates)
-			{
-				SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-
-				if (rstate->state == SUBREL_STATE_SYNCDONE)
-					continue;
-
-				/*
-				 * Take exclusive lock to prevent any other sync worker from
-				 * picking the same table.
-				 */
-				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
-
-				/*
-				 * Pick the table for the next run if it is not already picked
-				 * up by another worker.
-				 */
-				if (!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
-				{
-					/* Update worker state for the next table */
-					MyLogicalRepWorker->relid = rstate->relid;
-					MyLogicalRepWorker->relstate = rstate->state;
-					MyLogicalRepWorker->relstate_lsn = rstate->lsn;
-					LWLockRelease(LogicalRepWorkerLock);
-
-					/* Found a table for next iteration */
-					is_table_found = true;
-					clean_sync_worker();
-
-					StartTransactionCommand();
-					ereport(LOG,
-							(errmsg("%s for subscription \"%s\" has moved to sync table \"%s\" with relid %u.",
-									get_worker_name(),
-									MySubscription->name,
-									get_rel_name(MyLogicalRepWorker->relid),
-									MyLogicalRepWorker->relid)));
-					CommitTransactionCommand();
-
-					break;
-				}
-				LWLockRelease(LogicalRepWorkerLock);
-			}
-
-			if (!is_table_found)
+			tsp_set_exec_state(shared, TS_DONE);
+			/* wait for apply worker to assign a new table with INIT state. */
+			if (!wait_for_tsp_worker_state_change(TS_INIT, shared->parent_pid,
+												  InvalidOid, 1L, shared))
 				break;
 		}
 	}
 
-	finish_sync_worker();
+	finish_sync_worker(false);
 }
 
 /*
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index af6fd339f7..1484b7b0b9 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -19,6 +19,7 @@
 #include "datatype/timestamp.h"
 #include "miscadmin.h"
 #include "replication/logicalrelation.h"
+#include "replication/walreceiver.h"
 #include "storage/buffile.h"
 #include "storage/fileset.h"
 #include "storage/lock.h"
@@ -213,6 +214,51 @@ typedef struct ParallelApplyWorkerInfo
 	ParallelApplyWorkerShared *shared;
 } ParallelApplyWorkerInfo;
 
+typedef enum TablesyncPoolState
+{
+	TS_FREE,
+	TS_INIT,
+	TS_IN_PROGRESS,
+	TS_DONE,
+	TS_INIT_STOP,
+	TS_STOP
+} TablesyncPoolState;
+
+/*
+ * Struct for sharing information between leader apply worker and parallel
+ * apply workers.
+ */
+typedef struct TablesyncPoolShared
+{
+	slock_t		mutex;
+
+	Oid 		relid;
+	char		relstate;
+	XLogRecPtr	relstate_lsn;
+	bool		exited;
+	pid_t			parent_pid;
+	pid_t			child_pid;
+
+	TablesyncPoolState exec_state;
+} TablesyncPoolShared;
+
+/*
+ * Information which is used to manage the parallel apply worker.
+ */
+typedef struct TablesyncPoolInfo
+{
+	dsm_segment *dsm_seg;
+	TablesyncPoolShared *shared;
+} TablesyncPoolInfo;
+
+
+#define PG_LOGICAL_TABLESYNC_POOL_SHM_MAGIC 0x5450d157
+
+/*
+ * DSM keys for table sync pool.
+ */
+#define TABLESYNC_POOL_KEY_SHARED	1
+
 /* Main memory context for apply worker. Permanent during worker lifetime. */
 extern PGDLLIMPORT MemoryContext ApplyContext;
 
@@ -312,9 +358,16 @@ extern void pa_decr_and_wait_stream_block(void);
 extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
 						   XLogRecPtr remote_lsn);
 
+extern void run_tablesync_worker(WalRcvStreamOptions *options,
+								 char *slotname,
+								 char *originname,
+								 int originname_size,
+								 XLogRecPtr *origin_startpos,
+								 int worker_slot);
+
 #define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)
 
-extern void finish_sync_worker(void);
+extern void finish_sync_worker(bool reuse);
 extern void clean_sync_worker(void);
 
 static inline bool
@@ -336,4 +389,20 @@ am_parallel_apply_worker(void)
 	return isParallelApplyWorker(MyLogicalRepWorker);
 }
 
+extern bool tsp_allocate_free_worker(Oid relid, char state, XLogRecPtr lsn);
+extern TablesyncPoolInfo *tsp_launch_worker(Oid relid);
+extern void tsp_free_all_workers(void);
+extern TablesyncPoolState tsp_get_exec_state(TablesyncPoolShared *wshared);
+extern void tsp_set_exec_state(TablesyncPoolShared *wshared,
+							   TablesyncPoolState exec_state);
+extern void tsp_worker_cleanup(void);
+extern int tsp_get_apply_worker_pid(void);
+extern int tsp_send_signal(int pid);
+extern const char *get_worker_name(void);
+extern void tsp_set_shared(TablesyncPoolShared *shared);
+extern bool wait_for_tsp_worker_state_change(TablesyncPoolState expected_state,
+											 int pid,
+											 Oid relid, int wait_time,
+											 TablesyncPoolShared *shared);
+
 #endif							/* WORKER_INTERNAL_H */
diff --git a/src/include/storage/procsignal.h b/src/include/storage/procsignal.h
index 2f52100b00..55967c3d02 100644
--- a/src/include/storage/procsignal.h
+++ b/src/include/storage/procsignal.h
@@ -36,6 +36,7 @@ typedef enum
 	PROCSIG_BARRIER,			/* global barrier interrupt  */
 	PROCSIG_LOG_MEMORY_CONTEXT, /* ask backend to log the memory contexts */
 	PROCSIG_PARALLEL_APPLY_MESSAGE, /* Message from parallel apply workers */
+	PROCSIG_TABLESYNC_WORKER_POOL, /* parallel tablesync worker pool interrupt */
 
 	/* Recovery conflict reasons */
 	PROCSIG_RECOVERY_CONFLICT_DATABASE,
-- 
2.34.1

Perftest_Results.xlsxapplication/vnd.openxmlformats-officedocument.spreadsheetml.sheet; name=Perftest_Results.xlsxDownload
#107Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#101)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Some review comments for v19-0001

======
src/backend/replication/logical/tablesync.c

1. run_tablesync_worker
+run_tablesync_worker(WalRcvStreamOptions *options,
+ char *slotname,
+ char *originname,
+ int originname_size,
+ XLogRecPtr *origin_startpos)
+{
+ /* Start table synchronization. */
+ start_table_sync(origin_startpos, &slotname);

There was no such comment ("/* Start table synchronization. */") in
the original HEAD code, so I didn't see that it adds much value by
adding it in the refactored code.

~~~

2. LogicalRepSyncTableStart

/*
* Finally, wait until the leader apply worker tells us to catch up and
* then return to let LogicalRepApplyLoop do it.
*/
wait_for_worker_state_change(SUBREL_STATE_CATCHUP);

~

Should LogicalRepApplyLoop still be mentioned here, since that is
static in worker.c? Maybe it is better to refer instead to the common
'start_apply' wrapper? (see also #5a below)

======
src/backend/replication/logical/worker.c

3. set_stream_options

+/*
+ * Sets streaming options including replication slot name and origin start
+ * position. Workers need these options for logical replication.
+ */
+void
+set_stream_options(WalRcvStreamOptions *options,

I'm not sure if the last sentence of the comment is adding anything useful.

~~~

4. start_apply
/*
* Run the apply loop with error handling. Disable the subscription,
* if necessary.
*
* Note that we don't handle FATAL errors which are probably because
* of system resource error and are not repeatable.
*/
void
start_apply(XLogRecPtr origin_startpos)

~

4a.
Somehow I found the function names to be confusing. Intuitively (IMO)
'start_apply' is for apply worker and 'start_tablesync' is for
tablesync worker. But actually, the start_apply() function is the
*common* function for both kinds of worker. Might be easier to
understand if start_apply function name can be changed to indicate it
is really common -- e.g. common_apply_loop(), or similar.

~

4b.
If adverse to changing the function name, it might be helpful anyway
if the function comment can emphasize this function is shared by
different worker types. e.g. "Common function to run the apply
loop..."

~~~

5. run_apply_worker

+ ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+    originname, originname_size);
+
+ /* Setup replication origin tracking. */
+ StartTransactionCommand();

Even if you wish ReplicationOriginNameForLogicalRep() to be outside of
the transaction I thought it should still come *after* the comment,
same as it does in the HEAD code.

~~~

6. ApplyWorkerMain

- /* Run the main loop. */
- start_apply(origin_startpos);
+ /* This is leader apply worker */
+ run_apply_worker(&options, myslotname, originname,
sizeof(originname), &origin_startpos);

proc_exit(0);
}

~

6a.
The comment "/* This is leader apply worker */" is redundant now. This
function is the entry point for leader apply workers so it can't be
anything else.

~

6b.

Caller parameter wrapping differs from the similar code in
TablesyncWorkerMain. Shouldn't they be similar?

e.g.
+ run_apply_worker(&options, myslotname, originname,
sizeof(originname), &origin_startpos);

versus
+ run_tablesync_worker(&options,
+ myslotname,
+ originname,
+ sizeof(originname),
+ &origin_startpos);

======
src/include/replication/worker_internal.h

7.
+
+extern void set_stream_options(WalRcvStreamOptions *options,
+    char *slotname,
+    XLogRecPtr *origin_startpos);
+extern void start_apply(XLogRecPtr origin_startpos);
+extern void DisableSubscriptionAndExit(void);
+

Maybe all the externs belong together? It doesn't seem right for just
these 3 externs to be separated from all the others, with those static
inline functions in-between.

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#108Amit Kapila
amit.kapila16@gmail.com
In reply to: Peter Smith (#107)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Wed, Jul 19, 2023 at 8:38 AM Peter Smith <smithpb2250@gmail.com> wrote:

Some review comments for v19-0001

...

======
src/backend/replication/logical/worker.c

3. set_stream_options

+/*
+ * Sets streaming options including replication slot name and origin start
+ * position. Workers need these options for logical replication.
+ */
+void
+set_stream_options(WalRcvStreamOptions *options,

I'm not sure if the last sentence of the comment is adding anything useful.

Personally, I find it useful as at a high-level it tells the purpose
of setting these options.

~~~

4. start_apply
/*
* Run the apply loop with error handling. Disable the subscription,
* if necessary.
*
* Note that we don't handle FATAL errors which are probably because
* of system resource error and are not repeatable.
*/
void
start_apply(XLogRecPtr origin_startpos)

~

4a.
Somehow I found the function names to be confusing. Intuitively (IMO)
'start_apply' is for apply worker and 'start_tablesync' is for
tablesync worker. But actually, the start_apply() function is the
*common* function for both kinds of worker. Might be easier to
understand if start_apply function name can be changed to indicate it
is really common -- e.g. common_apply_loop(), or similar.

~

4b.
If adverse to changing the function name, it might be helpful anyway
if the function comment can emphasize this function is shared by
different worker types. e.g. "Common function to run the apply
loop..."

I would prefer to change the comments as suggested by you in 4b
because both the workers (apply and tablesync) need to perform apply,
so it seems logical for both of them to invoke start_apply.

--
With Regards,
Amit Kapila.

#109Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#101)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Tue, Jul 18, 2023 at 1:54 AM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi,

PFA updated patches. Rebased 0003 with minor changes. Addressed Peter's reviews for 0001 and 0002 with some small comments below.

Peter Smith <smithpb2250@gmail.com>, 10 Tem 2023 Pzt, 10:09 tarihinde şunu yazdı:

6. LogicalRepApplyLoop

+ /*
+ * apply_dispatch() may have gone into apply_handle_commit()
+ * which can call process_syncing_tables_for_sync.
+ *
+ * process_syncing_tables_for_sync decides whether the sync of
+ * the current table is completed. If it is completed,
+ * streaming must be already ended. So, we can break the loop.
+ */
+ if (MyLogicalRepWorker->is_sync_completed)
+ {
+ endofstream = true;
+ break;
+ }
+

and

+ /*
+ * If is_sync_completed is true, this means that the tablesync
+ * worker is done with synchronization. Streaming has already been
+ * ended by process_syncing_tables_for_sync. We should move to the
+ * next table if needed, or exit.
+ */
+ if (MyLogicalRepWorker->is_sync_completed)
+ endofstream = true;

~

Instead of those code fragments above assigning 'endofstream' as a
side-effect, would it be the same (but tidier) to just modify the
other "breaking" condition below:

BEFORE:
/* Check if we need to exit the streaming loop. */
if (endofstream)
break;

AFTER:
/* Check if we need to exit the streaming loop. */
if (endofstream || MyLogicalRepWorker->is_sync_completed)
break;

First place you mentioned also breaks the infinite loop. Such an if statement is needed there with or without endofstream assignment.

I think if there is a flag to break a loop, using that flag to indicate that we should exit the loop seems more appropriate to me. I see that it would be a bit tidier without endofstream = true lines, but I feel like it would also be less readable.

I don't have a strong opinion though. I'm just keeping them as they are for now, but I can change them if you disagree.

I felt it was slightly sneaky to re-use the existing variable as a
convenient way to do what you want. But, I don’t feel strongly enough
on this point to debate it -- maybe see later if others have an
opinion about this.

10b.
All the other tablesync-related fields of this struct are named as
relXXX, so I wonder if is better for this to follow the same pattern.
e.g. 'relsync_completed'

Aren't those start with rel because they're related to the relation that the tablesync worker is syncing? is_sync_completed is not a relation specific field. I'm okay with changing the name but feel like relsync_completed would be misleading.

My reading of the code is slightly different: Only these fields have
the prefix ‘rel’ and they are all grouped under the comment “/* Used
for initial table synchronization. */” because AFAIK only these fields
are TWS specific (not used for other kinds of workers).

Since this new flag field is also TWS-specific, therefore IMO it
should follow the same consistent name pattern. But, if you are
unconvinced, maybe see later if others have an opinion about it.

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#110Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#104)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Some review comments for patch v20-0002

======
src/backend/replication/logical/tablesync.c

1. finish_sync_worker
/*
* Exit routine for synchronization worker.
*
* If reuse_worker is false, the worker will not be reused and exit.
*/

~

IMO the "will not be reused" part doesn't need saying -- it is
self-evident from the fact "reuse_worker is false".

SUGGESTION
If reuse_worker is false, at the conclusion of this function the
worker process will exit.

~~~

2. finish_sync_worker

- StartTransactionCommand();
- ereport(LOG,
- (errmsg("logical replication table synchronization worker for
subscription \"%s\", table \"%s\" has finished",
- MySubscription->name,
- get_rel_name(MyLogicalRepWorker->relid))));
- CommitTransactionCommand();
-
/* Find the leader apply worker and signal it. */
logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);

- /* Stop gracefully */
- proc_exit(0);
+ if (!reuse_worker)
+ {
+ StartTransactionCommand();
+ ereport(LOG,
+ (errmsg("logical replication table synchronization worker for
subscription \"%s\" has finished",
+ MySubscription->name)));
+ CommitTransactionCommand();
+
+ /* Stop gracefully */
+ proc_exit(0);
+ }

In the HEAD code the log message came *before* it signalled to the
apply leader. Won't it be better to keep the logic in that same order?

~~~

3. process_syncing_tables_for_sync

- finish_sync_worker();
+ /* Sync worker has completed synchronization of the current table. */
+ MyLogicalRepWorker->is_sync_completed = true;
+
+ ereport(LOG,
+ (errmsg("logical replication table synchronization worker for
subscription \"%s\", relation \"%s\" with relid %u has finished",
+ MySubscription->name,
+ get_rel_name(MyLogicalRepWorker->relid),
+ MyLogicalRepWorker->relid)));
+ CommitTransactionCommand();

IIUC it is only the " table synchronization" part that is finished
here; not the whole "table synchronization worker" (compared to
finish_sync_worker function), so maybe the word "worker" should not
be in this message.

~~~

4. TablesyncWorkerMain

+ if (MyLogicalRepWorker->is_sync_completed)
+ {
+ /* tablesync is done unless a table that needs syncning is found */
+ done = true;

SUGGESTION (Typo "syncning" and minor rewording.)
This tablesync worker is 'done' unless another table that needs
syncing is found.

~

5.
+ /* Found a table for next iteration */
+ finish_sync_worker(true);
+
+ StartTransactionCommand();
+ ereport(LOG,
+ (errmsg("logical replication worker for subscription \"%s\" will be
reused to sync table \"%s\" with relid %u.",
+ MySubscription->name,
+ get_rel_name(MyLogicalRepWorker->relid),
+ MyLogicalRepWorker->relid)));
+ CommitTransactionCommand();
+
+ done = false;
+ break;
+ }
+ LWLockRelease(LogicalRepWorkerLock);

5a.
IMO it seems better to put this ereport *inside* the
finish_sync_worker() function alongside the similar log for when the
worker is not reused.

~

5b.
Isn't there a missing call to that LWLockRelease, if the 'break' happens?

======
src/backend/replication/logical/worker.c

6. LogicalRepApplyLoop

Refer to [1]Replies to previous 0002 comments -- /messages/by-id/CAHut+PtiAtGJC52SGNdobOah5ctYDDhWWKd=uP=rkRgXzg5rdg@mail.gmail.com for my reply to a previous review comment

~~~

7. InitializeLogRepWorker

  if (am_tablesync_worker())
  ereport(LOG,
- (errmsg("logical replication worker for subscription \"%s\", table
\"%s\" has started",
+ (errmsg("logical replication worker for subscription \"%s\", table
\"%s\" with relid %u has started",
  MySubscription->name,
- get_rel_name(MyLogicalRepWorker->relid))));
+ get_rel_name(MyLogicalRepWorker->relid),
+ MyLogicalRepWorker->relid)));

But this is certainly a tablesync worker so the message here should
say "logical replication table synchronization worker" like the HEAD
code used to do.

It seems this mistake was introduced in patch v20-0001.

======
src/include/replication/worker_internal.h

8.
Refer to [1]Replies to previous 0002 comments -- /messages/by-id/CAHut+PtiAtGJC52SGNdobOah5ctYDDhWWKd=uP=rkRgXzg5rdg@mail.gmail.com for my reply to a previous review comment

------
[1]: Replies to previous 0002 comments -- /messages/by-id/CAHut+PtiAtGJC52SGNdobOah5ctYDDhWWKd=uP=rkRgXzg5rdg@mail.gmail.com
/messages/by-id/CAHut+PtiAtGJC52SGNdobOah5ctYDDhWWKd=uP=rkRgXzg5rdg@mail.gmail.com

Kind Regards,
Peter Smith.
Fujitsu Australia

#111Amit Kapila
amit.kapila16@gmail.com
In reply to: Peter Smith (#109)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Thu, Jul 20, 2023 at 8:02 AM Peter Smith <smithpb2250@gmail.com> wrote:

On Tue, Jul 18, 2023 at 1:54 AM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi,

PFA updated patches. Rebased 0003 with minor changes. Addressed Peter's reviews for 0001 and 0002 with some small comments below.

Peter Smith <smithpb2250@gmail.com>, 10 Tem 2023 Pzt, 10:09 tarihinde şunu yazdı:

6. LogicalRepApplyLoop

+ /*
+ * apply_dispatch() may have gone into apply_handle_commit()
+ * which can call process_syncing_tables_for_sync.
+ *
+ * process_syncing_tables_for_sync decides whether the sync of
+ * the current table is completed. If it is completed,
+ * streaming must be already ended. So, we can break the loop.
+ */
+ if (MyLogicalRepWorker->is_sync_completed)
+ {
+ endofstream = true;
+ break;
+ }
+

and

+ /*
+ * If is_sync_completed is true, this means that the tablesync
+ * worker is done with synchronization. Streaming has already been
+ * ended by process_syncing_tables_for_sync. We should move to the
+ * next table if needed, or exit.
+ */
+ if (MyLogicalRepWorker->is_sync_completed)
+ endofstream = true;

~

Instead of those code fragments above assigning 'endofstream' as a
side-effect, would it be the same (but tidier) to just modify the
other "breaking" condition below:

BEFORE:
/* Check if we need to exit the streaming loop. */
if (endofstream)
break;

AFTER:
/* Check if we need to exit the streaming loop. */
if (endofstream || MyLogicalRepWorker->is_sync_completed)
break;

First place you mentioned also breaks the infinite loop. Such an if statement is needed there with or without endofstream assignment.

I think if there is a flag to break a loop, using that flag to indicate that we should exit the loop seems more appropriate to me. I see that it would be a bit tidier without endofstream = true lines, but I feel like it would also be less readable.

I don't have a strong opinion though. I'm just keeping them as they are for now, but I can change them if you disagree.

I felt it was slightly sneaky to re-use the existing variable as a
convenient way to do what you want. But, I don’t feel strongly enough
on this point to debate it -- maybe see later if others have an
opinion about this.

I feel it is okay to use the existing variable 'endofstream' here but
shall we have an assertion that it is a tablesync worker?

10b.
All the other tablesync-related fields of this struct are named as
relXXX, so I wonder if is better for this to follow the same pattern.
e.g. 'relsync_completed'

Aren't those start with rel because they're related to the relation that the tablesync worker is syncing? is_sync_completed is not a relation specific field. I'm okay with changing the name but feel like relsync_completed would be misleading.

My reading of the code is slightly different: Only these fields have
the prefix ‘rel’ and they are all grouped under the comment “/* Used
for initial table synchronization. */” because AFAIK only these fields
are TWS specific (not used for other kinds of workers).

Since this new flag field is also TWS-specific, therefore IMO it
should follow the same consistent name pattern. But, if you are
unconvinced, maybe see later if others have an opinion about it.

+1 to use the prefix 'rel' here as the sync is specific to the
relation. Even during apply phase, we will apply the relation-specific
changes. See should_apply_changes_for_rel().

--
With Regards,
Amit Kapila.

#112Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#104)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi, I had a look at the latest 00003 patch (v20-0003).

Although this patch was recently modified, the updates are mostly only
to make it compatible with the updated v20-0002 patch. Specifically,
the v20-0003 updates did not yet address my review comments from
v17-0003 [1]v17-0003 review - /messages/by-id/CAHut+PuMAiO_X_Kw6ud-jr5WOm+rpkdu7CppDU6mu=gY7UVMzQ@mail.gmail.com.

Anyway, this post is just a reminder so the earlier review doesn't get
forgotten.

------
[1]: v17-0003 review - /messages/by-id/CAHut+PuMAiO_X_Kw6ud-jr5WOm+rpkdu7CppDU6mu=gY7UVMzQ@mail.gmail.com
/messages/by-id/CAHut+PuMAiO_X_Kw6ud-jr5WOm+rpkdu7CppDU6mu=gY7UVMzQ@mail.gmail.com

Kind Regards,
Peter Smith.
Fujitsu Australia

#113Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Peter Smith (#112)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Peter,

Peter Smith <smithpb2250@gmail.com>, 20 Tem 2023 Per, 07:10 tarihinde şunu
yazdı:

Hi, I had a look at the latest 00003 patch (v20-0003).

Although this patch was recently modified, the updates are mostly only
to make it compatible with the updated v20-0002 patch. Specifically,
the v20-0003 updates did not yet address my review comments from
v17-0003 [1].

Yes, I only addressed your reviews for 0001 and 0002, and rebased 0003 in
latest patches as stated here [1]/messages/by-id/CAGPVpCTvALKEXe0=N-+iMmVxVQ-+P8KZ_1qQ1KsSSZ-V9wJ5hw@mail.gmail.com.

I'll update the patch soon according to recent reviews, including yours for
0003.

[1]: /messages/by-id/CAGPVpCTvALKEXe0=N-+iMmVxVQ-+P8KZ_1qQ1KsSSZ-V9wJ5hw@mail.gmail.com
/messages/by-id/CAGPVpCTvALKEXe0=N-+iMmVxVQ-+P8KZ_1qQ1KsSSZ-V9wJ5hw@mail.gmail.com

Thanks for the reminder.
--
Melih Mutlu
Microsoft

#114Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Peter Smith (#110)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi,

Peter Smith <smithpb2250@gmail.com>, 20 Tem 2023 Per, 05:41 tarihinde şunu
yazdı:

7. InitializeLogRepWorker

if (am_tablesync_worker())
ereport(LOG,
- (errmsg("logical replication worker for subscription \"%s\", table
\"%s\" has started",
+ (errmsg("logical replication worker for subscription \"%s\", table
\"%s\" with relid %u has started",
MySubscription->name,
- get_rel_name(MyLogicalRepWorker->relid))));
+ get_rel_name(MyLogicalRepWorker->relid),
+ MyLogicalRepWorker->relid)));

But this is certainly a tablesync worker so the message here should
say "logical replication table synchronization worker" like the HEAD
code used to do.

It seems this mistake was introduced in patch v20-0001.

I'm a bit confused here. Isn't it decided to use "logical replication
worker" regardless of the worker's type [1]/messages/by-id/CAHut+Pt1xwATviPGjjtJy5L631SGf3qjV9XUCmxLu16cHamfgg@mail.gmail.com. That's why I made this change.
If that's not the case here, I'll put it back.

[1]: /messages/by-id/CAHut+Pt1xwATviPGjjtJy5L631SGf3qjV9XUCmxLu16cHamfgg@mail.gmail.com
/messages/by-id/CAHut+Pt1xwATviPGjjtJy5L631SGf3qjV9XUCmxLu16cHamfgg@mail.gmail.com

Thanks,
--
Melih Mutlu
Microsoft

#115Amit Kapila
amit.kapila16@gmail.com
In reply to: Melih Mutlu (#114)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Thu, Jul 20, 2023 at 5:12 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Peter Smith <smithpb2250@gmail.com>, 20 Tem 2023 Per, 05:41 tarihinde şunu yazdı:

7. InitializeLogRepWorker

if (am_tablesync_worker())
ereport(LOG,
- (errmsg("logical replication worker for subscription \"%s\", table
\"%s\" has started",
+ (errmsg("logical replication worker for subscription \"%s\", table
\"%s\" with relid %u has started",
MySubscription->name,
- get_rel_name(MyLogicalRepWorker->relid))));
+ get_rel_name(MyLogicalRepWorker->relid),
+ MyLogicalRepWorker->relid)));

But this is certainly a tablesync worker so the message here should
say "logical replication table synchronization worker" like the HEAD
code used to do.

It seems this mistake was introduced in patch v20-0001.

I'm a bit confused here. Isn't it decided to use "logical replication worker" regardless of the worker's type [1]. That's why I made this change. If that's not the case here, I'll put it back.

I feel where the worker type is clear, it is better to use it unless
the same can lead to translation issues.

--
With Regards,
Amit Kapila.

#116Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Amit Kapila (#115)
3 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi,

Attached the updated patches with recent reviews addressed.

See below for my comments:

Peter Smith <smithpb2250@gmail.com>, 19 Tem 2023 Çar, 06:08 tarihinde şunu
yazdı:

Some review comments for v19-0001

2. LogicalRepSyncTableStart

/*
* Finally, wait until the leader apply worker tells us to catch up and
* then return to let LogicalRepApplyLoop do it.
*/
wait_for_worker_state_change(SUBREL_STATE_CATCHUP);

~

Should LogicalRepApplyLoop still be mentioned here, since that is
static in worker.c? Maybe it is better to refer instead to the common
'start_apply' wrapper? (see also #5a below)

Isn't' LogicalRepApplyLoop static on HEAD and also mentioned in the exact
comment in tablesync.c while the common "start_apply" function also exists?
I'm not sure how such a change would be related to this patch.

---

5.

+ /* Found a table for next iteration */
+ finish_sync_worker(true);
+
+ StartTransactionCommand();
+ ereport(LOG,
+ (errmsg("logical replication worker for subscription \"%s\" will be
reused to sync table \"%s\" with relid %u.",
+ MySubscription->name,
+ get_rel_name(MyLogicalRepWorker->relid),
+ MyLogicalRepWorker->relid)));
+ CommitTransactionCommand();
+
+ done = false;
+ break;
+ }
+ LWLockRelease(LogicalRepWorkerLock);

5b.
Isn't there a missing call to that LWLockRelease, if the 'break' happens?

Lock is already released before break, if that's the lock you meant:

/* Update worker state for the next table */

MyLogicalRepWorker->relid = rstate->relid;
MyLogicalRepWorker->relstate = rstate->state;
MyLogicalRepWorker->relstate_lsn = rstate->lsn;
LWLockRelease(LogicalRepWorkerLock);

/* Found a table for next iteration */
finish_sync_worker(true);
done = false;
break;

---

2.

As for the publisher node, this patch allows to reuse logical
walsender processes
after the streaming is done once.

~

Is this paragraph even needed? Since the connection is reused then it
already implies the other end (the Wlasender) is being reused, right?

I actually see no harm in explaining this explicitly.

Thanks,
--
Melih Mutlu
Microsoft

Attachments:

v21-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchapplication/octet-stream; name=v21-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchDownload
From e8954bd0f0aeccd911d6dffe9e39c1636634218a Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Mon, 5 Jun 2023 15:04:41 +0300
Subject: [PATCH v21 1/5] Refactor to split Apply and Tablesync Workers

Both apply and tablesync workers were using ApplyWorkerMain() as entry
point. As the name implies, ApplyWorkerMain() should be considered as
the main function for apply workers. Tablesync worker's path was hidden
and does not have enough in common to share the same main function with
apply worker.

Also, most of the code shared by both worker types is already combined
in LogicalRepApplyLoop(). There is no need to combine the rest in
ApplyWorkerMain() anymore.

This patch introduces TablesyncWorkerMain() as a new entry point for
tablesync workers. This aims to increase code readability and help to
the upcoming reuse tablesync worker improvements.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/postmaster/bgworker.c             |   3 +
 .../replication/logical/applyparallelworker.c |   2 +-
 src/backend/replication/logical/launcher.c    |  32 +-
 src/backend/replication/logical/tablesync.c   |  98 ++++-
 src/backend/replication/logical/worker.c      | 386 ++++++++----------
 src/include/replication/logicalworker.h       |   1 +
 src/include/replication/worker_internal.h     |  11 +-
 7 files changed, 307 insertions(+), 226 deletions(-)

diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 5b4bd71694..505e38376c 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -131,6 +131,9 @@ static const struct
 	},
 	{
 		"ParallelApplyWorkerMain", ParallelApplyWorkerMain
+	},
+	{
+		"TablesyncWorkerMain", TablesyncWorkerMain
 	}
 };
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 6fb96148f4..1d4e83c4c1 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -942,7 +942,7 @@ ParallelApplyWorkerMain(Datum main_arg)
 	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
 		MyLogicalRepWorker->reply_time = 0;
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 542af7d863..e231fa7f95 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -459,24 +459,30 @@ retry:
 	snprintf(bgw.bgw_library_name, MAXPGPATH, "postgres");
 
 	if (is_parallel_apply_worker)
+	{
 		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ParallelApplyWorkerMain");
-	else
-		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
-
-	if (OidIsValid(relid))
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication worker for subscription %u sync %u", subid, relid);
-	else if (is_parallel_apply_worker)
+				 "logical replication parallel apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
+	}
+	else if (OidIsValid(relid))
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "TablesyncWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication parallel apply worker for subscription %u", subid);
+				 "logical replication tablesync worker for subscription %u sync %u",
+				 subid,
+				 relid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication tablesync worker");
+	}
 	else
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication apply worker for subscription %u", subid);
-
-	if (is_parallel_apply_worker)
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
-	else
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication worker");
+				 "logical replication apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication apply worker");
+	}
 
 	bgw.bgw_restart_time = BGW_NEVER_RESTART;
 	bgw.bgw_notify_pid = MyProcPid;
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 6d461654ab..729f48a3b5 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -104,17 +104,21 @@
 #include "nodes/makefuncs.h"
 #include "parser/parse_relation.h"
 #include "pgstat.h"
+#include "postmaster/interrupt.h"
 #include "replication/logicallauncher.h"
 #include "replication/logicalrelation.h"
+#include "replication/logicalworker.h"
 #include "replication/walreceiver.h"
 #include "replication/worker_internal.h"
 #include "replication/slot.h"
 #include "replication/origin.h"
 #include "storage/ipc.h"
 #include "storage/lmgr.h"
+#include "tcop/tcopprot.h"
 #include "utils/acl.h"
 #include "utils/array.h"
 #include "utils/builtins.h"
+#include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
 #include "utils/rls.h"
@@ -1241,7 +1245,7 @@ ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
  *
  * The returned slot name is palloc'ed in current memory context.
  */
-char *
+static char *
 LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 {
 	char	   *slotname;
@@ -1584,6 +1588,98 @@ FetchTableStates(bool *started_tx)
 	return has_subrels;
 }
 
+/*
+ * Execute the initial sync with error handling. Disable the subscription,
+ * if it's required.
+ *
+ * Allocate the slot name in long-lived context on return. Note that we don't
+ * handle FATAL errors which are probably because of system resource error and
+ * are not repeatable.
+ */
+static void
+start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+{
+	char	   *syncslotname = NULL;
+
+	Assert(am_tablesync_worker());
+
+	PG_TRY();
+	{
+		/* Call initial sync. */
+		syncslotname = LogicalRepSyncTableStart(origin_startpos);
+	}
+	PG_CATCH();
+	{
+		if (MySubscription->disableonerr)
+			DisableSubscriptionAndExit();
+		else
+		{
+			/*
+			 * Report the worker failed during table synchronization. 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();
+
+	/* allocate slot name in long-lived context */
+	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+	pfree(syncslotname);
+}
+
+/*
+ * Runs the tablesync worker.
+ *
+ * It starts syncing tables. After a successful sync, sets streaming options
+ * and starts streaming to catchup.
+ */
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+					 char *slotname,
+					 char *originname,
+					 int originname_size,
+					 XLogRecPtr *origin_startpos)
+{
+	start_table_sync(origin_startpos, &slotname);
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   originname_size);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	/* Start applying changes to catchup. */
+	start_apply(*origin_startpos);
+}
+
+/* Logical Replication Tablesync worker entry point */
+void
+TablesyncWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *myslotname = NULL;
+	WalRcvStreamOptions options;
+
+	StartLogRepWorker(worker_slot);
+
+	run_tablesync_worker(&options, myslotname, originname,
+						 sizeof(originname), &origin_startpos);
+
+	finish_sync_worker();
+}
+
 /*
  * If the subscription has no tables then return false.
  *
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index dd353fd1cb..3b8976f717 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -395,8 +395,6 @@ static void stream_close_file(void);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
-static void DisableSubscriptionAndExit(void);
-
 static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
 static void apply_handle_insert_internal(ApplyExecutionData *edata,
 										 ResultRelInfo *relinfo,
@@ -4313,6 +4311,71 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+/*
+ * Sets streaming options including replication slot name and origin start
+ * position. Workers need these options for logical replication.
+ */
+void
+set_stream_options(WalRcvStreamOptions *options,
+				   char *slotname,
+				   XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (!am_tablesync_worker() &&
+		MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
+		options->proto.logical.twophase = true;
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4347,24 +4410,18 @@ TwoPhaseTransactionGid(Oid subid, TransactionId xid, char *gid, int szgid)
 }
 
 /*
- * Execute the initial sync with error handling. Disable the subscription,
- * if it's required.
+ * Common function to run the apply loop with error handling. Disable the
+ * subscription, if necessary.
  *
- * Allocate the slot name in long-lived context on return. Note that we don't
- * handle FATAL errors which are probably because of system resource error and
- * are not repeatable.
+ * Note that we don't handle FATAL errors which are probably because
+ * of system resource error and are not repeatable.
  */
-static void
-start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+void
+start_apply(XLogRecPtr origin_startpos)
 {
-	char	   *syncslotname = NULL;
-
-	Assert(am_tablesync_worker());
-
 	PG_TRY();
 	{
-		/* Call initial sync. */
-		syncslotname = LogicalRepSyncTableStart(origin_startpos);
+		LogicalRepApplyLoop(origin_startpos);
 	}
 	PG_CATCH();
 	{
@@ -4373,65 +4430,117 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 		else
 		{
 			/*
-			 * Report the worker failed during table synchronization. Abort
-			 * the current transaction so that the stats message is sent in an
+			 * Report the worker failed while applying changes. Abort the
+			 * current transaction so that the stats message is sent in an
 			 * idle state.
 			 */
 			AbortOutOfAnyTransaction();
-			pgstat_report_subscription_error(MySubscription->oid, false);
+			pgstat_report_subscription_error(MySubscription->oid, !am_tablesync_worker());
 
 			PG_RE_THROW();
 		}
 	}
 	PG_END_TRY();
-
-	/* allocate slot name in long-lived context */
-	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
-	pfree(syncslotname);
 }
 
 /*
- * Run the apply loop with error handling. Disable the subscription,
- * if necessary.
+ * Runs the leader apply worker.
  *
- * Note that we don't handle FATAL errors which are probably because
- * of system resource error and are not repeatable.
+ * It sets up replication origin, streaming options and then starts streaming.
  */
 static void
-start_apply(XLogRecPtr origin_startpos)
+run_apply_worker(WalRcvStreamOptions *options,
+				 char *slotname,
+				 char *originname,
+				 int originname_size,
+				 XLogRecPtr *origin_startpos)
 {
-	PG_TRY();
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+	bool		must_use_password;
+
+	slotname = MySubscription->slotname;
+
+	/*
+	 * This shouldn't happen if the subscription is enabled, but guard
+	 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+	 * crash if slot is NULL.)
+	 */
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	/* Setup replication origin tracking. */
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, originname_size);
+	StartTransactionCommand();
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	*origin_startpos = replorigin_session_get_progress(false);
+
+	/* Is the use of a password mandatory? */
+	must_use_password = MySubscription->passwordrequired &&
+		!superuser_arg(MySubscription->owner);
+
+	/* Note that the superuser_arg call can access the DB */
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											must_use_password,
+											MySubscription->name, &err);
+
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+	 * We don't really use the output identify_system for anything but it
+	 * does some initializations on the upstream so let's still call it.
+	 */
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(options, slotname, origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
 	{
-		LogicalRepApplyLoop(origin_startpos);
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
 	}
-	PG_CATCH();
-	{
-		if (MySubscription->disableonerr)
-			DisableSubscriptionAndExit();
-		else
-		{
-			/*
-			 * Report the worker failed while applying changes. Abort the
-			 * current transaction so that the stats message is sent in an
-			 * idle state.
-			 */
-			AbortOutOfAnyTransaction();
-			pgstat_report_subscription_error(MySubscription->oid, !am_tablesync_worker());
 
-			PG_RE_THROW();
-		}
-	}
-	PG_END_TRY();
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+
+	/* Run the main loop. */
+	start_apply(*origin_startpos);
 }
 
 /*
- * Common initialization for leader apply worker and parallel apply worker.
+ * Common initialization for leader apply worker, parallel apply worker and
+ * tablesync worker.
  *
  * Initialize the database connection, in-memory subscription and necessary
  * config options.
  */
 void
-InitializeApplyWorker(void)
+InitializeLogRepWorker(void)
 {
 	MemoryContext oldctx;
 
@@ -4493,7 +4602,7 @@ InitializeApplyWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has started",
+				(errmsg("logical replication worker for subscription \"%s\", table \"%s\" has started",
 						MySubscription->name,
 						get_rel_name(MyLogicalRepWorker->relid))));
 	else
@@ -4504,19 +4613,10 @@ InitializeApplyWorker(void)
 	CommitTransactionCommand();
 }
 
-/* Logical Replication Apply worker entry point */
+/* Common function to start the leader apply or tablesync worker. */
 void
-ApplyWorkerMain(Datum main_arg)
+StartLogRepWorker(int worker_slot)
 {
-	int			worker_slot = DatumGetInt32(main_arg);
-	char		originname[NAMEDATALEN];
-	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
-	char	   *myslotname = NULL;
-	WalRcvStreamOptions options;
-	int			server_version;
-
-	InitializingApplyWorker = true;
-
 	/* Attach to slot */
 	logicalrep_worker_attach(worker_slot);
 
@@ -4537,79 +4637,12 @@ ApplyWorkerMain(Datum main_arg)
 	/* Load the libpq-specific functions */
 	load_file("libpqwalreceiver", false);
 
-	InitializeApplyWorker();
-
-	InitializingApplyWorker = false;
+	InitializeLogRepWorker();
 
 	/* Connect to the origin and start the replication. */
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-		bool		must_use_password;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-
-		/* Is the use of a password mandatory? */
-		must_use_password = MySubscription->passwordrequired &&
-			!superuser_arg(MySubscription->owner);
-
-		/* Note that the superuser_arg call can access the DB */
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												must_use_password,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
 	 * the subscription relation state.
@@ -4617,91 +4650,26 @@ ApplyWorkerMain(Datum main_arg)
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
+}
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
-
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
-
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-
-	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
-	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
+/* Logical Replication Apply worker entry point */
+void
+ApplyWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *myslotname = NULL;
+	WalRcvStreamOptions options;
 
-	if (!am_tablesync_worker())
-	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
-		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+	InitializingApplyWorker = true;
 
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
-		}
-		else
-		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-		}
+	StartLogRepWorker(worker_slot);
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-								 MySubscription->name,
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-								 "?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
+	InitializingApplyWorker = false;
 
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+	run_apply_worker(&options, myslotname, originname,
+					 sizeof(originname), &origin_startpos);
 
 	proc_exit(0);
 }
@@ -4710,7 +4678,7 @@ ApplyWorkerMain(Datum main_arg)
  * After error recovery, disable the subscription in a new transaction
  * and exit cleanly.
  */
-static void
+void
 DisableSubscriptionAndExit(void)
 {
 	/*
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 39588da79f..bbd71d0b42 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -18,6 +18,7 @@ extern PGDLLIMPORT volatile sig_atomic_t ParallelApplyMessagePending;
 
 extern void ApplyWorkerMain(Datum main_arg);
 extern void ParallelApplyWorkerMain(Datum main_arg);
+extern void TablesyncWorkerMain(Datum main_arg);
 
 extern bool IsLogicalWorker(void);
 extern bool IsLogicalParallelApplyWorker(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 343e781896..9012af38cd 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -19,6 +19,7 @@
 #include "datatype/timestamp.h"
 #include "miscadmin.h"
 #include "replication/logicalrelation.h"
+#include "replication/walreceiver.h"
 #include "storage/buffile.h"
 #include "storage/fileset.h"
 #include "storage/lock.h"
@@ -243,7 +244,6 @@ extern int	logicalrep_sync_worker_count(Oid subid);
 
 extern void ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 											   char *originname, Size szoriginname);
-extern char *LogicalRepSyncTableStart(XLogRecPtr *origin_startpos);
 
 extern bool AllTablesyncsReady(void);
 extern void UpdateTwoPhaseState(Oid suboid, char new_state);
@@ -265,7 +265,7 @@ extern void maybe_reread_subscription(void);
 
 extern void stream_cleanup_files(Oid subid, TransactionId xid);
 
-extern void InitializeApplyWorker(void);
+extern void InitializeLogRepWorker(void);
 
 extern void store_flush_position(XLogRecPtr remote_lsn, XLogRecPtr local_lsn);
 
@@ -304,6 +304,13 @@ extern void pa_decr_and_wait_stream_block(void);
 
 extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
 						   XLogRecPtr remote_lsn);
+extern void set_stream_options(WalRcvStreamOptions *options,
+							   char *slotname,
+							   XLogRecPtr *origin_startpos);
+
+extern void start_apply(XLogRecPtr origin_startpos);
+extern void DisableSubscriptionAndExit(void);
+extern void StartLogRepWorker(int worker_slot);
 
 #define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)
 
-- 
2.25.1

v21-0002-Reuse-Tablesync-Workers.patchapplication/octet-stream; name=v21-0002-Reuse-Tablesync-Workers.patchDownload
From 60db4932687bffd16e291638256d844a6ddc12d0 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:04:46 +0300
Subject: [PATCH v21 2/5] Reuse Tablesync Workers

Before this patch, tablesync workers were capable of syncing only one
table. For each table, a new sync worker was launched and that worker would
exit when done processing the table.

Now, tablesync workers are not limited to processing only one
table. When done, they can move to processing another table in
the same subscription.

If there is a table that needs to be synced, an available tablesync
worker picks up that table and syncs it. Each tablesync worker
continues to pick new tables to sync until there are no tables left
requiring synchronization. If there was no available worker to
process the table, then a new tablesync worker will be launched,
provided the number of tablesync workers for the subscription does not
exceed max_sync_workers_per_subscription.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/replication/logical/launcher.c  |   1 +
 src/backend/replication/logical/tablesync.c | 124 ++++++++++++++++++--
 src/backend/replication/logical/worker.c    |  38 +++++-
 src/include/replication/worker_internal.h   |   6 +
 4 files changed, 153 insertions(+), 16 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e231fa7f95..25dd06b8af 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -440,6 +440,7 @@ retry:
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
+	worker->relsync_completed = false;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 729f48a3b5..3eaa47119a 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -134,10 +134,12 @@ static StringInfo copybuf = NULL;
 
 /*
  * Exit routine for synchronization worker.
+ *
+ * If reuse_worker is false, at the conclusion of this function the worker
+ * process will exit.
  */
 static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+finish_sync_worker(bool reuse_worker)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -149,21 +151,42 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
+	/*
+	 * Disconnect from the publisher otherwise reusing the sync worker can
+	 * error due to exceeding max_wal_senders.
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		walrcv_disconnect(LogRepWorkerWalRcvConn);
+		LogRepWorkerWalRcvConn = NULL;
+	}
+
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
 	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
+	if (!reuse_worker)
+	{
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\" has finished",
+						MySubscription->name)));
+	}
+	else
+	{
+		ereport(LOG,
+				(errmsg("logical replication worker for subscription \"%s\" will be reused to sync table \"%s\" with relid %u.",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
+	}
 	CommitTransactionCommand();
 
 	/* Find the leader apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
 
 	/* Stop gracefully */
-	proc_exit(0);
+	if (!reuse_worker)
+		proc_exit(0);
 }
 
 /*
@@ -383,7 +406,15 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 */
 		replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
+		/* Sync worker has completed synchronization of the current table. */
+		MyLogicalRepWorker->relsync_completed = true;
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization for subscription \"%s\", relation \"%s\" with relid %u has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
+		CommitTransactionCommand();
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -1288,7 +1319,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		case SUBREL_STATE_SYNCDONE:
 		case SUBREL_STATE_READY:
 		case SUBREL_STATE_UNKNOWN:
-			finish_sync_worker();	/* doesn't return */
+			finish_sync_worker(false);	/* doesn't return */
 	}
 
 	/* Calculate the name of the tablesync slot. */
@@ -1645,6 +1676,9 @@ run_tablesync_worker(WalRcvStreamOptions *options,
 					 int originname_size,
 					 XLogRecPtr *origin_startpos)
 {
+	MyLogicalRepWorker->relsync_completed = false;
+
+	/* Start table synchronization. */
 	start_table_sync(origin_startpos, &slotname);
 
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
@@ -1671,13 +1705,79 @@ TablesyncWorkerMain(Datum main_arg)
 	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
 	char	   *myslotname = NULL;
 	WalRcvStreamOptions options;
+	bool 		done = false;
 
 	StartLogRepWorker(worker_slot);
 
-	run_tablesync_worker(&options, myslotname, originname,
-						 sizeof(originname), &origin_startpos);
+	/*
+	 * The loop where worker does its job. It loops until there is no relation
+	 * left to sync.
+	 */
+	for (;!done;)
+	{
+		List	   *rstates;
+		ListCell   *lc;
+
+		run_tablesync_worker(&options, myslotname, originname,
+							sizeof(originname), &origin_startpos);
+
+		if (IsTransactionState())
+			CommitTransactionCommand();
+
+		if (MyLogicalRepWorker->relsync_completed)
+		{
+			/*
+			 * This tablesync worker is 'done' unless another table that needs
+			 * syncing is found.
+			 */
+			done = true;
+
+			/* This transaction will be committed by finish_sync_worker. */
+			StartTransactionCommand();
+
+			/*
+			 * Check if there is any table whose relation state is still INIT.
+			 * If a table in INIT state is found, the worker will not be
+			 * finished, it will be reused instead.
+			 */
+			rstates = GetSubscriptionRelations(MySubscription->oid, true);
+
+			foreach(lc, rstates)
+			{
+				SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
+
+				if (rstate->state == SUBREL_STATE_SYNCDONE)
+					continue;
+
+				/*
+				 * Take exclusive lock to prevent any other sync worker from
+				 * picking the same table.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
+
+				/*
+				 * Pick the table for the next run if it is not already picked
+				 * up by another worker.
+				 */
+				if (!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+				{
+					/* Update worker state for the next table */
+					MyLogicalRepWorker->relid = rstate->relid;
+					MyLogicalRepWorker->relstate = rstate->state;
+					MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+					LWLockRelease(LogicalRepWorkerLock);
+
+					/* Found a table for next iteration */
+					finish_sync_worker(true);
+					done = false;
+					break;
+				}
+				LWLockRelease(LogicalRepWorkerLock);
+			}
+		}
+	}
 
-	finish_sync_worker();
+	finish_sync_worker(false);
 }
 
 /*
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 3b8976f717..7442fd308d 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3607,6 +3607,21 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				/*
+				 * apply_dispatch() may have gone into apply_handle_commit()
+				 * which can call process_syncing_tables_for_sync.
+				 *
+				 * process_syncing_tables_for_sync decides whether the sync of
+				 * the current table is completed. If it is completed,
+				 * streaming must be already ended. So, we can break the loop.
+				 */
+				if (am_tablesync_worker() &&
+					MyLogicalRepWorker->relsync_completed)
+				{
+					endofstream = true;
+					break;
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -3626,6 +3641,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+
+			/*
+			 * If relsync_completed is true, this means that the tablesync
+			 * worker is done with synchronization. Streaming has already been
+			 * ended by process_syncing_tables_for_sync. We should move to the
+			 * next table if needed, or exit.
+			 */
+			if (am_tablesync_worker() &&
+				MyLogicalRepWorker->relsync_completed)
+				endofstream = true;
 		}
 
 		/* Cleanup the memory. */
@@ -3728,8 +3753,12 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * End streaming here for only apply workers. Ending streaming for
+	 * tablesync workers is deferred until the worker exits its main loop.
+	 */
+	if (!am_tablesync_worker())
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
 }
 
 /*
@@ -4602,9 +4631,10 @@ InitializeLogRepWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("logical replication worker for subscription \"%s\", table \"%s\" has started",
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" with relid %u has started",
 						MySubscription->name,
-						get_rel_name(MyLogicalRepWorker->relid))));
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
 	else
 		ereport(LOG,
 				(errmsg("logical replication apply worker for subscription \"%s\" has started",
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 9012af38cd..f552ecbc09 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -57,6 +57,12 @@ typedef struct LogicalRepWorker
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
 
+	/*
+	 * Indicates whether tablesync worker has completed syncing its assigned
+	 * table.
+	 */
+	bool		relsync_completed;
+
 	/*
 	 * Used to create the changes and subxact files for the streaming
 	 * transactions.  Upon the arrival of the first streaming transaction or
-- 
2.25.1

v21-0003-Reuse-connection-when-tablesync-workers-change-t.patchapplication/octet-stream; name=v21-0003-Reuse-connection-when-tablesync-workers-change-t.patchDownload
From 356e6ff2f38775a14e7ec8c60507c520f5c32bae Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:13:52 +0300
Subject: [PATCH v21 3/5] Reuse connection when tablesync workers change the
 target

Previously tablesync workers establish new connections when it changes the syncing
table, but this might have additional overhead. This patch allows the existing
connection to be reused.

As for the publisher node, this patch allows to reuse logical walsender processes
after the streaming is done once.
---
 src/backend/replication/logical/launcher.c  |  1 +
 src/backend/replication/logical/tablesync.c | 60 ++++++++++++++-------
 src/backend/replication/logical/worker.c    | 18 ++++---
 src/backend/replication/walsender.c         |  7 +++
 src/include/replication/worker_internal.h   |  3 ++
 5 files changed, 62 insertions(+), 27 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 25dd06b8af..657e446eaf 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
 	worker->relsync_completed = false;
+	worker->slotnum = slot;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 3eaa47119a..cdddff1923 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -151,16 +151,6 @@ finish_sync_worker(bool reuse_worker)
 		pgstat_report_stat(true);
 	}
 
-	/*
-	 * Disconnect from the publisher otherwise reusing the sync worker can
-	 * error due to exceeding max_wal_senders.
-	 */
-	if (LogRepWorkerWalRcvConn != NULL)
-	{
-		walrcv_disconnect(LogRepWorkerWalRcvConn);
-		LogRepWorkerWalRcvConn = NULL;
-	}
-
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
@@ -1268,6 +1258,27 @@ ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
 			 relid, GetSystemIdentifier());
 }
 
+/*
+ * Determine the application_name for tablesync workers.
+ *
+ * Previously, the replication slot name was used as application_name. Since
+ * it's possible to reuse tablesync workers now, a tablesync worker can handle
+ * several different replication slots during its lifetime. Therefore, we
+ * cannot use the slot name as application_name anymore. Instead, the slot
+ * number of the tablesync worker is used as a part of the application_name.
+ *
+ * FIXME: if the tablesync worker starts to reuse the replication slot during
+ * synchronization, we should again use the replication slot name as
+ * application_name.
+ */
+static void
+ApplicationNameForTablesync(Oid suboid, int worker_slot,
+							char *application_name, Size szapp)
+{
+	snprintf(application_name, szapp, "pg_%u_sync_%i_" UINT64_FORMAT, suboid,
+			 worker_slot, GetSystemIdentifier());
+}
+
 /*
  * Start syncing the table in the sync worker.
  *
@@ -1329,15 +1340,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 									slotname,
 									NAMEDATALEN);
 
-	/*
-	 * Here we use the slot name instead of the subscription name as the
-	 * application_name, so that it is different from the leader apply worker,
-	 * so that synchronous replication can distinguish them.
-	 */
-	LogRepWorkerWalRcvConn =
-		walrcv_connect(MySubscription->conninfo, true,
-					   must_use_password,
-					   slotname, &err);
+	/* Connect to the publisher if haven't done so already. */
+	if (LogRepWorkerWalRcvConn == NULL)
+	{
+		char application_name[NAMEDATALEN];
+
+		/*
+		 * The application_name must differ from the subscription name (used by
+		 * the leader apply worker) because synchronous replication has to be
+		 * able to distinguish this worker from the leader apply worker.
+		 */
+		ApplicationNameForTablesync(MySubscription->oid,
+									MyLogicalRepWorker->slotnum,
+									application_name,
+									NAMEDATALEN);
+		LogRepWorkerWalRcvConn =
+			walrcv_connect(MySubscription->conninfo, true,
+						   must_use_password,
+						   application_name, &err);
+	}
+
 	if (LogRepWorkerWalRcvConn == NULL)
 		ereport(ERROR,
 				(errcode(ERRCODE_CONNECTION_FAILURE),
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 7442fd308d..9338c51248 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3480,20 +3480,22 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	ErrorContextCallback errcallback;
 
 	/*
-	 * Init the ApplyMessageContext which we clean up after each replication
-	 * protocol message.
+	 * Init the ApplyMessageContext if needed. This context is cleaned up
+	 * after each replication protocol message.
 	 */
-	ApplyMessageContext = AllocSetContextCreate(ApplyContext,
-												"ApplyMessageContext",
-												ALLOCSET_DEFAULT_SIZES);
+	if (!ApplyMessageContext)
+		ApplyMessageContext = AllocSetContextCreate(ApplyContext,
+													"ApplyMessageContext",
+													ALLOCSET_DEFAULT_SIZES);
 
 	/*
 	 * This memory context is used for per-stream data when the streaming mode
 	 * is enabled. This context is reset on each stream stop.
 	 */
-	LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
-													"LogicalStreamingContext",
-													ALLOCSET_DEFAULT_SIZES);
+	if (!LogicalStreamingContext)
+		LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
+														"LogicalStreamingContext",
+														ALLOCSET_DEFAULT_SIZES);
 
 	/* mark as idle, before starting to loop */
 	pgstat_report_activity(STATE_IDLE, NULL);
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index d27ef2985d..2f3e93cc40 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1830,7 +1830,14 @@ exec_replication_command(const char *cmd_string)
 				if (cmd->kind == REPLICATION_KIND_PHYSICAL)
 					StartReplication(cmd);
 				else
+				{
+					/*
+					 * Reset flags because reusing tablesync workers can mean
+					 * this is the second time here.
+					 */
+					streamingDoneSending = streamingDoneReceiving = false;
 					StartLogicalReplication(cmd);
+				}
 
 				/* dupe, but necessary per libpqrcv_endstreaming */
 				EndReplicationCommand(cmdtag);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index f552ecbc09..4a94b13423 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -39,6 +39,9 @@ typedef struct LogicalRepWorker
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
+	/* Slot number of this worker. */
+	int			slotnum;
+
 	/* Pointer to proc array. NULL if not running. */
 	PGPROC	   *proc;
 
-- 
2.25.1

#117Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#116)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Thu, Jul 20, 2023 at 11:41 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi,

Attached the updated patches with recent reviews addressed.

See below for my comments:

Peter Smith <smithpb2250@gmail.com>, 19 Tem 2023 Çar, 06:08 tarihinde şunu yazdı:

Some review comments for v19-0001

2. LogicalRepSyncTableStart

/*
* Finally, wait until the leader apply worker tells us to catch up and
* then return to let LogicalRepApplyLoop do it.
*/
wait_for_worker_state_change(SUBREL_STATE_CATCHUP);

~

Should LogicalRepApplyLoop still be mentioned here, since that is
static in worker.c? Maybe it is better to refer instead to the common
'start_apply' wrapper? (see also #5a below)

Isn't' LogicalRepApplyLoop static on HEAD and also mentioned in the exact comment in tablesync.c while the common "start_apply" function also exists? I'm not sure how such a change would be related to this patch.

Fair enough. I thought it was questionable for one module to refer to
another module's static functions, but you are correct - it is not
really related to your patch. Sorry for the noise.

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#118Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#116)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Some review comments for v21-0001

======
src/backend/replication/logical/worker.c

1. InitializeLogRepWorker

  if (am_tablesync_worker())
  ereport(LOG,
- (errmsg("logical replication table synchronization worker for
subscription \"%s\", table \"%s\" has started",
+ (errmsg("logical replication worker for subscription \"%s\", table
\"%s\" has started",
  MySubscription->name,
  get_rel_name(MyLogicalRepWorker->relid))));

I think this should not be changed. IIUC that decision for using the
generic worker name for translations was only when the errmsg was in
shared code where the worker type was not clear from existing
conditions. See also previous review comments [1]worker name in errmsg - /messages/by-id/CAA4eK1++wkxxMjsPh-z2aKa9ZjNhKsjv0Tnw+TVX-hCBkDHusw@mail.gmail.com.

~~~

2. StartLogRepWorker

/* Common function to start the leader apply or tablesync worker. */
void
StartLogRepWorker(int worker_slot)
{
/* Attach to slot */
logicalrep_worker_attach(worker_slot);

/* Setup signal handling */
pqsignal(SIGHUP, SignalHandlerForConfigReload);
pqsignal(SIGTERM, die);
BackgroundWorkerUnblockSignals();

/*
* We don't currently need any ResourceOwner in a walreceiver process, but
* if we did, we could call CreateAuxProcessResourceOwner here.
*/

/* Initialise stats to a sanish value */
MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
MyLogicalRepWorker->reply_time = GetCurrentTimestamp();

/* Load the libpq-specific functions */
load_file("libpqwalreceiver", false);

InitializeLogRepWorker();

/* Connect to the origin and start the replication. */
elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
MySubscription->conninfo);

/*
* Setup callback for syscache so that we know when something changes in
* the subscription relation state.
*/
CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
invalidate_syncing_table_states,
(Datum) 0);
}

~

2a.
The function name seems a bit misleading because it is not really
"starting" anything here - it is just more "initialization" code,
right? Nor is it common to all kinds of LogRepWorker. Maybe the
function could be named something else like 'InitApplyOrSyncWorker()'.
-- see also #2c

~

2b.
Should this have Assert to ensure this is only called from leader
apply or tablesync? -- see also #2c

~

2c.
IMO maybe the best/tidiest way to do this is not to introduce a new
function at all. Instead, just put all this "common init" code into
the existing "common init" function ('InitializeLogRepWorker') and
execute it only if (am_tablesync_worker() || am_leader_apply_worker())
{ }.

======
src/include/replication/worker_internal.h

3.
 extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
     XLogRecPtr remote_lsn);
+extern void set_stream_options(WalRcvStreamOptions *options,
+    char *slotname,
+    XLogRecPtr *origin_startpos);
+
+extern void start_apply(XLogRecPtr origin_startpos);
+extern void DisableSubscriptionAndExit(void);
+extern void StartLogRepWorker(int worker_slot);

This placement (esp. with the missing whitespace) seems to be grouping
the set_stream_options with the other 'pa' externs, which are all
under the comment "/* Parallel apply worker setup and interactions
*/".

Putting all these up near the other "extern void
InitializeLogRepWorker(void)" might be less ambiguous.

------
[1]: worker name in errmsg - /messages/by-id/CAA4eK1++wkxxMjsPh-z2aKa9ZjNhKsjv0Tnw+TVX-hCBkDHusw@mail.gmail.com
/messages/by-id/CAA4eK1++wkxxMjsPh-z2aKa9ZjNhKsjv0Tnw+TVX-hCBkDHusw@mail.gmail.com

Kind Regards,
Peter Smith.
Fujitsu Australia

#119Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#116)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Some review comments for v21-0002.

On Thu, Jul 20, 2023 at 11:41 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi,

Attached the updated patches with recent reviews addressed.

See below for my comments:

Peter Smith <smithpb2250@gmail.com>, 19 Tem 2023 Çar, 06:08 tarihinde şunu yazdı:

5.
+ /* Found a table for next iteration */
+ finish_sync_worker(true);
+
+ StartTransactionCommand();
+ ereport(LOG,
+ (errmsg("logical replication worker for subscription \"%s\" will be
reused to sync table \"%s\" with relid %u.",
+ MySubscription->name,
+ get_rel_name(MyLogicalRepWorker->relid),
+ MyLogicalRepWorker->relid)));
+ CommitTransactionCommand();
+
+ done = false;
+ break;
+ }
+ LWLockRelease(LogicalRepWorkerLock);

5b.
Isn't there a missing call to that LWLockRelease, if the 'break' happens?

Lock is already released before break, if that's the lock you meant:

/* Update worker state for the next table */
MyLogicalRepWorker->relid = rstate->relid;
MyLogicalRepWorker->relstate = rstate->state;
MyLogicalRepWorker->relstate_lsn = rstate->lsn;
LWLockRelease(LogicalRepWorkerLock);

/* Found a table for next iteration */
finish_sync_worker(true);
done = false;
break;

Sorry, I misread the code. You are right.

======
src/backend/replication/logical/tablesync.c

1.
+ if (!reuse_worker)
+ {
+ ereport(LOG,
+ (errmsg("logical replication table synchronization worker for
subscription \"%s\" has finished",
+ MySubscription->name)));
+ }
+ else
+ {
+ ereport(LOG,
+ (errmsg("logical replication worker for subscription \"%s\" will be
reused to sync table \"%s\" with relid %u.",
+ MySubscription->name,
+ get_rel_name(MyLogicalRepWorker->relid),
+ MyLogicalRepWorker->relid)));
+ }

1a.
We know this must be a tablesync worker, so I think that second errmsg
should also be saying "logical replication table synchronization
worker".

~

1b.
Since this is if/else anyway, is it simpler to be positive and say "if
(reuse_worker)" instead of the negative "if (!reuse_worker)"

~~~

2. run_tablesync_worker
 {
+ MyLogicalRepWorker->relsync_completed = false;
+
+ /* Start table synchronization. */
  start_table_sync(origin_startpos, &slotname);
This still contains the added comment that I'd previously posted I
thought was adding anything useful. Also, I didn't think this comment
exists in the HEAD code.
======
src/backend/replication/logical/worker.c

3. LogicalRepApplyLoop

+ /*
+ * apply_dispatch() may have gone into apply_handle_commit()
+ * which can call process_syncing_tables_for_sync.
+ *
+ * process_syncing_tables_for_sync decides whether the sync of
+ * the current table is completed. If it is completed,
+ * streaming must be already ended. So, we can break the loop.
+ */
+ if (am_tablesync_worker() &&
+ MyLogicalRepWorker->relsync_completed)
+ {
+ endofstream = true;
+ break;
+ }
+

Maybe just personal taste, but IMO it is better to rearrange like
below because then there is no reason to read the long comment except
for tablesync workers.

if (am_tablesync_worker())
{
/*
* apply_dispatch() may have gone into apply_handle_commit()
* which can call process_syncing_tables_for_sync.
*
* process_syncing_tables_for_sync decides whether the sync of
* the current table is completed. If it is completed,
* streaming must be already ended. So, we can break the loop.
*/
if (MyLogicalRepWorker->relsync_completed)
{
endofstream = true;
break;
}
}

~~~

4. LogicalRepApplyLoop

+
+ /*
+ * If relsync_completed is true, this means that the tablesync
+ * worker is done with synchronization. Streaming has already been
+ * ended by process_syncing_tables_for_sync. We should move to the
+ * next table if needed, or exit.
+ */
+ if (am_tablesync_worker() &&
+ MyLogicalRepWorker->relsync_completed)
+ endofstream = true;

Ditto the same comment about rearranging the condition, as #3 above.

======
src/include/replication/worker_internal.h

5.
+ /*
+ * Indicates whether tablesync worker has completed syncing its assigned
+ * table.
+ */
+ bool relsync_completed;
+

Isn't it better to arrange this to be adjacent to other relXXX fields,
so they all clearly belong to that "Used for initial table
synchronization." group?

For example, something like:

/* Used for initial table synchronization. */
Oid relid;
char relstate;
XLogRecPtr relstate_lsn;
slock_t relmutex;
bool relsync_completed; /* has tablesync finished syncing
the assigned table? */

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#120Amit Kapila
amit.kapila16@gmail.com
In reply to: Peter Smith (#118)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Fri, Jul 21, 2023 at 7:30 AM Peter Smith <smithpb2250@gmail.com> wrote:

~~~

2. StartLogRepWorker

/* Common function to start the leader apply or tablesync worker. */
void
StartLogRepWorker(int worker_slot)
{
/* Attach to slot */
logicalrep_worker_attach(worker_slot);

/* Setup signal handling */
pqsignal(SIGHUP, SignalHandlerForConfigReload);
pqsignal(SIGTERM, die);
BackgroundWorkerUnblockSignals();

/*
* We don't currently need any ResourceOwner in a walreceiver process, but
* if we did, we could call CreateAuxProcessResourceOwner here.
*/

/* Initialise stats to a sanish value */
MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
MyLogicalRepWorker->reply_time = GetCurrentTimestamp();

/* Load the libpq-specific functions */
load_file("libpqwalreceiver", false);

InitializeLogRepWorker();

/* Connect to the origin and start the replication. */
elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
MySubscription->conninfo);

/*
* Setup callback for syscache so that we know when something changes in
* the subscription relation state.
*/
CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
invalidate_syncing_table_states,
(Datum) 0);
}

~

2a.
The function name seems a bit misleading because it is not really
"starting" anything here - it is just more "initialization" code,
right? Nor is it common to all kinds of LogRepWorker. Maybe the
function could be named something else like 'InitApplyOrSyncWorker()'.
-- see also #2c

How about SetupLogRepWorker? The other thing I noticed is that we
don't seem to be consistent in naming functions in these files. For
example, shall we make all exposed functions follow camel case (like
InitializeLogRepWorker) and static functions follow _ style (like
run_apply_worker) or the other possibility is to use _ style for all
functions except may be the entry functions like ApplyWorkerMain()? I
don't know if there is already a pattern but if not then let's form it
now, so that code looks consistent.

~

2b.
Should this have Assert to ensure this is only called from leader
apply or tablesync? -- see also #2c

~

2c.
IMO maybe the best/tidiest way to do this is not to introduce a new
function at all. Instead, just put all this "common init" code into
the existing "common init" function ('InitializeLogRepWorker') and
execute it only if (am_tablesync_worker() || am_leader_apply_worker())
{ }.

I don't like 2c much because it will make InitializeLogRepWorker()
have two kinds of initializations.

======
src/include/replication/worker_internal.h

3.
extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
XLogRecPtr remote_lsn);
+extern void set_stream_options(WalRcvStreamOptions *options,
+    char *slotname,
+    XLogRecPtr *origin_startpos);
+
+extern void start_apply(XLogRecPtr origin_startpos);
+extern void DisableSubscriptionAndExit(void);
+extern void StartLogRepWorker(int worker_slot);

This placement (esp. with the missing whitespace) seems to be grouping
the set_stream_options with the other 'pa' externs, which are all
under the comment "/* Parallel apply worker setup and interactions
*/".

Putting all these up near the other "extern void
InitializeLogRepWorker(void)" might be less ambiguous.

+1. Also, note that they should be in the same order as they are in .c files.

--
With Regards,
Amit Kapila.

#121Peter Smith
smithpb2250@gmail.com
In reply to: Amit Kapila (#120)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Fri, Jul 21, 2023 at 3:39 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

On Fri, Jul 21, 2023 at 7:30 AM Peter Smith <smithpb2250@gmail.com> wrote:

~~~

2. StartLogRepWorker

/* Common function to start the leader apply or tablesync worker. */
void
StartLogRepWorker(int worker_slot)
{
/* Attach to slot */
logicalrep_worker_attach(worker_slot);

/* Setup signal handling */
pqsignal(SIGHUP, SignalHandlerForConfigReload);
pqsignal(SIGTERM, die);
BackgroundWorkerUnblockSignals();

/*
* We don't currently need any ResourceOwner in a walreceiver process, but
* if we did, we could call CreateAuxProcessResourceOwner here.
*/

/* Initialise stats to a sanish value */
MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
MyLogicalRepWorker->reply_time = GetCurrentTimestamp();

/* Load the libpq-specific functions */
load_file("libpqwalreceiver", false);

InitializeLogRepWorker();

/* Connect to the origin and start the replication. */
elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
MySubscription->conninfo);

/*
* Setup callback for syscache so that we know when something changes in
* the subscription relation state.
*/
CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
invalidate_syncing_table_states,
(Datum) 0);
}

~

2a.
The function name seems a bit misleading because it is not really
"starting" anything here - it is just more "initialization" code,
right? Nor is it common to all kinds of LogRepWorker. Maybe the
function could be named something else like 'InitApplyOrSyncWorker()'.
-- see also #2c

How about SetupLogRepWorker?

The name is better than StartXXX, but still, SetupXXX seems a synonym
of InitXXX. That is why I thought it is a bit awkward having 2
functions with effectively the same name and the same
initialization/setup purpose (the only difference is one function
excludes parallel workers, and the other function is common to all
workers).

The other thing I noticed is that we
don't seem to be consistent in naming functions in these files. For
example, shall we make all exposed functions follow camel case (like
InitializeLogRepWorker) and static functions follow _ style (like
run_apply_worker) or the other possibility is to use _ style for all
functions except may be the entry functions like ApplyWorkerMain()? I
don't know if there is already a pattern but if not then let's form it
now, so that code looks consistent.

+1 for using some consistent rule, but I think this may result in
*many* changes, so it would be safer to itemize all the changes first,
just to make sure everybody is OK with it first before updating
everything.

------
Kind Regards,
Peter Smith

#122Amit Kapila
amit.kapila16@gmail.com
In reply to: Peter Smith (#121)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Fri, Jul 21, 2023 at 12:05 PM Peter Smith <smithpb2250@gmail.com> wrote:

On Fri, Jul 21, 2023 at 3:39 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

On Fri, Jul 21, 2023 at 7:30 AM Peter Smith <smithpb2250@gmail.com> wrote:

~~~

2. StartLogRepWorker

/* Common function to start the leader apply or tablesync worker. */
void
StartLogRepWorker(int worker_slot)
{
/* Attach to slot */
logicalrep_worker_attach(worker_slot);

/* Setup signal handling */
pqsignal(SIGHUP, SignalHandlerForConfigReload);
pqsignal(SIGTERM, die);
BackgroundWorkerUnblockSignals();

/*
* We don't currently need any ResourceOwner in a walreceiver process, but
* if we did, we could call CreateAuxProcessResourceOwner here.
*/

/* Initialise stats to a sanish value */
MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
MyLogicalRepWorker->reply_time = GetCurrentTimestamp();

/* Load the libpq-specific functions */
load_file("libpqwalreceiver", false);

InitializeLogRepWorker();

/* Connect to the origin and start the replication. */
elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
MySubscription->conninfo);

/*
* Setup callback for syscache so that we know when something changes in
* the subscription relation state.
*/
CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
invalidate_syncing_table_states,
(Datum) 0);
}

~

2a.
The function name seems a bit misleading because it is not really
"starting" anything here - it is just more "initialization" code,
right? Nor is it common to all kinds of LogRepWorker. Maybe the
function could be named something else like 'InitApplyOrSyncWorker()'.
-- see also #2c

How about SetupLogRepWorker?

The name is better than StartXXX, but still, SetupXXX seems a synonym
of InitXXX. That is why I thought it is a bit awkward having 2
functions with effectively the same name and the same
initialization/setup purpose (the only difference is one function
excludes parallel workers, and the other function is common to all
workers).

I can't know of a better way. We can probably name it as
SetupApplyOrSyncWorker or something like that if you find that better.

The other thing I noticed is that we
don't seem to be consistent in naming functions in these files. For
example, shall we make all exposed functions follow camel case (like
InitializeLogRepWorker) and static functions follow _ style (like
run_apply_worker) or the other possibility is to use _ style for all
functions except may be the entry functions like ApplyWorkerMain()? I
don't know if there is already a pattern but if not then let's form it
now, so that code looks consistent.

+1 for using some consistent rule, but I think this may result in
*many* changes, so it would be safer to itemize all the changes first,
just to make sure everybody is OK with it first before updating
everything.

Fair enough. We can do that as a first patch and then work on the
refactoring patch to avoid introducing more inconsistencies or we can
do the refactoring patch first but keep all the new function names to
follow _ style.

Apart from this, few more comments on 0001:
1.
+run_apply_worker(WalRcvStreamOptions *options,
+ char *slotname,
+ char *originname,
+ int originname_size,
+ XLogRecPtr *origin_startpos)

The caller neither uses nor passes the value of origin_startpos. So,
isn't it better to make origin_startpos local to run_apply_worker()?
It seems the same is true for some of the other parameters slotname,
originname, originname_size. Is there a reason to keep these as
arguments in this function?

2.
+static void
+run_tablesync_worker(WalRcvStreamOptions *options,
+ char *slotname,
+ char *originname,
+ int originname_size,
+ XLogRecPtr *origin_startpos)

The comments in the previous point seem to apply to this as well.

3.
+ set_stream_options(options, slotname, origin_startpos);
+
+ walrcv_startstreaming(LogRepWorkerWalRcvConn, options);
+
+ if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+ AllTablesyncsReady())

This last check is done in set_stream_options() and here as well. I
don't see any reason to give different answers at both places but
before the patch, we were not relying on any such assumption that this
check will always give the same answer considering the answer could be
different due to AllTablesyncsReady(). Can we move this check outside
set_stream_options()?

--
With Regards,
Amit Kapila.

#123Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Amit Kapila (#120)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Amit Kapila <amit.kapila16@gmail.com>, 21 Tem 2023 Cum, 08:39 tarihinde
şunu yazdı:

On Fri, Jul 21, 2023 at 7:30 AM Peter Smith <smithpb2250@gmail.com> wrote:
How about SetupLogRepWorker? The other thing I noticed is that we
don't seem to be consistent in naming functions in these files. For
example, shall we make all exposed functions follow camel case (like
InitializeLogRepWorker) and static functions follow _ style (like
run_apply_worker) or the other possibility is to use _ style for all
functions except may be the entry functions like ApplyWorkerMain()? I
don't know if there is already a pattern but if not then let's form it
now, so that code looks consistent.

I agree that these files have inconsistencies in naming things.
Most of the time I can't really figure out which naming convention I should
use. I try to name things by looking at other functions with similar
responsibilities.

3.

extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
XLogRecPtr remote_lsn);
+extern void set_stream_options(WalRcvStreamOptions *options,
+    char *slotname,
+    XLogRecPtr *origin_startpos);
+
+extern void start_apply(XLogRecPtr origin_startpos);
+extern void DisableSubscriptionAndExit(void);
+extern void StartLogRepWorker(int worker_slot);

This placement (esp. with the missing whitespace) seems to be grouping
the set_stream_options with the other 'pa' externs, which are all
under the comment "/* Parallel apply worker setup and interactions
*/".

Putting all these up near the other "extern void
InitializeLogRepWorker(void)" might be less ambiguous.

+1. Also, note that they should be in the same order as they are in .c
files.

I did not realize the order is the same with .c files. Good to know. I'll
fix it along with other comments.

Thanks,
--
Melih Mutlu
Microsoft

#124Peter Smith
smithpb2250@gmail.com
In reply to: Amit Kapila (#122)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Fri, Jul 21, 2023 at 5:24 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

On Fri, Jul 21, 2023 at 12:05 PM Peter Smith <smithpb2250@gmail.com> wrote:

On Fri, Jul 21, 2023 at 3:39 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

The other thing I noticed is that we
don't seem to be consistent in naming functions in these files. For
example, shall we make all exposed functions follow camel case (like
InitializeLogRepWorker) and static functions follow _ style (like
run_apply_worker) or the other possibility is to use _ style for all
functions except may be the entry functions like ApplyWorkerMain()? I
don't know if there is already a pattern but if not then let's form it
now, so that code looks consistent.

+1 for using some consistent rule, but I think this may result in
*many* changes, so it would be safer to itemize all the changes first,
just to make sure everybody is OK with it first before updating
everything.

Fair enough. We can do that as a first patch and then work on the
refactoring patch to avoid introducing more inconsistencies or we can
do the refactoring patch first but keep all the new function names to
follow _ style.

Fixing the naming inconsistency will be more far-reaching than just a
few functions affected by these "reuse" patches. There are plenty of
existing functions already inconsistently named in the HEAD code. So
perhaps this topic should be moved to a separate thread?

For example, here are some existing/proposed names:

===

worker.c (HEAD)

static functions
DisableSubscriptionAndExit -> disable_subscription_and_exit
FindReplTupleInLocalRel -> find_repl_tuple_in_local_rel
TwoPhaseTransactionGid -> two_phase_transaction_gid
TargetPrivilegesCheck -> target_privileges_check
UpdateWorkerStats -> update_worker_stats
LogicalRepApplyLoop -> logical_rep_apply_loop

non-static functions
stream_stop_internal -> StreamStopInternal
apply_spooled_messages -> ApplySpooledMessages
apply_dispatch -> ApplyDispatch
store_flush_position -> StoreFlushPosition
set_apply_error_context_origin -> SetApplyErrorContextOrigin

===

tablesync.c (HEAD)

static functions
FetchTableStates -> fetch_table_states

non-static functions
invalidate_syncing_table_states -> InvalidateSyncingTableStates

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#125Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Peter Smith (#124)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Peter Smith <smithpb2250@gmail.com>, 21 Tem 2023 Cum, 12:48 tarihinde şunu
yazdı:

On Fri, Jul 21, 2023 at 5:24 PM Amit Kapila <amit.kapila16@gmail.com>
wrote:

On Fri, Jul 21, 2023 at 12:05 PM Peter Smith <smithpb2250@gmail.com>

wrote:

On Fri, Jul 21, 2023 at 3:39 PM Amit Kapila <amit.kapila16@gmail.com>

wrote:

The other thing I noticed is that we
don't seem to be consistent in naming functions in these files. For
example, shall we make all exposed functions follow camel case (like
InitializeLogRepWorker) and static functions follow _ style (like
run_apply_worker) or the other possibility is to use _ style for all
functions except may be the entry functions like ApplyWorkerMain()? I
don't know if there is already a pattern but if not then let's form

it

now, so that code looks consistent.

+1 for using some consistent rule, but I think this may result in
*many* changes, so it would be safer to itemize all the changes first,
just to make sure everybody is OK with it first before updating
everything.

Fair enough. We can do that as a first patch and then work on the
refactoring patch to avoid introducing more inconsistencies or we can
do the refactoring patch first but keep all the new function names to
follow _ style.

Fixing the naming inconsistency will be more far-reaching than just a
few functions affected by these "reuse" patches. There are plenty of
existing functions already inconsistently named in the HEAD code. So
perhaps this topic should be moved to a separate thread?

+1 for moving it to a separate thread. This is not something particularly
introduced by this patch.

Thanks,
--
Melih Mutlu
Microsoft

#126Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Melih Mutlu (#123)
3 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi,

Melih Mutlu <m.melihmutlu@gmail.com>, 21 Tem 2023 Cum, 12:47 tarihinde şunu
yazdı:

I did not realize the order is the same with .c files. Good to know. I'll
fix it along with other comments.

Addressed the recent reviews and attached the updated patches.

Thanks,
--
Melih Mutlu
Microsoft

Attachments:

v22-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchapplication/octet-stream; name=v22-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchDownload
From 86a02f4fd54d5d5db89df21e327777eaf2aeaed3 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Mon, 5 Jun 2023 15:04:41 +0300
Subject: [PATCH v22 1/3] Refactor to split Apply and Tablesync Workers

Both apply and tablesync workers were using ApplyWorkerMain() as entry
point. As the name implies, ApplyWorkerMain() should be considered as
the main function for apply workers. Tablesync worker's path was hidden
and does not have enough in common to share the same main function with
apply worker.

Also, most of the code shared by both worker types is already combined
in LogicalRepApplyLoop(). There is no need to combine the rest in
ApplyWorkerMain() anymore.

This patch introduces TablesyncWorkerMain() as a new entry point for
tablesync workers. This aims to increase code readability and help to
the upcoming reuse tablesync worker improvements.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/postmaster/bgworker.c             |   3 +
 .../replication/logical/applyparallelworker.c |   2 +-
 src/backend/replication/logical/launcher.c    |  32 +-
 src/backend/replication/logical/tablesync.c   |  94 ++++-
 src/backend/replication/logical/worker.c      | 380 ++++++++----------
 src/include/replication/logicalworker.h       |   1 +
 src/include/replication/worker_internal.h     |  15 +-
 7 files changed, 303 insertions(+), 224 deletions(-)

diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 5b4bd71694..505e38376c 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -131,6 +131,9 @@ static const struct
 	},
 	{
 		"ParallelApplyWorkerMain", ParallelApplyWorkerMain
+	},
+	{
+		"TablesyncWorkerMain", TablesyncWorkerMain
 	}
 };
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 6fb96148f4..1d4e83c4c1 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -942,7 +942,7 @@ ParallelApplyWorkerMain(Datum main_arg)
 	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
 		MyLogicalRepWorker->reply_time = 0;
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 542af7d863..e231fa7f95 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -459,24 +459,30 @@ retry:
 	snprintf(bgw.bgw_library_name, MAXPGPATH, "postgres");
 
 	if (is_parallel_apply_worker)
+	{
 		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ParallelApplyWorkerMain");
-	else
-		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
-
-	if (OidIsValid(relid))
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication worker for subscription %u sync %u", subid, relid);
-	else if (is_parallel_apply_worker)
+				 "logical replication parallel apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
+	}
+	else if (OidIsValid(relid))
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "TablesyncWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication parallel apply worker for subscription %u", subid);
+				 "logical replication tablesync worker for subscription %u sync %u",
+				 subid,
+				 relid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication tablesync worker");
+	}
 	else
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication apply worker for subscription %u", subid);
-
-	if (is_parallel_apply_worker)
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
-	else
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication worker");
+				 "logical replication apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication apply worker");
+	}
 
 	bgw.bgw_restart_time = BGW_NEVER_RESTART;
 	bgw.bgw_notify_pid = MyProcPid;
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 6d461654ab..ff859e0910 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -104,17 +104,21 @@
 #include "nodes/makefuncs.h"
 #include "parser/parse_relation.h"
 #include "pgstat.h"
+#include "postmaster/interrupt.h"
 #include "replication/logicallauncher.h"
 #include "replication/logicalrelation.h"
+#include "replication/logicalworker.h"
 #include "replication/walreceiver.h"
 #include "replication/worker_internal.h"
 #include "replication/slot.h"
 #include "replication/origin.h"
 #include "storage/ipc.h"
 #include "storage/lmgr.h"
+#include "tcop/tcopprot.h"
 #include "utils/acl.h"
 #include "utils/array.h"
 #include "utils/builtins.h"
+#include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
 #include "utils/rls.h"
@@ -1241,7 +1245,7 @@ ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
  *
  * The returned slot name is palloc'ed in current memory context.
  */
-char *
+static char *
 LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 {
 	char	   *slotname;
@@ -1584,6 +1588,94 @@ FetchTableStates(bool *started_tx)
 	return has_subrels;
 }
 
+/*
+ * Execute the initial sync with error handling. Disable the subscription,
+ * if it's required.
+ *
+ * Allocate the slot name in long-lived context on return. Note that we don't
+ * handle FATAL errors which are probably because of system resource error and
+ * are not repeatable.
+ */
+static void
+start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+{
+	char	   *syncslotname = NULL;
+
+	Assert(am_tablesync_worker());
+
+	PG_TRY();
+	{
+		/* Call initial sync. */
+		syncslotname = LogicalRepSyncTableStart(origin_startpos);
+	}
+	PG_CATCH();
+	{
+		if (MySubscription->disableonerr)
+			DisableSubscriptionAndExit();
+		else
+		{
+			/*
+			 * Report the worker failed during table synchronization. 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();
+
+	/* allocate slot name in long-lived context */
+	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+	pfree(syncslotname);
+}
+
+/*
+ * Runs the tablesync worker.
+ *
+ * It starts syncing tables. After a successful sync, sets streaming options
+ * and starts streaming to catchup.
+ */
+static void
+run_tablesync_worker()
+{
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *slotname = NULL;
+	WalRcvStreamOptions options;
+
+	start_table_sync(&origin_startpos, &slotname);
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   sizeof(originname));
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(&options, slotname, &origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+
+	/* Start applying changes to catchup. */
+	start_apply(origin_startpos);
+}
+
+/* Logical Replication Tablesync worker entry point */
+void
+TablesyncWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
+
+	SetupApplyOrSyncWorker(worker_slot);
+
+	run_tablesync_worker();
+
+	finish_sync_worker();
+}
+
 /*
  * If the subscription has no tables then return false.
  *
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 832b1cf764..8ed6fa7acf 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -396,8 +396,6 @@ static void stream_close_file(void);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
-static void DisableSubscriptionAndExit(void);
-
 static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
 static void apply_handle_insert_internal(ApplyExecutionData *edata,
 										 ResultRelInfo *relinfo,
@@ -4327,6 +4325,57 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+/*
+ * Sets streaming options including replication slot name and origin start
+ * position. Workers need these options for logical replication.
+ */
+void
+set_stream_options(WalRcvStreamOptions *options,
+				   char *slotname,
+				   XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4361,24 +4410,18 @@ TwoPhaseTransactionGid(Oid subid, TransactionId xid, char *gid, int szgid)
 }
 
 /*
- * Execute the initial sync with error handling. Disable the subscription,
- * if it's required.
+ * Common function to run the apply loop with error handling. Disable the
+ * subscription, if necessary.
  *
- * Allocate the slot name in long-lived context on return. Note that we don't
- * handle FATAL errors which are probably because of system resource error and
- * are not repeatable.
+ * Note that we don't handle FATAL errors which are probably because
+ * of system resource error and are not repeatable.
  */
-static void
-start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+void
+start_apply(XLogRecPtr origin_startpos)
 {
-	char	   *syncslotname = NULL;
-
-	Assert(am_tablesync_worker());
-
 	PG_TRY();
 	{
-		/* Call initial sync. */
-		syncslotname = LogicalRepSyncTableStart(origin_startpos);
+		LogicalRepApplyLoop(origin_startpos);
 	}
 	PG_CATCH();
 	{
@@ -4387,65 +4430,132 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 		else
 		{
 			/*
-			 * Report the worker failed during table synchronization. Abort
-			 * the current transaction so that the stats message is sent in an
+			 * Report the worker failed while applying changes. Abort the
+			 * current transaction so that the stats message is sent in an
 			 * idle state.
 			 */
 			AbortOutOfAnyTransaction();
-			pgstat_report_subscription_error(MySubscription->oid, false);
+			pgstat_report_subscription_error(MySubscription->oid, !am_tablesync_worker());
 
 			PG_RE_THROW();
 		}
 	}
 	PG_END_TRY();
-
-	/* allocate slot name in long-lived context */
-	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
-	pfree(syncslotname);
 }
 
 /*
- * Run the apply loop with error handling. Disable the subscription,
- * if necessary.
+ * Runs the leader apply worker.
  *
- * Note that we don't handle FATAL errors which are probably because
- * of system resource error and are not repeatable.
+ * It sets up replication origin, streaming options and then starts streaming.
  */
 static void
-start_apply(XLogRecPtr origin_startpos)
+run_apply_worker()
 {
-	PG_TRY();
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *slotname = NULL;
+	WalRcvStreamOptions options;
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+	bool		must_use_password;
+
+	slotname = MySubscription->slotname;
+
+	/*
+	 * This shouldn't happen if the subscription is enabled, but guard
+	 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+	 * crash if slot is NULL.)
+	 */
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	/* Setup replication origin tracking. */
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, sizeof(originname));
+	StartTransactionCommand();
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	origin_startpos = replorigin_session_get_progress(false);
+
+	/* Is the use of a password mandatory? */
+	must_use_password = MySubscription->passwordrequired &&
+		!superuser_arg(MySubscription->owner);
+
+	/* Note that the superuser_arg call can access the DB */
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											must_use_password,
+											MySubscription->name, &err);
+
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+	 * We don't really use the output identify_system for anything but it
+	 * does some initializations on the upstream so let's still call it.
+	 */
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(&options, slotname, &origin_startpos);
+
+	/*
+		* Even when the two_phase mode is requested by the user, it remains
+		* as the tri-state PENDING until all tablesyncs have reached READY
+		* state. Only then, can it become ENABLED.
+		*
+		* Note: If the subscription has no tables then leave the state as
+		* PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+		* work.
+		*/
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
 	{
-		LogicalRepApplyLoop(origin_startpos);
+		/* Start streaming with two_phase enabled */
+		options.proto.logical.twophase = true;
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
 	}
-	PG_CATCH();
+	else
 	{
-		if (MySubscription->disableonerr)
-			DisableSubscriptionAndExit();
-		else
-		{
-			/*
-			 * Report the worker failed while applying changes. Abort the
-			 * current transaction so that the stats message is sent in an
-			 * idle state.
-			 */
-			AbortOutOfAnyTransaction();
-			pgstat_report_subscription_error(MySubscription->oid, !am_tablesync_worker());
-
-			PG_RE_THROW();
-		}
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
 	}
-	PG_END_TRY();
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+
+	/* Run the main loop. */
+	start_apply(origin_startpos);
 }
 
 /*
- * Common initialization for leader apply worker and parallel apply worker.
+ * Common initialization for leader apply worker, parallel apply worker and
+ * tablesync worker.
  *
  * Initialize the database connection, in-memory subscription and necessary
  * config options.
  */
 void
-InitializeApplyWorker(void)
+InitializeLogRepWorker(void)
 {
 	MemoryContext oldctx;
 
@@ -4518,22 +4628,15 @@ InitializeApplyWorker(void)
 	CommitTransactionCommand();
 }
 
-/* Logical Replication Apply worker entry point */
+/* Common function to setup the leader apply or tablesync worker. */
 void
-ApplyWorkerMain(Datum main_arg)
+SetupApplyOrSyncWorker(int worker_slot)
 {
-	int			worker_slot = DatumGetInt32(main_arg);
-	char		originname[NAMEDATALEN];
-	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
-	char	   *myslotname = NULL;
-	WalRcvStreamOptions options;
-	int			server_version;
-
-	InitializingApplyWorker = true;
-
 	/* Attach to slot */
 	logicalrep_worker_attach(worker_slot);
 
+	Assert(am_tablesync_worker() || am_leader_apply_worker());
+
 	/* Setup signal handling */
 	pqsignal(SIGHUP, SignalHandlerForConfigReload);
 	pqsignal(SIGTERM, die);
@@ -4551,79 +4654,12 @@ ApplyWorkerMain(Datum main_arg)
 	/* Load the libpq-specific functions */
 	load_file("libpqwalreceiver", false);
 
-	InitializeApplyWorker();
-
-	InitializingApplyWorker = false;
+	InitializeLogRepWorker();
 
 	/* Connect to the origin and start the replication. */
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-		bool		must_use_password;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-
-		/* Is the use of a password mandatory? */
-		must_use_password = MySubscription->passwordrequired &&
-			!superuser_arg(MySubscription->owner);
-
-		/* Note that the superuser_arg call can access the DB */
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												must_use_password,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
 	 * the subscription relation state.
@@ -4631,91 +4667,21 @@ ApplyWorkerMain(Datum main_arg)
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
+}
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
-
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
-
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-
-	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
-	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
+/* Logical Replication Apply worker entry point */
+void
+ApplyWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
 
-	if (!am_tablesync_worker())
-	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
-		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+	InitializingApplyWorker = true;
 
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
-		}
-		else
-		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-		}
+	SetupApplyOrSyncWorker(worker_slot);
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-								 MySubscription->name,
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-								 "?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
+	InitializingApplyWorker = false;
 
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+	run_apply_worker();
 
 	proc_exit(0);
 }
@@ -4724,7 +4690,7 @@ ApplyWorkerMain(Datum main_arg)
  * After error recovery, disable the subscription in a new transaction
  * and exit cleanly.
  */
-static void
+void
 DisableSubscriptionAndExit(void)
 {
 	/*
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 39588da79f..bbd71d0b42 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -18,6 +18,7 @@ extern PGDLLIMPORT volatile sig_atomic_t ParallelApplyMessagePending;
 
 extern void ApplyWorkerMain(Datum main_arg);
 extern void ParallelApplyWorkerMain(Datum main_arg);
+extern void TablesyncWorkerMain(Datum main_arg);
 
 extern bool IsLogicalWorker(void);
 extern bool IsLogicalParallelApplyWorker(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 343e781896..fdbc1183f2 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -19,6 +19,7 @@
 #include "datatype/timestamp.h"
 #include "miscadmin.h"
 #include "replication/logicalrelation.h"
+#include "replication/walreceiver.h"
 #include "storage/buffile.h"
 #include "storage/fileset.h"
 #include "storage/lock.h"
@@ -243,7 +244,6 @@ extern int	logicalrep_sync_worker_count(Oid subid);
 
 extern void ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 											   char *originname, Size szoriginname);
-extern char *LogicalRepSyncTableStart(XLogRecPtr *origin_startpos);
 
 extern bool AllTablesyncsReady(void);
 extern void UpdateTwoPhaseState(Oid suboid, char new_state);
@@ -265,7 +265,17 @@ extern void maybe_reread_subscription(void);
 
 extern void stream_cleanup_files(Oid subid, TransactionId xid);
 
-extern void InitializeApplyWorker(void);
+extern void set_stream_options(WalRcvStreamOptions *options,
+							   char *slotname,
+							   XLogRecPtr *origin_startpos);
+
+extern void start_apply(XLogRecPtr origin_startpos);
+
+extern void InitializeLogRepWorker(void);
+
+extern void SetupApplyOrSyncWorker(int worker_slot);
+
+extern void DisableSubscriptionAndExit(void);
 
 extern void store_flush_position(XLogRecPtr remote_lsn, XLogRecPtr local_lsn);
 
@@ -305,6 +315,7 @@ extern void pa_decr_and_wait_stream_block(void);
 extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
 						   XLogRecPtr remote_lsn);
 
+
 #define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)
 
 static inline bool
-- 
2.25.1

v22-0002-Reuse-Tablesync-Workers.patchapplication/octet-stream; name=v22-0002-Reuse-Tablesync-Workers.patchDownload
From bd18bd59be0a263cb3385353e73ec25542bdeff2 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:04:46 +0300
Subject: [PATCH v22 2/3] Reuse Tablesync Workers

Before this patch, tablesync workers were capable of syncing only one
table. For each table, a new sync worker was launched and that worker would
exit when done processing the table.

Now, tablesync workers are not limited to processing only one
table. When done, they can move to processing another table in
the same subscription.

If there is a table that needs to be synced, an available tablesync
worker picks up that table and syncs it. Each tablesync worker
continues to pick new tables to sync until there are no tables left
requiring synchronization. If there was no available worker to
process the table, then a new tablesync worker will be launched,
provided the number of tablesync workers for the subscription does not
exceed max_sync_workers_per_subscription.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/replication/logical/launcher.c  |   1 +
 src/backend/replication/logical/tablesync.c | 121 ++++++++++++++++++--
 src/backend/replication/logical/worker.c    |  40 ++++++-
 src/include/replication/worker_internal.h   |   2 +
 4 files changed, 149 insertions(+), 15 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e231fa7f95..25dd06b8af 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -440,6 +440,7 @@ retry:
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
+	worker->relsync_completed = false;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index ff859e0910..63b5bed88a 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -134,10 +134,12 @@ static StringInfo copybuf = NULL;
 
 /*
  * Exit routine for synchronization worker.
+ *
+ * If reuse_worker is false, at the conclusion of this function the worker
+ * process will exit.
  */
 static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+finish_sync_worker(bool reuse_worker)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -149,21 +151,42 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
+	/*
+	 * Disconnect from the publisher otherwise reusing the sync worker can
+	 * error due to exceeding max_wal_senders.
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		walrcv_disconnect(LogRepWorkerWalRcvConn);
+		LogRepWorkerWalRcvConn = NULL;
+	}
+
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
 	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
+	if (reuse_worker)
+	{
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\" will be reused to sync table \"%s\" with relid %u.",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
+	}
+	else
+	{
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\" has finished",
+						MySubscription->name)));
+	}
 	CommitTransactionCommand();
 
 	/* Find the leader apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
 
 	/* Stop gracefully */
-	proc_exit(0);
+	if (!reuse_worker)
+		proc_exit(0);
 }
 
 /*
@@ -383,7 +406,15 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 */
 		replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
+		/* Sync worker has completed synchronization of the current table. */
+		MyLogicalRepWorker->relsync_completed = true;
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization for subscription \"%s\", relation \"%s\" with relid %u has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
+		CommitTransactionCommand();
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -1288,7 +1319,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		case SUBREL_STATE_SYNCDONE:
 		case SUBREL_STATE_READY:
 		case SUBREL_STATE_UNKNOWN:
-			finish_sync_worker();	/* doesn't return */
+			finish_sync_worker(false);	/* doesn't return */
 	}
 
 	/* Calculate the name of the tablesync slot. */
@@ -1646,6 +1677,8 @@ run_tablesync_worker()
 	char	   *slotname = NULL;
 	WalRcvStreamOptions options;
 
+	MyLogicalRepWorker->relsync_completed = false;
+
 	start_table_sync(&origin_startpos, &slotname);
 
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
@@ -1668,12 +1701,78 @@ void
 TablesyncWorkerMain(Datum main_arg)
 {
 	int			worker_slot = DatumGetInt32(main_arg);
+	bool 		done = false;
 
 	SetupApplyOrSyncWorker(worker_slot);
 
-	run_tablesync_worker();
+	/*
+	 * The loop where worker does its job. It loops until there is no relation
+	 * left to sync.
+	 */
+	for (;!done;)
+	{
+		List	   *rstates;
+		ListCell   *lc;
+
+		run_tablesync_worker();
+
+		if (IsTransactionState())
+			CommitTransactionCommand();
+
+		if (MyLogicalRepWorker->relsync_completed)
+		{
+			/*
+			 * This tablesync worker is 'done' unless another table that needs
+			 * syncing is found.
+			 */
+			done = true;
+
+			/* This transaction will be committed by finish_sync_worker. */
+			StartTransactionCommand();
+
+			/*
+			 * Check if there is any table whose relation state is still INIT.
+			 * If a table in INIT state is found, the worker will not be
+			 * finished, it will be reused instead.
+			 */
+			rstates = GetSubscriptionRelations(MySubscription->oid, true);
+
+			foreach(lc, rstates)
+			{
+				SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
+
+				if (rstate->state == SUBREL_STATE_SYNCDONE)
+					continue;
+
+				/*
+				 * Take exclusive lock to prevent any other sync worker from
+				 * picking the same table.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
+
+				/*
+				 * Pick the table for the next run if it is not already picked
+				 * up by another worker.
+				 */
+				if (!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+				{
+					/* Update worker state for the next table */
+					MyLogicalRepWorker->relid = rstate->relid;
+					MyLogicalRepWorker->relstate = rstate->state;
+					MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+					LWLockRelease(LogicalRepWorkerLock);
+
+					/* Found a table for next iteration */
+					finish_sync_worker(true);
+					done = false;
+					break;
+				}
+				LWLockRelease(LogicalRepWorkerLock);
+			}
+		}
+	}
 
-	finish_sync_worker();
+	finish_sync_worker(false);
 }
 
 /*
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 8ed6fa7acf..d25bf5bea2 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3621,6 +3621,23 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				if (am_tablesync_worker())
+				{
+					/*
+					 * apply_dispatch() may have gone into apply_handle_commit()
+					 * which can call process_syncing_tables_for_sync.
+					 *
+					 * process_syncing_tables_for_sync decides whether the sync of
+					 * the current table is completed. If it is completed,
+					 * streaming must be already ended. So, we can break the loop.
+					 */
+					if (MyLogicalRepWorker->relsync_completed)
+					{
+						endofstream = true;
+						break;
+					}
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -3640,6 +3657,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+
+			if (am_tablesync_worker())
+				/*
+				 * If relsync_completed is true, this means that the tablesync
+				 * worker is done with synchronization. Streaming has already been
+				 * ended by process_syncing_tables_for_sync. We should move to the
+				 * next table if needed, or exit.
+				 */
+				if (MyLogicalRepWorker->relsync_completed)
+					endofstream = true;
 		}
 
 		/* Cleanup the memory. */
@@ -3742,8 +3769,12 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * End streaming here for only apply workers. Ending streaming for
+	 * tablesync workers is deferred until the worker exits its main loop.
+	 */
+	if (!am_tablesync_worker())
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
 }
 
 /*
@@ -4617,9 +4648,10 @@ InitializeLogRepWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has started",
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" with relid %u has started",
 						MySubscription->name,
-						get_rel_name(MyLogicalRepWorker->relid))));
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
 	else
 		ereport(LOG,
 				(errmsg("logical replication apply worker for subscription \"%s\" has started",
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index fdbc1183f2..39b1721dee 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -56,6 +56,8 @@ typedef struct LogicalRepWorker
 	char		relstate;
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
+	bool		relsync_completed; /* has tablesync finished syncing
+									* the assigned table? */
 
 	/*
 	 * Used to create the changes and subxact files for the streaming
-- 
2.25.1

v22-0003-Reuse-connection-when-tablesync-workers-change-t.patchapplication/octet-stream; name=v22-0003-Reuse-connection-when-tablesync-workers-change-t.patchDownload
From d15ae6faa94033a4fa66f9d29c524d459d0f7f71 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:13:52 +0300
Subject: [PATCH v22 3/3] Reuse connection when tablesync workers change the
 target

Previously tablesync workers establish new connections when it changes the syncing
table, but this might have additional overhead. This patch allows the existing
connection to be reused.

As for the publisher node, this patch allows to reuse logical walsender processes
after the streaming is done once.
---
 src/backend/replication/logical/launcher.c  |  1 +
 src/backend/replication/logical/tablesync.c | 60 ++++++++++++++-------
 src/backend/replication/logical/worker.c    | 18 ++++---
 src/backend/replication/walsender.c         |  7 +++
 src/include/replication/worker_internal.h   |  3 ++
 5 files changed, 62 insertions(+), 27 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 25dd06b8af..657e446eaf 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
 	worker->relsync_completed = false;
+	worker->slotnum = slot;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 63b5bed88a..45e753b189 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -151,16 +151,6 @@ finish_sync_worker(bool reuse_worker)
 		pgstat_report_stat(true);
 	}
 
-	/*
-	 * Disconnect from the publisher otherwise reusing the sync worker can
-	 * error due to exceeding max_wal_senders.
-	 */
-	if (LogRepWorkerWalRcvConn != NULL)
-	{
-		walrcv_disconnect(LogRepWorkerWalRcvConn);
-		LogRepWorkerWalRcvConn = NULL;
-	}
-
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
@@ -1268,6 +1258,27 @@ ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
 			 relid, GetSystemIdentifier());
 }
 
+/*
+ * Determine the application_name for tablesync workers.
+ *
+ * Previously, the replication slot name was used as application_name. Since
+ * it's possible to reuse tablesync workers now, a tablesync worker can handle
+ * several different replication slots during its lifetime. Therefore, we
+ * cannot use the slot name as application_name anymore. Instead, the slot
+ * number of the tablesync worker is used as a part of the application_name.
+ *
+ * FIXME: if the tablesync worker starts to reuse the replication slot during
+ * synchronization, we should again use the replication slot name as
+ * application_name.
+ */
+static void
+ApplicationNameForTablesync(Oid suboid, int worker_slot,
+							char *application_name, Size szapp)
+{
+	snprintf(application_name, szapp, "pg_%u_sync_%i_" UINT64_FORMAT, suboid,
+			 worker_slot, GetSystemIdentifier());
+}
+
 /*
  * Start syncing the table in the sync worker.
  *
@@ -1329,15 +1340,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 									slotname,
 									NAMEDATALEN);
 
-	/*
-	 * Here we use the slot name instead of the subscription name as the
-	 * application_name, so that it is different from the leader apply worker,
-	 * so that synchronous replication can distinguish them.
-	 */
-	LogRepWorkerWalRcvConn =
-		walrcv_connect(MySubscription->conninfo, true,
-					   must_use_password,
-					   slotname, &err);
+	/* Connect to the publisher if haven't done so already. */
+	if (LogRepWorkerWalRcvConn == NULL)
+	{
+		char application_name[NAMEDATALEN];
+
+		/*
+		 * The application_name must differ from the subscription name (used by
+		 * the leader apply worker) because synchronous replication has to be
+		 * able to distinguish this worker from the leader apply worker.
+		 */
+		ApplicationNameForTablesync(MySubscription->oid,
+									MyLogicalRepWorker->slotnum,
+									application_name,
+									NAMEDATALEN);
+		LogRepWorkerWalRcvConn =
+			walrcv_connect(MySubscription->conninfo, true,
+						   must_use_password,
+						   application_name, &err);
+	}
+
 	if (LogRepWorkerWalRcvConn == NULL)
 		ereport(ERROR,
 				(errcode(ERRCODE_CONNECTION_FAILURE),
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index d25bf5bea2..c482f707dc 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3494,20 +3494,22 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	ErrorContextCallback errcallback;
 
 	/*
-	 * Init the ApplyMessageContext which we clean up after each replication
-	 * protocol message.
+	 * Init the ApplyMessageContext if needed. This context is cleaned up
+	 * after each replication protocol message.
 	 */
-	ApplyMessageContext = AllocSetContextCreate(ApplyContext,
-												"ApplyMessageContext",
-												ALLOCSET_DEFAULT_SIZES);
+	if (!ApplyMessageContext)
+		ApplyMessageContext = AllocSetContextCreate(ApplyContext,
+													"ApplyMessageContext",
+													ALLOCSET_DEFAULT_SIZES);
 
 	/*
 	 * This memory context is used for per-stream data when the streaming mode
 	 * is enabled. This context is reset on each stream stop.
 	 */
-	LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
-													"LogicalStreamingContext",
-													ALLOCSET_DEFAULT_SIZES);
+	if (!LogicalStreamingContext)
+		LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
+														"LogicalStreamingContext",
+														ALLOCSET_DEFAULT_SIZES);
 
 	/* mark as idle, before starting to loop */
 	pgstat_report_activity(STATE_IDLE, NULL);
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index d27ef2985d..2f3e93cc40 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1830,7 +1830,14 @@ exec_replication_command(const char *cmd_string)
 				if (cmd->kind == REPLICATION_KIND_PHYSICAL)
 					StartReplication(cmd);
 				else
+				{
+					/*
+					 * Reset flags because reusing tablesync workers can mean
+					 * this is the second time here.
+					 */
+					streamingDoneSending = streamingDoneReceiving = false;
 					StartLogicalReplication(cmd);
+				}
 
 				/* dupe, but necessary per libpqrcv_endstreaming */
 				EndReplicationCommand(cmdtag);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 39b1721dee..b88ff6a646 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -39,6 +39,9 @@ typedef struct LogicalRepWorker
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
+	/* Slot number of this worker. */
+	int			slotnum;
+
 	/* Pointer to proc array. NULL if not running. */
 	PGPROC	   *proc;
 
-- 
2.25.1

#127Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#126)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Here are some comments for patch v22-0001.

======
1. General -- naming conventions

There is quite a lot of inconsistency with variable/parameter naming
styles in this patch. I understand in most cases the names are copied
unchanged from the original functions. Still, since this is a big
refactor anyway, it can also be a good opportunity to clean up those
inconsistencies instead of just propagating them to different places.
IIUC, the usual reluctance to rename things because it would cause
backpatch difficulties doesn't apply here (since everything is being
refactored anyway).

E.g. Consider using use snake_case names more consistently in the
following places:

~

1a. start_table_sync

+static void
+start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+{
+ char    *syncslotname = NULL;

origin_startpos -> (no change)
myslotname -> my_slot_name (But, is there a better name for this than
calling it "my" slot name)
syncslotname -> sync_slot_name

~

1b. run_tablesync_worker

+static void
+run_tablesync_worker()
+{
+ char originname[NAMEDATALEN];
+ XLogRecPtr origin_startpos = InvalidXLogRecPtr;
+ char    *slotname = NULL;
+ WalRcvStreamOptions options;

originname -> origin_name
origin_startpos -> (no change)
slotname -> slot_name

~

1c. set_stream_options

+void
+set_stream_options(WalRcvStreamOptions *options,
+    char *slotname,
+    XLogRecPtr *origin_startpos)
+{
+ int server_version;

options -> (no change)
slotname -> slot_name
origin_startpos -> (no change)
server_version -> (no change)

~

1d. run_apply_worker

 static void
-start_apply(XLogRecPtr origin_startpos)
+run_apply_worker()
 {
- PG_TRY();
+ char originname[NAMEDATALEN];
+ XLogRecPtr origin_startpos = InvalidXLogRecPtr;
+ char    *slotname = NULL;
+ WalRcvStreamOptions options;
+ RepOriginId originid;
+ TimeLineID startpointTLI;
+ char    *err;
+ bool must_use_password;

originname -> origin_name
origin_startpos => (no change)
slotname -> slot_name
originid -> origin_id

======
src/backend/replication/logical/worker.c

2. SetupApplyOrSyncWorker

-ApplyWorkerMain(Datum main_arg)
+SetupApplyOrSyncWorker(int worker_slot)
 {
- int worker_slot = DatumGetInt32(main_arg);
- char originname[NAMEDATALEN];
- XLogRecPtr origin_startpos = InvalidXLogRecPtr;
- char    *myslotname = NULL;
- WalRcvStreamOptions options;
- int server_version;
-
- InitializingApplyWorker = true;
-
  /* Attach to slot */
  logicalrep_worker_attach(worker_slot);
+ Assert(am_tablesync_worker() || am_leader_apply_worker());
+

Why is the Assert not the very first statement of this function?

======
Kind Regards,
Peter Smith.
Fujitsu Australia

#128Amit Kapila
amit.kapila16@gmail.com
In reply to: Peter Smith (#127)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Wed, Jul 26, 2023 at 10:10 AM Peter Smith <smithpb2250@gmail.com> wrote:

Here are some comments for patch v22-0001.

======
1. General -- naming conventions

There is quite a lot of inconsistency with variable/parameter naming
styles in this patch. I understand in most cases the names are copied
unchanged from the original functions. Still, since this is a big
refactor anyway, it can also be a good opportunity to clean up those
inconsistencies instead of just propagating them to different places.

I am not against improving consistency in the naming of existing
variables but I feel it would be better to do as a separate patch
along with improving the consistency function names. For new
functions/variables, it would be good to follow a consistent style.

--
With Regards,
Amit Kapila.

#129Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#126)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Here are some review comments for v22-0002

======
1. General - errmsg

AFAIK, the errmsg part does not need to be enclosed by extra parentheses.

e.g.
BEFORE
ereport(LOG,
(errmsg("logical replication table synchronization worker for
subscription \"%s\" has finished",
MySubscription->name)));
AFTER
ereport(LOG,
errmsg("logical replication table synchronization worker for
subscription \"%s\" has finished",
MySubscription->name));

~

The patch has multiple cases similar to that example.

======
src/backend/replication/logical/tablesync.c

2.
+ if (reuse_worker)
+ {
+ ereport(LOG,
+ (errmsg("logical replication table synchronization worker for
subscription \"%s\" will be reused to sync table \"%s\" with relid
%u.",
+ MySubscription->name,
+ get_rel_name(MyLogicalRepWorker->relid),
+ MyLogicalRepWorker->relid)));
+ }
+ else
+ {
+ ereport(LOG,
+ (errmsg("logical replication table synchronization worker for
subscription \"%s\" has finished",
+ MySubscription->name)));
+ }

These brackets { } are not really necessary.

~~~

3. TablesyncWorkerMain
+ for (;!done;)
+ {
+ List    *rstates;
+ ListCell   *lc;
+
+ run_tablesync_worker();
+
+ if (IsTransactionState())
+ CommitTransactionCommand();
+
+ if (MyLogicalRepWorker->relsync_completed)
+ {
+ /*
+ * This tablesync worker is 'done' unless another table that needs
+ * syncing is found.
+ */
+ done = true;

Those variables 'rstates' and 'lc' do not need to be declared at this
scope -- they can be declared further down, closer to where they are
needed.

=====
src/backend/replication/logical/worker.c

4. LogicalRepApplyLoop
+
+ if (am_tablesync_worker())
+ /*
+ * If relsync_completed is true, this means that the tablesync
+ * worker is done with synchronization. Streaming has already been
+ * ended by process_syncing_tables_for_sync. We should move to the
+ * next table if needed, or exit.
+ */
+ if (MyLogicalRepWorker->relsync_completed)
+ endofstream = true;

Here I think it is better to use bracketing { } for the outer "if",
instead of only relying on the indentation for readability. YMMV.

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#130Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#126)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Here are some review comments for v22-0003

======

1. ApplicationNameForTablesync
+/*
+ * Determine the application_name for tablesync workers.
+ *
+ * Previously, the replication slot name was used as application_name. Since
+ * it's possible to reuse tablesync workers now, a tablesync worker can handle
+ * several different replication slots during its lifetime. Therefore, we
+ * cannot use the slot name as application_name anymore. Instead, the slot
+ * number of the tablesync worker is used as a part of the application_name.
+ *
+ * FIXME: if the tablesync worker starts to reuse the replication slot during
+ * synchronization, we should again use the replication slot name as
+ * application_name.
+ */
+static void
+ApplicationNameForTablesync(Oid suboid, int worker_slot,
+ char *application_name, Size szapp)
+{
+ snprintf(application_name, szapp, "pg_%u_sync_%i_" UINT64_FORMAT, suboid,
+ worker_slot, GetSystemIdentifier());
+}

1a.
The intent of the "FIXME" comment was not clear. Is this some existing
problem that needs addressing, or is this really more like just an
"XXX" warning/note for the future, in case the tablesync logic
changes?

~

1b.
Since this is a new function, should it be named according to the
convention for static functions?

e.g.
ApplicationNameForTablesync -> app_name_for_tablesync

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#131Amit Kapila
amit.kapila16@gmail.com
In reply to: Peter Smith (#130)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Thu, Jul 27, 2023 at 6:46 AM Peter Smith <smithpb2250@gmail.com> wrote:

Here are some review comments for v22-0003

======

1. ApplicationNameForTablesync
+/*
+ * Determine the application_name for tablesync workers.
+ *
+ * Previously, the replication slot name was used as application_name. Since
+ * it's possible to reuse tablesync workers now, a tablesync worker can handle
+ * several different replication slots during its lifetime. Therefore, we
+ * cannot use the slot name as application_name anymore. Instead, the slot
+ * number of the tablesync worker is used as a part of the application_name.
+ *
+ * FIXME: if the tablesync worker starts to reuse the replication slot during
+ * synchronization, we should again use the replication slot name as
+ * application_name.
+ */
+static void
+ApplicationNameForTablesync(Oid suboid, int worker_slot,
+ char *application_name, Size szapp)
+{
+ snprintf(application_name, szapp, "pg_%u_sync_%i_" UINT64_FORMAT, suboid,
+ worker_slot, GetSystemIdentifier());
+}

1a.
The intent of the "FIXME" comment was not clear. Is this some existing
problem that needs addressing, or is this really more like just an
"XXX" warning/note for the future, in case the tablesync logic
changes?

This seems to be a Note for the future, so better to use XXX notation here.

~

1b.
Since this is a new function, should it be named according to the
convention for static functions?

e.g.
ApplicationNameForTablesync -> app_name_for_tablesync

I think for now let's follow the style for similar functions like
ReplicationOriginNameForLogicalRep() and
ReplicationSlotNameForTablesync().

--
With Regards,
Amit Kapila.

#132Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Peter Smith (#127)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Peter,

Peter Smith <smithpb2250@gmail.com>, 26 Tem 2023 Çar, 07:40 tarihinde şunu
yazdı:

Here are some comments for patch v22-0001.

======
1. General -- naming conventions

There is quite a lot of inconsistency with variable/parameter naming
styles in this patch. I understand in most cases the names are copied
unchanged from the original functions. Still, since this is a big
refactor anyway, it can also be a good opportunity to clean up those
inconsistencies instead of just propagating them to different places.
IIUC, the usual reluctance to rename things because it would cause
backpatch difficulties doesn't apply here (since everything is being
refactored anyway).

E.g. Consider using use snake_case names more consistently in the
following places:

I can simply change the places you mentioned, that seems okay to me.
The reason why I did not change the namings in existing variables/functions
is because I did (and still do) not get what's the naming conventions in
those files. Is snake_case the convention for variables in those files (or
in general)?

2. SetupApplyOrSyncWorker

-ApplyWorkerMain(Datum main_arg)
+SetupApplyOrSyncWorker(int worker_slot)
{
- int worker_slot = DatumGetInt32(main_arg);
- char originname[NAMEDATALEN];
- XLogRecPtr origin_startpos = InvalidXLogRecPtr;
- char    *myslotname = NULL;
- WalRcvStreamOptions options;
- int server_version;
-
- InitializingApplyWorker = true;
-
/* Attach to slot */
logicalrep_worker_attach(worker_slot);
+ Assert(am_tablesync_worker() || am_leader_apply_worker());
+

Why is the Assert not the very first statement of this function?

I would also prefer to assert in the very beginning but am_tablesync_worker
and am_leader_apply_worker require MyLogicalRepWorker to be not NULL.
And MyLogicalRepWorker is assigned in logicalrep_worker_attach. I can
change this if you think there is a better way to check the worker type.

Thanks,
--
Melih Mutlu
Microsoft

#133Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#132)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Thu, Jul 27, 2023 at 11:30 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi Peter,

Peter Smith <smithpb2250@gmail.com>, 26 Tem 2023 Çar, 07:40 tarihinde şunu yazdı:

Here are some comments for patch v22-0001.

======
1. General -- naming conventions

There is quite a lot of inconsistency with variable/parameter naming
styles in this patch. I understand in most cases the names are copied
unchanged from the original functions. Still, since this is a big
refactor anyway, it can also be a good opportunity to clean up those
inconsistencies instead of just propagating them to different places.
IIUC, the usual reluctance to rename things because it would cause
backpatch difficulties doesn't apply here (since everything is being
refactored anyway).

E.g. Consider using use snake_case names more consistently in the
following places:

I can simply change the places you mentioned, that seems okay to me.
The reason why I did not change the namings in existing variables/functions is because I did (and still do) not get what's the naming conventions in those files. Is snake_case the convention for variables in those files (or in general)?

TBH, I also don't know if there is a specific Postgres coding
guideline to use snake_case or not (and Chat-GPT did not know either
when I asked about it). I only assumed snake_case in my previous
review comment because the mentioned vars were already all lowercase.
Anyway, the point was that whatever style is chosen, it ought to be
used *consistently* because having a random mixture of styles in the
same function (e.g. worker_slot, originname, origin_startpos,
myslotname, options, server_version) seems messy. Meanwhile, I think
Amit suggested [1]/messages/by-id/CAA4eK1+h9hWDAKupsoiw556xqh7uvj_F1pjFJc4jQhL89HdGww@mail.gmail.com that for now, we only need to worry about the name
consistency in new code.

2. SetupApplyOrSyncWorker

-ApplyWorkerMain(Datum main_arg)
+SetupApplyOrSyncWorker(int worker_slot)
{
- int worker_slot = DatumGetInt32(main_arg);
- char originname[NAMEDATALEN];
- XLogRecPtr origin_startpos = InvalidXLogRecPtr;
- char    *myslotname = NULL;
- WalRcvStreamOptions options;
- int server_version;
-
- InitializingApplyWorker = true;
-
/* Attach to slot */
logicalrep_worker_attach(worker_slot);
+ Assert(am_tablesync_worker() || am_leader_apply_worker());
+

Why is the Assert not the very first statement of this function?

I would also prefer to assert in the very beginning but am_tablesync_worker and am_leader_apply_worker require MyLogicalRepWorker to be not NULL. And MyLogicalRepWorker is assigned in logicalrep_worker_attach. I can change this if you think there is a better way to check the worker type.

I see. In that case your Assert LGTM.

------
[1]: /messages/by-id/CAA4eK1+h9hWDAKupsoiw556xqh7uvj_F1pjFJc4jQhL89HdGww@mail.gmail.com

Kind Regards,
Peter Smith.
Fujitsu Australia

#134Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#126)
9 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Melih,

BACKGROUND
----------

We wanted to compare performance for the 2 different reuse-worker
designs, when the apply worker is already busy handling other
replications, and then simultaneously the test table tablesyncs are
occurring.

To test this scenario, some test scripts were written (described
below). For comparisons, the scripts were then run using a build of
HEAD; design #1 (v21); design #2 (0718).

HOW THE TEST WORKS
------------------

Overview:
1. The apply worker is made to subscribe to a 'busy_tbl'.
2. After the SUBSCRIPTION is created, the publisher-side then loops
(forever) doing INSERTS into that busy_tbl.
3. While the apply worker is now busy, the subscriber does an ALTER
SUBSCRIPTION REFRESH PUBLICATION to subscribe to all the other test
tables.
4. We time how long it takes for all tablsyncs to complete
5. Repeat above for different numbers of empty tables (10, 100, 1000,
2000) and different numbers of sync workers (2, 4, 8, 16)

Scripts
-------

(PSA 4 scripts to implement this logic)

testrun script
- this does common setup (do_one_test_setup) and then the pub/sub
scripts (do_one_test_PUB and do_one_test_SUB -- see below) are run in
parallel
- repeat 10 times

do_one_test_setup script
- init and start instances
- ipc setup tables and procedures

do_one_test_PUB script
- ipc setup pub/sub
- table setup
- publishes the "busy_tbl", but then waits for the subscriber to
subscribe to only this one
- alters the publication to include all other tables (so subscriber
will see these only after the ALTER SUBSCRIPTION PUBLICATION REFRESH)
- enter a busy INSERT loop until it informed by the subscriber that
the test is finished

do_one_test_SUB script
- ipc setup pub/sub
- table setup
- subscribes only to "busy_tbl", then informs the publisher when that
is done (this will cause the publisher to commence the stay_busy loop)
- after it knows the publishing busy loop has started it does
- ALTER SUBSCRIPTION REFRESH PUBLICATION
- wait until all the tablesyncs are ready <=== This is the part that
is timed for the test RESULT

PROBLEM
-------

Looking at the output files (e.g. *.dat_PUB and *.dat_SUB) they seem
to confirm the tests are working how we wanted.

Unfortunately, there is some slot problem for the patched builds (both
designs #1 and #2). e.g. Search "ERROR" in the *.log files and see
many slot-related errors.

Please note - running these same scripts with HEAD build gave no such
errors. So it appears to be a patch problem.

------
Kind Regards
Peter Smith.
Fujitsu Australia

Attachments:

mel0721_100_0_2_1.dat_PUBapplication/octet-stream; name=mel0721_100_0_2_1.dat_PUBDownload
mel0721_100_0_2_1.dat_SUBapplication/octet-stream; name=mel0721_100_0_2_1.dat_SUBDownload
mel0721_100_0_2_1.datapplication/octet-stream; name=mel0721_100_0_2_1.datDownload
pub_100_0_2_1.logapplication/octet-stream; name=pub_100_0_2_1.logDownload
sub_100_0_2_1.logapplication/octet-stream; name=sub_100_0_2_1.logDownload
do_one_test_setup.shtext/x-sh; charset=US-ASCII; name=do_one_test_setup.shDownload
do_one_test_PUB.shtext/x-sh; charset=US-ASCII; name=do_one_test_PUB.shDownload
do_one_test_SUB.shtext/x-sh; charset=US-ASCII; name=do_one_test_SUB.shDownload
testrun.shtext/x-sh; charset=US-ASCII; name=testrun.shDownload
#135Peter Smith
smithpb2250@gmail.com
In reply to: Peter Smith (#134)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Fri, Jul 28, 2023 at 5:22 PM Peter Smith <smithpb2250@gmail.com> wrote:

Hi Melih,

BACKGROUND
----------

We wanted to compare performance for the 2 different reuse-worker
designs, when the apply worker is already busy handling other
replications, and then simultaneously the test table tablesyncs are
occurring.

To test this scenario, some test scripts were written (described
below). For comparisons, the scripts were then run using a build of
HEAD; design #1 (v21); design #2 (0718).

HOW THE TEST WORKS
------------------

Overview:
1. The apply worker is made to subscribe to a 'busy_tbl'.
2. After the SUBSCRIPTION is created, the publisher-side then loops
(forever) doing INSERTS into that busy_tbl.
3. While the apply worker is now busy, the subscriber does an ALTER
SUBSCRIPTION REFRESH PUBLICATION to subscribe to all the other test
tables.
4. We time how long it takes for all tablsyncs to complete
5. Repeat above for different numbers of empty tables (10, 100, 1000,
2000) and different numbers of sync workers (2, 4, 8, 16)

Scripts
-------

(PSA 4 scripts to implement this logic)

testrun script
- this does common setup (do_one_test_setup) and then the pub/sub
scripts (do_one_test_PUB and do_one_test_SUB -- see below) are run in
parallel
- repeat 10 times

do_one_test_setup script
- init and start instances
- ipc setup tables and procedures

do_one_test_PUB script
- ipc setup pub/sub
- table setup
- publishes the "busy_tbl", but then waits for the subscriber to
subscribe to only this one
- alters the publication to include all other tables (so subscriber
will see these only after the ALTER SUBSCRIPTION PUBLICATION REFRESH)
- enter a busy INSERT loop until it informed by the subscriber that
the test is finished

do_one_test_SUB script
- ipc setup pub/sub
- table setup
- subscribes only to "busy_tbl", then informs the publisher when that
is done (this will cause the publisher to commence the stay_busy loop)
- after it knows the publishing busy loop has started it does
- ALTER SUBSCRIPTION REFRESH PUBLICATION
- wait until all the tablesyncs are ready <=== This is the part that
is timed for the test RESULT

PROBLEM
-------

Looking at the output files (e.g. *.dat_PUB and *.dat_SUB) they seem
to confirm the tests are working how we wanted.

Unfortunately, there is some slot problem for the patched builds (both
designs #1 and #2). e.g. Search "ERROR" in the *.log files and see
many slot-related errors.

Please note - running these same scripts with HEAD build gave no such
errors. So it appears to be a patch problem.

Hi

FYI, here is some more information about ERRORs seen.

The patches were re-tested -- applied in stages (and also against the
different scripts) to identify where the problem was introduced. Below
are the observations:

~~~

Using original test scripts

1. Using only patch v21-0001
- no errors

2. Using only patch v21-0001+0002
- no errors

3. Using patch v21-0001+0002+0003
- no errors

~~~

Using the "busy loop" test scripts for long transactions

1. Using only patch v21-0001
- no errors

2. Using only patch v21-0001+0002
- gives errors for "no copy in progress issue"
e.g. ERROR: could not send data to WAL stream: no COPY in progress

3. Using patch v21-0001+0002+0003
- gives the same "no copy in progress issue" errors as above
e.g. ERROR: could not send data to WAL stream: no COPY in progress
- and also gives slot consistency point errors
e.g. ERROR: could not create replication slot
"pg_16700_sync_16514_7261998170966054867": ERROR: could not find
logical decoding starting point
e.g. LOG: could not drop replication slot
"pg_16700_sync_16454_7261998170966054867" on publisher: ERROR:
replication slot "pg_16700_sync_16454_7261998170966054867" does not
exist

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#136vignesh C
vignesh21@gmail.com
In reply to: Peter Smith (#135)
4 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Tue, 1 Aug 2023 at 09:44, Peter Smith <smithpb2250@gmail.com> wrote:

On Fri, Jul 28, 2023 at 5:22 PM Peter Smith <smithpb2250@gmail.com> wrote:

Hi Melih,

BACKGROUND
----------

We wanted to compare performance for the 2 different reuse-worker
designs, when the apply worker is already busy handling other
replications, and then simultaneously the test table tablesyncs are
occurring.

To test this scenario, some test scripts were written (described
below). For comparisons, the scripts were then run using a build of
HEAD; design #1 (v21); design #2 (0718).

HOW THE TEST WORKS
------------------

Overview:
1. The apply worker is made to subscribe to a 'busy_tbl'.
2. After the SUBSCRIPTION is created, the publisher-side then loops
(forever) doing INSERTS into that busy_tbl.
3. While the apply worker is now busy, the subscriber does an ALTER
SUBSCRIPTION REFRESH PUBLICATION to subscribe to all the other test
tables.
4. We time how long it takes for all tablsyncs to complete
5. Repeat above for different numbers of empty tables (10, 100, 1000,
2000) and different numbers of sync workers (2, 4, 8, 16)

Scripts
-------

(PSA 4 scripts to implement this logic)

testrun script
- this does common setup (do_one_test_setup) and then the pub/sub
scripts (do_one_test_PUB and do_one_test_SUB -- see below) are run in
parallel
- repeat 10 times

do_one_test_setup script
- init and start instances
- ipc setup tables and procedures

do_one_test_PUB script
- ipc setup pub/sub
- table setup
- publishes the "busy_tbl", but then waits for the subscriber to
subscribe to only this one
- alters the publication to include all other tables (so subscriber
will see these only after the ALTER SUBSCRIPTION PUBLICATION REFRESH)
- enter a busy INSERT loop until it informed by the subscriber that
the test is finished

do_one_test_SUB script
- ipc setup pub/sub
- table setup
- subscribes only to "busy_tbl", then informs the publisher when that
is done (this will cause the publisher to commence the stay_busy loop)
- after it knows the publishing busy loop has started it does
- ALTER SUBSCRIPTION REFRESH PUBLICATION
- wait until all the tablesyncs are ready <=== This is the part that
is timed for the test RESULT

PROBLEM
-------

Looking at the output files (e.g. *.dat_PUB and *.dat_SUB) they seem
to confirm the tests are working how we wanted.

Unfortunately, there is some slot problem for the patched builds (both
designs #1 and #2). e.g. Search "ERROR" in the *.log files and see
many slot-related errors.

Please note - running these same scripts with HEAD build gave no such
errors. So it appears to be a patch problem.

Hi

FYI, here is some more information about ERRORs seen.

The patches were re-tested -- applied in stages (and also against the
different scripts) to identify where the problem was introduced. Below
are the observations:

~~~

Using original test scripts

1. Using only patch v21-0001
- no errors

2. Using only patch v21-0001+0002
- no errors

3. Using patch v21-0001+0002+0003
- no errors

~~~

Using the "busy loop" test scripts for long transactions

1. Using only patch v21-0001
- no errors

2. Using only patch v21-0001+0002
- gives errors for "no copy in progress issue"
e.g. ERROR: could not send data to WAL stream: no COPY in progress

3. Using patch v21-0001+0002+0003
- gives the same "no copy in progress issue" errors as above
e.g. ERROR: could not send data to WAL stream: no COPY in progress
- and also gives slot consistency point errors
e.g. ERROR: could not create replication slot
"pg_16700_sync_16514_7261998170966054867": ERROR: could not find
logical decoding starting point
e.g. LOG: could not drop replication slot
"pg_16700_sync_16454_7261998170966054867" on publisher: ERROR:
replication slot "pg_16700_sync_16454_7261998170966054867" does not
exist

I agree that "no copy in progress issue" issue has nothing to do with
0001 patch. This issue is present with the 0002 patch.
In the case when the tablesync worker has to apply the transactions
after the table is synced, the tablesync worker sends the feedback of
writepos, applypos and flushpos which results in "No copy in progress"
error as the stream has ended already. Fixed it by exiting the
streaming loop if the tablesync worker is done with the
synchronization. The attached 0004 patch has the changes for the same.
The rest of v22 patches are the same patch that were posted by Melih
in the earlier mail.

Regards,
Vignesh

Attachments:

v22-0002-Reuse-Tablesync-Workers.patchtext/x-patch; charset=US-ASCII; name=v22-0002-Reuse-Tablesync-Workers.patchDownload
From bd18bd59be0a263cb3385353e73ec25542bdeff2 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:04:46 +0300
Subject: [PATCH v22 2/3] Reuse Tablesync Workers

Before this patch, tablesync workers were capable of syncing only one
table. For each table, a new sync worker was launched and that worker would
exit when done processing the table.

Now, tablesync workers are not limited to processing only one
table. When done, they can move to processing another table in
the same subscription.

If there is a table that needs to be synced, an available tablesync
worker picks up that table and syncs it. Each tablesync worker
continues to pick new tables to sync until there are no tables left
requiring synchronization. If there was no available worker to
process the table, then a new tablesync worker will be launched,
provided the number of tablesync workers for the subscription does not
exceed max_sync_workers_per_subscription.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/replication/logical/launcher.c  |   1 +
 src/backend/replication/logical/tablesync.c | 121 ++++++++++++++++++--
 src/backend/replication/logical/worker.c    |  40 ++++++-
 src/include/replication/worker_internal.h   |   2 +
 4 files changed, 149 insertions(+), 15 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e231fa7f95..25dd06b8af 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -440,6 +440,7 @@ retry:
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
+	worker->relsync_completed = false;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index ff859e0910..63b5bed88a 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -134,10 +134,12 @@ static StringInfo copybuf = NULL;
 
 /*
  * Exit routine for synchronization worker.
+ *
+ * If reuse_worker is false, at the conclusion of this function the worker
+ * process will exit.
  */
 static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+finish_sync_worker(bool reuse_worker)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -149,21 +151,42 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
+	/*
+	 * Disconnect from the publisher otherwise reusing the sync worker can
+	 * error due to exceeding max_wal_senders.
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		walrcv_disconnect(LogRepWorkerWalRcvConn);
+		LogRepWorkerWalRcvConn = NULL;
+	}
+
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
 	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
+	if (reuse_worker)
+	{
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\" will be reused to sync table \"%s\" with relid %u.",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
+	}
+	else
+	{
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\" has finished",
+						MySubscription->name)));
+	}
 	CommitTransactionCommand();
 
 	/* Find the leader apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
 
 	/* Stop gracefully */
-	proc_exit(0);
+	if (!reuse_worker)
+		proc_exit(0);
 }
 
 /*
@@ -383,7 +406,15 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 */
 		replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
+		/* Sync worker has completed synchronization of the current table. */
+		MyLogicalRepWorker->relsync_completed = true;
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization for subscription \"%s\", relation \"%s\" with relid %u has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
+		CommitTransactionCommand();
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -1288,7 +1319,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		case SUBREL_STATE_SYNCDONE:
 		case SUBREL_STATE_READY:
 		case SUBREL_STATE_UNKNOWN:
-			finish_sync_worker();	/* doesn't return */
+			finish_sync_worker(false);	/* doesn't return */
 	}
 
 	/* Calculate the name of the tablesync slot. */
@@ -1646,6 +1677,8 @@ run_tablesync_worker()
 	char	   *slotname = NULL;
 	WalRcvStreamOptions options;
 
+	MyLogicalRepWorker->relsync_completed = false;
+
 	start_table_sync(&origin_startpos, &slotname);
 
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
@@ -1668,12 +1701,78 @@ void
 TablesyncWorkerMain(Datum main_arg)
 {
 	int			worker_slot = DatumGetInt32(main_arg);
+	bool 		done = false;
 
 	SetupApplyOrSyncWorker(worker_slot);
 
-	run_tablesync_worker();
+	/*
+	 * The loop where worker does its job. It loops until there is no relation
+	 * left to sync.
+	 */
+	for (;!done;)
+	{
+		List	   *rstates;
+		ListCell   *lc;
+
+		run_tablesync_worker();
+
+		if (IsTransactionState())
+			CommitTransactionCommand();
+
+		if (MyLogicalRepWorker->relsync_completed)
+		{
+			/*
+			 * This tablesync worker is 'done' unless another table that needs
+			 * syncing is found.
+			 */
+			done = true;
+
+			/* This transaction will be committed by finish_sync_worker. */
+			StartTransactionCommand();
+
+			/*
+			 * Check if there is any table whose relation state is still INIT.
+			 * If a table in INIT state is found, the worker will not be
+			 * finished, it will be reused instead.
+			 */
+			rstates = GetSubscriptionRelations(MySubscription->oid, true);
+
+			foreach(lc, rstates)
+			{
+				SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
+
+				if (rstate->state == SUBREL_STATE_SYNCDONE)
+					continue;
+
+				/*
+				 * Take exclusive lock to prevent any other sync worker from
+				 * picking the same table.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
+
+				/*
+				 * Pick the table for the next run if it is not already picked
+				 * up by another worker.
+				 */
+				if (!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+				{
+					/* Update worker state for the next table */
+					MyLogicalRepWorker->relid = rstate->relid;
+					MyLogicalRepWorker->relstate = rstate->state;
+					MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+					LWLockRelease(LogicalRepWorkerLock);
+
+					/* Found a table for next iteration */
+					finish_sync_worker(true);
+					done = false;
+					break;
+				}
+				LWLockRelease(LogicalRepWorkerLock);
+			}
+		}
+	}
 
-	finish_sync_worker();
+	finish_sync_worker(false);
 }
 
 /*
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 8ed6fa7acf..d25bf5bea2 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3621,6 +3621,23 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				if (am_tablesync_worker())
+				{
+					/*
+					 * apply_dispatch() may have gone into apply_handle_commit()
+					 * which can call process_syncing_tables_for_sync.
+					 *
+					 * process_syncing_tables_for_sync decides whether the sync of
+					 * the current table is completed. If it is completed,
+					 * streaming must be already ended. So, we can break the loop.
+					 */
+					if (MyLogicalRepWorker->relsync_completed)
+					{
+						endofstream = true;
+						break;
+					}
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
@@ -3640,6 +3657,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+
+			if (am_tablesync_worker())
+				/*
+				 * If relsync_completed is true, this means that the tablesync
+				 * worker is done with synchronization. Streaming has already been
+				 * ended by process_syncing_tables_for_sync. We should move to the
+				 * next table if needed, or exit.
+				 */
+				if (MyLogicalRepWorker->relsync_completed)
+					endofstream = true;
 		}
 
 		/* Cleanup the memory. */
@@ -3742,8 +3769,12 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * End streaming here for only apply workers. Ending streaming for
+	 * tablesync workers is deferred until the worker exits its main loop.
+	 */
+	if (!am_tablesync_worker())
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
 }
 
 /*
@@ -4617,9 +4648,10 @@ InitializeLogRepWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has started",
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" with relid %u has started",
 						MySubscription->name,
-						get_rel_name(MyLogicalRepWorker->relid))));
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
 	else
 		ereport(LOG,
 				(errmsg("logical replication apply worker for subscription \"%s\" has started",
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index fdbc1183f2..39b1721dee 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -56,6 +56,8 @@ typedef struct LogicalRepWorker
 	char		relstate;
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
+	bool		relsync_completed; /* has tablesync finished syncing
+									* the assigned table? */
 
 	/*
 	 * Used to create the changes and subxact files for the streaming
-- 
2.25.1

v22-0003-Reuse-connection-when-tablesync-workers-change-t.patchtext/x-patch; charset=US-ASCII; name=v22-0003-Reuse-connection-when-tablesync-workers-change-t.patchDownload
From d15ae6faa94033a4fa66f9d29c524d459d0f7f71 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:13:52 +0300
Subject: [PATCH v22 3/3] Reuse connection when tablesync workers change the
 target

Previously tablesync workers establish new connections when it changes the syncing
table, but this might have additional overhead. This patch allows the existing
connection to be reused.

As for the publisher node, this patch allows to reuse logical walsender processes
after the streaming is done once.
---
 src/backend/replication/logical/launcher.c  |  1 +
 src/backend/replication/logical/tablesync.c | 60 ++++++++++++++-------
 src/backend/replication/logical/worker.c    | 18 ++++---
 src/backend/replication/walsender.c         |  7 +++
 src/include/replication/worker_internal.h   |  3 ++
 5 files changed, 62 insertions(+), 27 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 25dd06b8af..657e446eaf 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
 	worker->relsync_completed = false;
+	worker->slotnum = slot;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 63b5bed88a..45e753b189 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -151,16 +151,6 @@ finish_sync_worker(bool reuse_worker)
 		pgstat_report_stat(true);
 	}
 
-	/*
-	 * Disconnect from the publisher otherwise reusing the sync worker can
-	 * error due to exceeding max_wal_senders.
-	 */
-	if (LogRepWorkerWalRcvConn != NULL)
-	{
-		walrcv_disconnect(LogRepWorkerWalRcvConn);
-		LogRepWorkerWalRcvConn = NULL;
-	}
-
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
@@ -1268,6 +1258,27 @@ ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
 			 relid, GetSystemIdentifier());
 }
 
+/*
+ * Determine the application_name for tablesync workers.
+ *
+ * Previously, the replication slot name was used as application_name. Since
+ * it's possible to reuse tablesync workers now, a tablesync worker can handle
+ * several different replication slots during its lifetime. Therefore, we
+ * cannot use the slot name as application_name anymore. Instead, the slot
+ * number of the tablesync worker is used as a part of the application_name.
+ *
+ * FIXME: if the tablesync worker starts to reuse the replication slot during
+ * synchronization, we should again use the replication slot name as
+ * application_name.
+ */
+static void
+ApplicationNameForTablesync(Oid suboid, int worker_slot,
+							char *application_name, Size szapp)
+{
+	snprintf(application_name, szapp, "pg_%u_sync_%i_" UINT64_FORMAT, suboid,
+			 worker_slot, GetSystemIdentifier());
+}
+
 /*
  * Start syncing the table in the sync worker.
  *
@@ -1329,15 +1340,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 									slotname,
 									NAMEDATALEN);
 
-	/*
-	 * Here we use the slot name instead of the subscription name as the
-	 * application_name, so that it is different from the leader apply worker,
-	 * so that synchronous replication can distinguish them.
-	 */
-	LogRepWorkerWalRcvConn =
-		walrcv_connect(MySubscription->conninfo, true,
-					   must_use_password,
-					   slotname, &err);
+	/* Connect to the publisher if haven't done so already. */
+	if (LogRepWorkerWalRcvConn == NULL)
+	{
+		char application_name[NAMEDATALEN];
+
+		/*
+		 * The application_name must differ from the subscription name (used by
+		 * the leader apply worker) because synchronous replication has to be
+		 * able to distinguish this worker from the leader apply worker.
+		 */
+		ApplicationNameForTablesync(MySubscription->oid,
+									MyLogicalRepWorker->slotnum,
+									application_name,
+									NAMEDATALEN);
+		LogRepWorkerWalRcvConn =
+			walrcv_connect(MySubscription->conninfo, true,
+						   must_use_password,
+						   application_name, &err);
+	}
+
 	if (LogRepWorkerWalRcvConn == NULL)
 		ereport(ERROR,
 				(errcode(ERRCODE_CONNECTION_FAILURE),
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index d25bf5bea2..c482f707dc 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3494,20 +3494,22 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	ErrorContextCallback errcallback;
 
 	/*
-	 * Init the ApplyMessageContext which we clean up after each replication
-	 * protocol message.
+	 * Init the ApplyMessageContext if needed. This context is cleaned up
+	 * after each replication protocol message.
 	 */
-	ApplyMessageContext = AllocSetContextCreate(ApplyContext,
-												"ApplyMessageContext",
-												ALLOCSET_DEFAULT_SIZES);
+	if (!ApplyMessageContext)
+		ApplyMessageContext = AllocSetContextCreate(ApplyContext,
+													"ApplyMessageContext",
+													ALLOCSET_DEFAULT_SIZES);
 
 	/*
 	 * This memory context is used for per-stream data when the streaming mode
 	 * is enabled. This context is reset on each stream stop.
 	 */
-	LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
-													"LogicalStreamingContext",
-													ALLOCSET_DEFAULT_SIZES);
+	if (!LogicalStreamingContext)
+		LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
+														"LogicalStreamingContext",
+														ALLOCSET_DEFAULT_SIZES);
 
 	/* mark as idle, before starting to loop */
 	pgstat_report_activity(STATE_IDLE, NULL);
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index d27ef2985d..2f3e93cc40 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1830,7 +1830,14 @@ exec_replication_command(const char *cmd_string)
 				if (cmd->kind == REPLICATION_KIND_PHYSICAL)
 					StartReplication(cmd);
 				else
+				{
+					/*
+					 * Reset flags because reusing tablesync workers can mean
+					 * this is the second time here.
+					 */
+					streamingDoneSending = streamingDoneReceiving = false;
 					StartLogicalReplication(cmd);
+				}
 
 				/* dupe, but necessary per libpqrcv_endstreaming */
 				EndReplicationCommand(cmdtag);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 39b1721dee..b88ff6a646 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -39,6 +39,9 @@ typedef struct LogicalRepWorker
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
+	/* Slot number of this worker. */
+	int			slotnum;
+
 	/* Pointer to proc array. NULL if not running. */
 	PGPROC	   *proc;
 
-- 
2.25.1

v22-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchtext/x-patch; charset=US-ASCII; name=v22-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchDownload
From 86a02f4fd54d5d5db89df21e327777eaf2aeaed3 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Mon, 5 Jun 2023 15:04:41 +0300
Subject: [PATCH v22 1/3] Refactor to split Apply and Tablesync Workers

Both apply and tablesync workers were using ApplyWorkerMain() as entry
point. As the name implies, ApplyWorkerMain() should be considered as
the main function for apply workers. Tablesync worker's path was hidden
and does not have enough in common to share the same main function with
apply worker.

Also, most of the code shared by both worker types is already combined
in LogicalRepApplyLoop(). There is no need to combine the rest in
ApplyWorkerMain() anymore.

This patch introduces TablesyncWorkerMain() as a new entry point for
tablesync workers. This aims to increase code readability and help to
the upcoming reuse tablesync worker improvements.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/postmaster/bgworker.c             |   3 +
 .../replication/logical/applyparallelworker.c |   2 +-
 src/backend/replication/logical/launcher.c    |  32 +-
 src/backend/replication/logical/tablesync.c   |  94 ++++-
 src/backend/replication/logical/worker.c      | 380 ++++++++----------
 src/include/replication/logicalworker.h       |   1 +
 src/include/replication/worker_internal.h     |  15 +-
 7 files changed, 303 insertions(+), 224 deletions(-)

diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 5b4bd71694..505e38376c 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -131,6 +131,9 @@ static const struct
 	},
 	{
 		"ParallelApplyWorkerMain", ParallelApplyWorkerMain
+	},
+	{
+		"TablesyncWorkerMain", TablesyncWorkerMain
 	}
 };
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 6fb96148f4..1d4e83c4c1 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -942,7 +942,7 @@ ParallelApplyWorkerMain(Datum main_arg)
 	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
 		MyLogicalRepWorker->reply_time = 0;
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 542af7d863..e231fa7f95 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -459,24 +459,30 @@ retry:
 	snprintf(bgw.bgw_library_name, MAXPGPATH, "postgres");
 
 	if (is_parallel_apply_worker)
+	{
 		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ParallelApplyWorkerMain");
-	else
-		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
-
-	if (OidIsValid(relid))
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication worker for subscription %u sync %u", subid, relid);
-	else if (is_parallel_apply_worker)
+				 "logical replication parallel apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
+	}
+	else if (OidIsValid(relid))
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "TablesyncWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication parallel apply worker for subscription %u", subid);
+				 "logical replication tablesync worker for subscription %u sync %u",
+				 subid,
+				 relid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication tablesync worker");
+	}
 	else
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication apply worker for subscription %u", subid);
-
-	if (is_parallel_apply_worker)
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
-	else
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication worker");
+				 "logical replication apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication apply worker");
+	}
 
 	bgw.bgw_restart_time = BGW_NEVER_RESTART;
 	bgw.bgw_notify_pid = MyProcPid;
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 6d461654ab..ff859e0910 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -104,17 +104,21 @@
 #include "nodes/makefuncs.h"
 #include "parser/parse_relation.h"
 #include "pgstat.h"
+#include "postmaster/interrupt.h"
 #include "replication/logicallauncher.h"
 #include "replication/logicalrelation.h"
+#include "replication/logicalworker.h"
 #include "replication/walreceiver.h"
 #include "replication/worker_internal.h"
 #include "replication/slot.h"
 #include "replication/origin.h"
 #include "storage/ipc.h"
 #include "storage/lmgr.h"
+#include "tcop/tcopprot.h"
 #include "utils/acl.h"
 #include "utils/array.h"
 #include "utils/builtins.h"
+#include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
 #include "utils/rls.h"
@@ -1241,7 +1245,7 @@ ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
  *
  * The returned slot name is palloc'ed in current memory context.
  */
-char *
+static char *
 LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 {
 	char	   *slotname;
@@ -1584,6 +1588,94 @@ FetchTableStates(bool *started_tx)
 	return has_subrels;
 }
 
+/*
+ * Execute the initial sync with error handling. Disable the subscription,
+ * if it's required.
+ *
+ * Allocate the slot name in long-lived context on return. Note that we don't
+ * handle FATAL errors which are probably because of system resource error and
+ * are not repeatable.
+ */
+static void
+start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+{
+	char	   *syncslotname = NULL;
+
+	Assert(am_tablesync_worker());
+
+	PG_TRY();
+	{
+		/* Call initial sync. */
+		syncslotname = LogicalRepSyncTableStart(origin_startpos);
+	}
+	PG_CATCH();
+	{
+		if (MySubscription->disableonerr)
+			DisableSubscriptionAndExit();
+		else
+		{
+			/*
+			 * Report the worker failed during table synchronization. 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();
+
+	/* allocate slot name in long-lived context */
+	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+	pfree(syncslotname);
+}
+
+/*
+ * Runs the tablesync worker.
+ *
+ * It starts syncing tables. After a successful sync, sets streaming options
+ * and starts streaming to catchup.
+ */
+static void
+run_tablesync_worker()
+{
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *slotname = NULL;
+	WalRcvStreamOptions options;
+
+	start_table_sync(&origin_startpos, &slotname);
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   sizeof(originname));
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(&options, slotname, &origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+
+	/* Start applying changes to catchup. */
+	start_apply(origin_startpos);
+}
+
+/* Logical Replication Tablesync worker entry point */
+void
+TablesyncWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
+
+	SetupApplyOrSyncWorker(worker_slot);
+
+	run_tablesync_worker();
+
+	finish_sync_worker();
+}
+
 /*
  * If the subscription has no tables then return false.
  *
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 832b1cf764..8ed6fa7acf 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -396,8 +396,6 @@ static void stream_close_file(void);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
-static void DisableSubscriptionAndExit(void);
-
 static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
 static void apply_handle_insert_internal(ApplyExecutionData *edata,
 										 ResultRelInfo *relinfo,
@@ -4327,6 +4325,57 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+/*
+ * Sets streaming options including replication slot name and origin start
+ * position. Workers need these options for logical replication.
+ */
+void
+set_stream_options(WalRcvStreamOptions *options,
+				   char *slotname,
+				   XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4361,24 +4410,18 @@ TwoPhaseTransactionGid(Oid subid, TransactionId xid, char *gid, int szgid)
 }
 
 /*
- * Execute the initial sync with error handling. Disable the subscription,
- * if it's required.
+ * Common function to run the apply loop with error handling. Disable the
+ * subscription, if necessary.
  *
- * Allocate the slot name in long-lived context on return. Note that we don't
- * handle FATAL errors which are probably because of system resource error and
- * are not repeatable.
+ * Note that we don't handle FATAL errors which are probably because
+ * of system resource error and are not repeatable.
  */
-static void
-start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+void
+start_apply(XLogRecPtr origin_startpos)
 {
-	char	   *syncslotname = NULL;
-
-	Assert(am_tablesync_worker());
-
 	PG_TRY();
 	{
-		/* Call initial sync. */
-		syncslotname = LogicalRepSyncTableStart(origin_startpos);
+		LogicalRepApplyLoop(origin_startpos);
 	}
 	PG_CATCH();
 	{
@@ -4387,65 +4430,132 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 		else
 		{
 			/*
-			 * Report the worker failed during table synchronization. Abort
-			 * the current transaction so that the stats message is sent in an
+			 * Report the worker failed while applying changes. Abort the
+			 * current transaction so that the stats message is sent in an
 			 * idle state.
 			 */
 			AbortOutOfAnyTransaction();
-			pgstat_report_subscription_error(MySubscription->oid, false);
+			pgstat_report_subscription_error(MySubscription->oid, !am_tablesync_worker());
 
 			PG_RE_THROW();
 		}
 	}
 	PG_END_TRY();
-
-	/* allocate slot name in long-lived context */
-	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
-	pfree(syncslotname);
 }
 
 /*
- * Run the apply loop with error handling. Disable the subscription,
- * if necessary.
+ * Runs the leader apply worker.
  *
- * Note that we don't handle FATAL errors which are probably because
- * of system resource error and are not repeatable.
+ * It sets up replication origin, streaming options and then starts streaming.
  */
 static void
-start_apply(XLogRecPtr origin_startpos)
+run_apply_worker()
 {
-	PG_TRY();
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *slotname = NULL;
+	WalRcvStreamOptions options;
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+	bool		must_use_password;
+
+	slotname = MySubscription->slotname;
+
+	/*
+	 * This shouldn't happen if the subscription is enabled, but guard
+	 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+	 * crash if slot is NULL.)
+	 */
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	/* Setup replication origin tracking. */
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, sizeof(originname));
+	StartTransactionCommand();
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	origin_startpos = replorigin_session_get_progress(false);
+
+	/* Is the use of a password mandatory? */
+	must_use_password = MySubscription->passwordrequired &&
+		!superuser_arg(MySubscription->owner);
+
+	/* Note that the superuser_arg call can access the DB */
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											must_use_password,
+											MySubscription->name, &err);
+
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+	 * We don't really use the output identify_system for anything but it
+	 * does some initializations on the upstream so let's still call it.
+	 */
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(&options, slotname, &origin_startpos);
+
+	/*
+		* Even when the two_phase mode is requested by the user, it remains
+		* as the tri-state PENDING until all tablesyncs have reached READY
+		* state. Only then, can it become ENABLED.
+		*
+		* Note: If the subscription has no tables then leave the state as
+		* PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+		* work.
+		*/
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
 	{
-		LogicalRepApplyLoop(origin_startpos);
+		/* Start streaming with two_phase enabled */
+		options.proto.logical.twophase = true;
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
 	}
-	PG_CATCH();
+	else
 	{
-		if (MySubscription->disableonerr)
-			DisableSubscriptionAndExit();
-		else
-		{
-			/*
-			 * Report the worker failed while applying changes. Abort the
-			 * current transaction so that the stats message is sent in an
-			 * idle state.
-			 */
-			AbortOutOfAnyTransaction();
-			pgstat_report_subscription_error(MySubscription->oid, !am_tablesync_worker());
-
-			PG_RE_THROW();
-		}
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
 	}
-	PG_END_TRY();
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+
+	/* Run the main loop. */
+	start_apply(origin_startpos);
 }
 
 /*
- * Common initialization for leader apply worker and parallel apply worker.
+ * Common initialization for leader apply worker, parallel apply worker and
+ * tablesync worker.
  *
  * Initialize the database connection, in-memory subscription and necessary
  * config options.
  */
 void
-InitializeApplyWorker(void)
+InitializeLogRepWorker(void)
 {
 	MemoryContext oldctx;
 
@@ -4518,22 +4628,15 @@ InitializeApplyWorker(void)
 	CommitTransactionCommand();
 }
 
-/* Logical Replication Apply worker entry point */
+/* Common function to setup the leader apply or tablesync worker. */
 void
-ApplyWorkerMain(Datum main_arg)
+SetupApplyOrSyncWorker(int worker_slot)
 {
-	int			worker_slot = DatumGetInt32(main_arg);
-	char		originname[NAMEDATALEN];
-	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
-	char	   *myslotname = NULL;
-	WalRcvStreamOptions options;
-	int			server_version;
-
-	InitializingApplyWorker = true;
-
 	/* Attach to slot */
 	logicalrep_worker_attach(worker_slot);
 
+	Assert(am_tablesync_worker() || am_leader_apply_worker());
+
 	/* Setup signal handling */
 	pqsignal(SIGHUP, SignalHandlerForConfigReload);
 	pqsignal(SIGTERM, die);
@@ -4551,79 +4654,12 @@ ApplyWorkerMain(Datum main_arg)
 	/* Load the libpq-specific functions */
 	load_file("libpqwalreceiver", false);
 
-	InitializeApplyWorker();
-
-	InitializingApplyWorker = false;
+	InitializeLogRepWorker();
 
 	/* Connect to the origin and start the replication. */
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-		bool		must_use_password;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-
-		/* Is the use of a password mandatory? */
-		must_use_password = MySubscription->passwordrequired &&
-			!superuser_arg(MySubscription->owner);
-
-		/* Note that the superuser_arg call can access the DB */
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												must_use_password,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
 	 * the subscription relation state.
@@ -4631,91 +4667,21 @@ ApplyWorkerMain(Datum main_arg)
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
+}
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
-
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
-
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-
-	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
-	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
+/* Logical Replication Apply worker entry point */
+void
+ApplyWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
 
-	if (!am_tablesync_worker())
-	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
-		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+	InitializingApplyWorker = true;
 
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
-		}
-		else
-		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-		}
+	SetupApplyOrSyncWorker(worker_slot);
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-								 MySubscription->name,
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-								 "?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
+	InitializingApplyWorker = false;
 
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+	run_apply_worker();
 
 	proc_exit(0);
 }
@@ -4724,7 +4690,7 @@ ApplyWorkerMain(Datum main_arg)
  * After error recovery, disable the subscription in a new transaction
  * and exit cleanly.
  */
-static void
+void
 DisableSubscriptionAndExit(void)
 {
 	/*
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 39588da79f..bbd71d0b42 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -18,6 +18,7 @@ extern PGDLLIMPORT volatile sig_atomic_t ParallelApplyMessagePending;
 
 extern void ApplyWorkerMain(Datum main_arg);
 extern void ParallelApplyWorkerMain(Datum main_arg);
+extern void TablesyncWorkerMain(Datum main_arg);
 
 extern bool IsLogicalWorker(void);
 extern bool IsLogicalParallelApplyWorker(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 343e781896..fdbc1183f2 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -19,6 +19,7 @@
 #include "datatype/timestamp.h"
 #include "miscadmin.h"
 #include "replication/logicalrelation.h"
+#include "replication/walreceiver.h"
 #include "storage/buffile.h"
 #include "storage/fileset.h"
 #include "storage/lock.h"
@@ -243,7 +244,6 @@ extern int	logicalrep_sync_worker_count(Oid subid);
 
 extern void ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 											   char *originname, Size szoriginname);
-extern char *LogicalRepSyncTableStart(XLogRecPtr *origin_startpos);
 
 extern bool AllTablesyncsReady(void);
 extern void UpdateTwoPhaseState(Oid suboid, char new_state);
@@ -265,7 +265,17 @@ extern void maybe_reread_subscription(void);
 
 extern void stream_cleanup_files(Oid subid, TransactionId xid);
 
-extern void InitializeApplyWorker(void);
+extern void set_stream_options(WalRcvStreamOptions *options,
+							   char *slotname,
+							   XLogRecPtr *origin_startpos);
+
+extern void start_apply(XLogRecPtr origin_startpos);
+
+extern void InitializeLogRepWorker(void);
+
+extern void SetupApplyOrSyncWorker(int worker_slot);
+
+extern void DisableSubscriptionAndExit(void);
 
 extern void store_flush_position(XLogRecPtr remote_lsn, XLogRecPtr local_lsn);
 
@@ -305,6 +315,7 @@ extern void pa_decr_and_wait_stream_block(void);
 extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
 						   XLogRecPtr remote_lsn);
 
+
 #define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)
 
 static inline bool
-- 
2.25.1

0004-Fix-for-Table-sync-worker-sending-the-feedback-even-.patchtext/x-patch; charset=US-ASCII; name=0004-Fix-for-Table-sync-worker-sending-the-feedback-even-.patchDownload
From 2c5223f7eb24798b1da0f5a080fe9c9038a250b5 Mon Sep 17 00:00:00 2001
From: Vignesh C <vignesh21@gmail.com>
Date: Mon, 31 Jul 2023 22:05:34 +0530
Subject: [PATCH] Fix for Table sync worker sending the feedback even after the
 streaming is ended.

In the case when the tablesync worker has to apply the transactions
after the table is synced, the tablesync worker sends the feedback of
writepos, applypos and flushpos which results in "No copy in progress" error
as the stream was ended already. Fixed it by exiting the streaming loop
if the tablesync worker is done with the synchronization.
---
 src/backend/replication/logical/worker.c | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index d25bf5bea2..23ed4adfa4 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3634,7 +3634,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					if (MyLogicalRepWorker->relsync_completed)
 					{
 						endofstream = true;
-						break;
+						goto streaming_done;
 					}
 				}
 
@@ -3669,6 +3669,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					endofstream = true;
 		}
 
+streaming_done:
 		/* Cleanup the memory. */
 		MemoryContextResetAndDeleteChildren(ApplyMessageContext);
 		MemoryContextSwitchTo(TopMemoryContext);
-- 
2.34.1

#137Amit Kapila
amit.kapila16@gmail.com
In reply to: Peter Smith (#135)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Tue, Aug 1, 2023 at 9:44 AM Peter Smith <smithpb2250@gmail.com> wrote:

FYI, here is some more information about ERRORs seen.

The patches were re-tested -- applied in stages (and also against the
different scripts) to identify where the problem was introduced. Below
are the observations:

~~~

Using original test scripts

1. Using only patch v21-0001
- no errors

2. Using only patch v21-0001+0002
- no errors

3. Using patch v21-0001+0002+0003
- no errors

~~~

Using the "busy loop" test scripts for long transactions

1. Using only patch v21-0001
- no errors

2. Using only patch v21-0001+0002
- gives errors for "no copy in progress issue"
e.g. ERROR: could not send data to WAL stream: no COPY in progress

3. Using patch v21-0001+0002+0003
- gives the same "no copy in progress issue" errors as above
e.g. ERROR: could not send data to WAL stream: no COPY in progress
- and also gives slot consistency point errors
e.g. ERROR: could not create replication slot
"pg_16700_sync_16514_7261998170966054867": ERROR: could not find
logical decoding starting point
e.g. LOG: could not drop replication slot
"pg_16700_sync_16454_7261998170966054867" on publisher: ERROR:
replication slot "pg_16700_sync_16454_7261998170966054867" does not
exist

I think we are getting the error (ERROR: could not find logical
decoding starting point) because we wouldn't have waited for WAL to
become available before reading it. It could happen due to the
following code:
WalSndWaitForWal()
{
...
if (streamingDoneReceiving && streamingDoneSending &&
!pq_is_send_pending())
break;
..
}

Now, it seems that in 0003 patch, instead of resetting flags
streamingDoneSending, and streamingDoneReceiving before start
replication, we should reset before create logical slots because we
need to read the WAL during that time as well to find the consistent
point.

--
With Regards,
Amit Kapila.

#138Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Amit Kapila (#137)
3 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi,

Amit Kapila <amit.kapila16@gmail.com>, 2 Ağu 2023 Çar, 12:01 tarihinde şunu
yazdı:

I think we are getting the error (ERROR: could not find logical
decoding starting point) because we wouldn't have waited for WAL to
become available before reading it. It could happen due to the
following code:
WalSndWaitForWal()
{
...
if (streamingDoneReceiving && streamingDoneSending &&
!pq_is_send_pending())
break;
..
}

Now, it seems that in 0003 patch, instead of resetting flags
streamingDoneSending, and streamingDoneReceiving before start
replication, we should reset before create logical slots because we
need to read the WAL during that time as well to find the consistent
point.

Thanks for the suggestion Amit. I've been looking into this recently and
couldn't figure out the cause until now.
I quickly made the fix in 0003. Seems like it resolved the "could not find
logical decoding starting point" errors.

vignesh C <vignesh21@gmail.com>, 1 Ağu 2023 Sal, 09:32 tarihinde şunu yazdı:

I agree that "no copy in progress issue" issue has nothing to do with
0001 patch. This issue is present with the 0002 patch.
In the case when the tablesync worker has to apply the transactions
after the table is synced, the tablesync worker sends the feedback of
writepos, applypos and flushpos which results in "No copy in progress"
error as the stream has ended already. Fixed it by exiting the
streaming loop if the tablesync worker is done with the
synchronization. The attached 0004 patch has the changes for the same.
The rest of v22 patches are the same patch that were posted by Melih
in the earlier mail.

Thanks for the fix. I placed it into 0002 with a slight change as follows:

- send_feedback(last_received, false, false);

+ if (!MyLogicalRepWorker->relsync_completed)
+ send_feedback(last_received, false, false);

IMHO relsync_completed means simply the same with streaming_done, that's
why I wanted to check that flag instead of an additional goto statement.
Does it make sense to you as well?

Thanks,
--
Melih Mutlu
Microsoft

Attachments:

v23-0002-Reuse-Tablesync-Workers.patchapplication/octet-stream; name=v23-0002-Reuse-Tablesync-Workers.patchDownload
From 61bb66fbb0946c23baa67634cf43c23982c0fb82 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:04:46 +0300
Subject: [PATCH v23 2/3] Reuse Tablesync Workers

Before this patch, tablesync workers were capable of syncing only one
table. For each table, a new sync worker was launched and that worker would
exit when done processing the table.

Now, tablesync workers are not limited to processing only one
table. When done, they can move to processing another table in
the same subscription.

If there is a table that needs to be synced, an available tablesync
worker picks up that table and syncs it. Each tablesync worker
continues to pick new tables to sync until there are no tables left
requiring synchronization. If there was no available worker to
process the table, then a new tablesync worker will be launched,
provided the number of tablesync workers for the subscription does not
exceed max_sync_workers_per_subscription.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/replication/logical/launcher.c  |   1 +
 src/backend/replication/logical/tablesync.c | 121 ++++++++++++++++++--
 src/backend/replication/logical/worker.c    |  43 ++++++-
 src/include/replication/worker_internal.h   |   2 +
 4 files changed, 151 insertions(+), 16 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e231fa7f95..25dd06b8af 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -440,6 +440,7 @@ retry:
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
+	worker->relsync_completed = false;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index ff859e0910..63b5bed88a 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -134,10 +134,12 @@ static StringInfo copybuf = NULL;
 
 /*
  * Exit routine for synchronization worker.
+ *
+ * If reuse_worker is false, at the conclusion of this function the worker
+ * process will exit.
  */
 static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+finish_sync_worker(bool reuse_worker)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -149,21 +151,42 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
+	/*
+	 * Disconnect from the publisher otherwise reusing the sync worker can
+	 * error due to exceeding max_wal_senders.
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		walrcv_disconnect(LogRepWorkerWalRcvConn);
+		LogRepWorkerWalRcvConn = NULL;
+	}
+
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
 	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
+	if (reuse_worker)
+	{
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\" will be reused to sync table \"%s\" with relid %u.",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
+	}
+	else
+	{
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\" has finished",
+						MySubscription->name)));
+	}
 	CommitTransactionCommand();
 
 	/* Find the leader apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
 
 	/* Stop gracefully */
-	proc_exit(0);
+	if (!reuse_worker)
+		proc_exit(0);
 }
 
 /*
@@ -383,7 +406,15 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 */
 		replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
+		/* Sync worker has completed synchronization of the current table. */
+		MyLogicalRepWorker->relsync_completed = true;
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization for subscription \"%s\", relation \"%s\" with relid %u has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
+		CommitTransactionCommand();
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -1288,7 +1319,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		case SUBREL_STATE_SYNCDONE:
 		case SUBREL_STATE_READY:
 		case SUBREL_STATE_UNKNOWN:
-			finish_sync_worker();	/* doesn't return */
+			finish_sync_worker(false);	/* doesn't return */
 	}
 
 	/* Calculate the name of the tablesync slot. */
@@ -1646,6 +1677,8 @@ run_tablesync_worker()
 	char	   *slotname = NULL;
 	WalRcvStreamOptions options;
 
+	MyLogicalRepWorker->relsync_completed = false;
+
 	start_table_sync(&origin_startpos, &slotname);
 
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
@@ -1668,12 +1701,78 @@ void
 TablesyncWorkerMain(Datum main_arg)
 {
 	int			worker_slot = DatumGetInt32(main_arg);
+	bool 		done = false;
 
 	SetupApplyOrSyncWorker(worker_slot);
 
-	run_tablesync_worker();
+	/*
+	 * The loop where worker does its job. It loops until there is no relation
+	 * left to sync.
+	 */
+	for (;!done;)
+	{
+		List	   *rstates;
+		ListCell   *lc;
+
+		run_tablesync_worker();
+
+		if (IsTransactionState())
+			CommitTransactionCommand();
+
+		if (MyLogicalRepWorker->relsync_completed)
+		{
+			/*
+			 * This tablesync worker is 'done' unless another table that needs
+			 * syncing is found.
+			 */
+			done = true;
+
+			/* This transaction will be committed by finish_sync_worker. */
+			StartTransactionCommand();
+
+			/*
+			 * Check if there is any table whose relation state is still INIT.
+			 * If a table in INIT state is found, the worker will not be
+			 * finished, it will be reused instead.
+			 */
+			rstates = GetSubscriptionRelations(MySubscription->oid, true);
+
+			foreach(lc, rstates)
+			{
+				SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
+
+				if (rstate->state == SUBREL_STATE_SYNCDONE)
+					continue;
+
+				/*
+				 * Take exclusive lock to prevent any other sync worker from
+				 * picking the same table.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
+
+				/*
+				 * Pick the table for the next run if it is not already picked
+				 * up by another worker.
+				 */
+				if (!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+				{
+					/* Update worker state for the next table */
+					MyLogicalRepWorker->relid = rstate->relid;
+					MyLogicalRepWorker->relstate = rstate->state;
+					MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+					LWLockRelease(LogicalRepWorkerLock);
+
+					/* Found a table for next iteration */
+					finish_sync_worker(true);
+					done = false;
+					break;
+				}
+				LWLockRelease(LogicalRepWorkerLock);
+			}
+		}
+	}
 
-	finish_sync_worker();
+	finish_sync_worker(false);
 }
 
 /*
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 8ed6fa7acf..60d9c9a5a2 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3621,12 +3621,30 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				if (am_tablesync_worker())
+				{
+					/*
+					 * apply_dispatch() may have gone into apply_handle_commit()
+					 * which can call process_syncing_tables_for_sync.
+					 *
+					 * process_syncing_tables_for_sync decides whether the sync of
+					 * the current table is completed. If it is completed,
+					 * streaming must be already ended. So, we can break the loop.
+					 */
+					if (MyLogicalRepWorker->relsync_completed)
+					{
+						endofstream = true;
+						break;
+					}
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
 
 		/* confirm all writes so far */
-		send_feedback(last_received, false, false);
+		if (!MyLogicalRepWorker->relsync_completed)
+			send_feedback(last_received, false, false);
 
 		if (!in_remote_transaction && !in_streamed_transaction)
 		{
@@ -3640,6 +3658,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+
+			if (am_tablesync_worker())
+				/*
+				 * If relsync_completed is true, this means that the tablesync
+				 * worker is done with synchronization. Streaming has already been
+				 * ended by process_syncing_tables_for_sync. We should move to the
+				 * next table if needed, or exit.
+				 */
+				if (MyLogicalRepWorker->relsync_completed)
+					endofstream = true;
 		}
 
 		/* Cleanup the memory. */
@@ -3742,8 +3770,12 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * End streaming here for only apply workers. Ending streaming for
+	 * tablesync workers is deferred until the worker exits its main loop.
+	 */
+	if (!am_tablesync_worker())
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
 }
 
 /*
@@ -4617,9 +4649,10 @@ InitializeLogRepWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has started",
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" with relid %u has started",
 						MySubscription->name,
-						get_rel_name(MyLogicalRepWorker->relid))));
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
 	else
 		ereport(LOG,
 				(errmsg("logical replication apply worker for subscription \"%s\" has started",
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index fdbc1183f2..39b1721dee 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -56,6 +56,8 @@ typedef struct LogicalRepWorker
 	char		relstate;
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
+	bool		relsync_completed; /* has tablesync finished syncing
+									* the assigned table? */
 
 	/*
 	 * Used to create the changes and subxact files for the streaming
-- 
2.25.1

v23-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchapplication/octet-stream; name=v23-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchDownload
From 638b60024fce0e14dbbd6572369e66d32f9d5d8f Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Mon, 5 Jun 2023 15:04:41 +0300
Subject: [PATCH v23 1/3] Refactor to split Apply and Tablesync Workers

Both apply and tablesync workers were using ApplyWorkerMain() as entry
point. As the name implies, ApplyWorkerMain() should be considered as
the main function for apply workers. Tablesync worker's path was hidden
and does not have enough in common to share the same main function with
apply worker.

Also, most of the code shared by both worker types is already combined
in LogicalRepApplyLoop(). There is no need to combine the rest in
ApplyWorkerMain() anymore.

This patch introduces TablesyncWorkerMain() as a new entry point for
tablesync workers. This aims to increase code readability and help to
the upcoming reuse tablesync worker improvements.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/postmaster/bgworker.c             |   3 +
 .../replication/logical/applyparallelworker.c |   2 +-
 src/backend/replication/logical/launcher.c    |  32 +-
 src/backend/replication/logical/tablesync.c   |  94 ++++-
 src/backend/replication/logical/worker.c      | 380 ++++++++----------
 src/include/replication/logicalworker.h       |   1 +
 src/include/replication/worker_internal.h     |  15 +-
 7 files changed, 303 insertions(+), 224 deletions(-)

diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 5b4bd71694..505e38376c 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -131,6 +131,9 @@ static const struct
 	},
 	{
 		"ParallelApplyWorkerMain", ParallelApplyWorkerMain
+	},
+	{
+		"TablesyncWorkerMain", TablesyncWorkerMain
 	}
 };
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 6fb96148f4..1d4e83c4c1 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -942,7 +942,7 @@ ParallelApplyWorkerMain(Datum main_arg)
 	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
 		MyLogicalRepWorker->reply_time = 0;
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 542af7d863..e231fa7f95 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -459,24 +459,30 @@ retry:
 	snprintf(bgw.bgw_library_name, MAXPGPATH, "postgres");
 
 	if (is_parallel_apply_worker)
+	{
 		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ParallelApplyWorkerMain");
-	else
-		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
-
-	if (OidIsValid(relid))
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication worker for subscription %u sync %u", subid, relid);
-	else if (is_parallel_apply_worker)
+				 "logical replication parallel apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
+	}
+	else if (OidIsValid(relid))
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "TablesyncWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication parallel apply worker for subscription %u", subid);
+				 "logical replication tablesync worker for subscription %u sync %u",
+				 subid,
+				 relid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication tablesync worker");
+	}
 	else
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication apply worker for subscription %u", subid);
-
-	if (is_parallel_apply_worker)
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
-	else
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication worker");
+				 "logical replication apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication apply worker");
+	}
 
 	bgw.bgw_restart_time = BGW_NEVER_RESTART;
 	bgw.bgw_notify_pid = MyProcPid;
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 6d461654ab..ff859e0910 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -104,17 +104,21 @@
 #include "nodes/makefuncs.h"
 #include "parser/parse_relation.h"
 #include "pgstat.h"
+#include "postmaster/interrupt.h"
 #include "replication/logicallauncher.h"
 #include "replication/logicalrelation.h"
+#include "replication/logicalworker.h"
 #include "replication/walreceiver.h"
 #include "replication/worker_internal.h"
 #include "replication/slot.h"
 #include "replication/origin.h"
 #include "storage/ipc.h"
 #include "storage/lmgr.h"
+#include "tcop/tcopprot.h"
 #include "utils/acl.h"
 #include "utils/array.h"
 #include "utils/builtins.h"
+#include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
 #include "utils/rls.h"
@@ -1241,7 +1245,7 @@ ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
  *
  * The returned slot name is palloc'ed in current memory context.
  */
-char *
+static char *
 LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 {
 	char	   *slotname;
@@ -1584,6 +1588,94 @@ FetchTableStates(bool *started_tx)
 	return has_subrels;
 }
 
+/*
+ * Execute the initial sync with error handling. Disable the subscription,
+ * if it's required.
+ *
+ * Allocate the slot name in long-lived context on return. Note that we don't
+ * handle FATAL errors which are probably because of system resource error and
+ * are not repeatable.
+ */
+static void
+start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+{
+	char	   *syncslotname = NULL;
+
+	Assert(am_tablesync_worker());
+
+	PG_TRY();
+	{
+		/* Call initial sync. */
+		syncslotname = LogicalRepSyncTableStart(origin_startpos);
+	}
+	PG_CATCH();
+	{
+		if (MySubscription->disableonerr)
+			DisableSubscriptionAndExit();
+		else
+		{
+			/*
+			 * Report the worker failed during table synchronization. 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();
+
+	/* allocate slot name in long-lived context */
+	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+	pfree(syncslotname);
+}
+
+/*
+ * Runs the tablesync worker.
+ *
+ * It starts syncing tables. After a successful sync, sets streaming options
+ * and starts streaming to catchup.
+ */
+static void
+run_tablesync_worker()
+{
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *slotname = NULL;
+	WalRcvStreamOptions options;
+
+	start_table_sync(&origin_startpos, &slotname);
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   sizeof(originname));
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(&options, slotname, &origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+
+	/* Start applying changes to catchup. */
+	start_apply(origin_startpos);
+}
+
+/* Logical Replication Tablesync worker entry point */
+void
+TablesyncWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
+
+	SetupApplyOrSyncWorker(worker_slot);
+
+	run_tablesync_worker();
+
+	finish_sync_worker();
+}
+
 /*
  * If the subscription has no tables then return false.
  *
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 832b1cf764..8ed6fa7acf 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -396,8 +396,6 @@ static void stream_close_file(void);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
-static void DisableSubscriptionAndExit(void);
-
 static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
 static void apply_handle_insert_internal(ApplyExecutionData *edata,
 										 ResultRelInfo *relinfo,
@@ -4327,6 +4325,57 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+/*
+ * Sets streaming options including replication slot name and origin start
+ * position. Workers need these options for logical replication.
+ */
+void
+set_stream_options(WalRcvStreamOptions *options,
+				   char *slotname,
+				   XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4361,24 +4410,18 @@ TwoPhaseTransactionGid(Oid subid, TransactionId xid, char *gid, int szgid)
 }
 
 /*
- * Execute the initial sync with error handling. Disable the subscription,
- * if it's required.
+ * Common function to run the apply loop with error handling. Disable the
+ * subscription, if necessary.
  *
- * Allocate the slot name in long-lived context on return. Note that we don't
- * handle FATAL errors which are probably because of system resource error and
- * are not repeatable.
+ * Note that we don't handle FATAL errors which are probably because
+ * of system resource error and are not repeatable.
  */
-static void
-start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+void
+start_apply(XLogRecPtr origin_startpos)
 {
-	char	   *syncslotname = NULL;
-
-	Assert(am_tablesync_worker());
-
 	PG_TRY();
 	{
-		/* Call initial sync. */
-		syncslotname = LogicalRepSyncTableStart(origin_startpos);
+		LogicalRepApplyLoop(origin_startpos);
 	}
 	PG_CATCH();
 	{
@@ -4387,65 +4430,132 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 		else
 		{
 			/*
-			 * Report the worker failed during table synchronization. Abort
-			 * the current transaction so that the stats message is sent in an
+			 * Report the worker failed while applying changes. Abort the
+			 * current transaction so that the stats message is sent in an
 			 * idle state.
 			 */
 			AbortOutOfAnyTransaction();
-			pgstat_report_subscription_error(MySubscription->oid, false);
+			pgstat_report_subscription_error(MySubscription->oid, !am_tablesync_worker());
 
 			PG_RE_THROW();
 		}
 	}
 	PG_END_TRY();
-
-	/* allocate slot name in long-lived context */
-	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
-	pfree(syncslotname);
 }
 
 /*
- * Run the apply loop with error handling. Disable the subscription,
- * if necessary.
+ * Runs the leader apply worker.
  *
- * Note that we don't handle FATAL errors which are probably because
- * of system resource error and are not repeatable.
+ * It sets up replication origin, streaming options and then starts streaming.
  */
 static void
-start_apply(XLogRecPtr origin_startpos)
+run_apply_worker()
 {
-	PG_TRY();
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *slotname = NULL;
+	WalRcvStreamOptions options;
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+	bool		must_use_password;
+
+	slotname = MySubscription->slotname;
+
+	/*
+	 * This shouldn't happen if the subscription is enabled, but guard
+	 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+	 * crash if slot is NULL.)
+	 */
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	/* Setup replication origin tracking. */
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, sizeof(originname));
+	StartTransactionCommand();
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	origin_startpos = replorigin_session_get_progress(false);
+
+	/* Is the use of a password mandatory? */
+	must_use_password = MySubscription->passwordrequired &&
+		!superuser_arg(MySubscription->owner);
+
+	/* Note that the superuser_arg call can access the DB */
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											must_use_password,
+											MySubscription->name, &err);
+
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+	 * We don't really use the output identify_system for anything but it
+	 * does some initializations on the upstream so let's still call it.
+	 */
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(&options, slotname, &origin_startpos);
+
+	/*
+		* Even when the two_phase mode is requested by the user, it remains
+		* as the tri-state PENDING until all tablesyncs have reached READY
+		* state. Only then, can it become ENABLED.
+		*
+		* Note: If the subscription has no tables then leave the state as
+		* PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+		* work.
+		*/
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
 	{
-		LogicalRepApplyLoop(origin_startpos);
+		/* Start streaming with two_phase enabled */
+		options.proto.logical.twophase = true;
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
 	}
-	PG_CATCH();
+	else
 	{
-		if (MySubscription->disableonerr)
-			DisableSubscriptionAndExit();
-		else
-		{
-			/*
-			 * Report the worker failed while applying changes. Abort the
-			 * current transaction so that the stats message is sent in an
-			 * idle state.
-			 */
-			AbortOutOfAnyTransaction();
-			pgstat_report_subscription_error(MySubscription->oid, !am_tablesync_worker());
-
-			PG_RE_THROW();
-		}
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
 	}
-	PG_END_TRY();
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+
+	/* Run the main loop. */
+	start_apply(origin_startpos);
 }
 
 /*
- * Common initialization for leader apply worker and parallel apply worker.
+ * Common initialization for leader apply worker, parallel apply worker and
+ * tablesync worker.
  *
  * Initialize the database connection, in-memory subscription and necessary
  * config options.
  */
 void
-InitializeApplyWorker(void)
+InitializeLogRepWorker(void)
 {
 	MemoryContext oldctx;
 
@@ -4518,22 +4628,15 @@ InitializeApplyWorker(void)
 	CommitTransactionCommand();
 }
 
-/* Logical Replication Apply worker entry point */
+/* Common function to setup the leader apply or tablesync worker. */
 void
-ApplyWorkerMain(Datum main_arg)
+SetupApplyOrSyncWorker(int worker_slot)
 {
-	int			worker_slot = DatumGetInt32(main_arg);
-	char		originname[NAMEDATALEN];
-	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
-	char	   *myslotname = NULL;
-	WalRcvStreamOptions options;
-	int			server_version;
-
-	InitializingApplyWorker = true;
-
 	/* Attach to slot */
 	logicalrep_worker_attach(worker_slot);
 
+	Assert(am_tablesync_worker() || am_leader_apply_worker());
+
 	/* Setup signal handling */
 	pqsignal(SIGHUP, SignalHandlerForConfigReload);
 	pqsignal(SIGTERM, die);
@@ -4551,79 +4654,12 @@ ApplyWorkerMain(Datum main_arg)
 	/* Load the libpq-specific functions */
 	load_file("libpqwalreceiver", false);
 
-	InitializeApplyWorker();
-
-	InitializingApplyWorker = false;
+	InitializeLogRepWorker();
 
 	/* Connect to the origin and start the replication. */
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-		bool		must_use_password;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-
-		/* Is the use of a password mandatory? */
-		must_use_password = MySubscription->passwordrequired &&
-			!superuser_arg(MySubscription->owner);
-
-		/* Note that the superuser_arg call can access the DB */
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												must_use_password,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
 	 * the subscription relation state.
@@ -4631,91 +4667,21 @@ ApplyWorkerMain(Datum main_arg)
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
+}
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
-
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
-
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-
-	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
-	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
+/* Logical Replication Apply worker entry point */
+void
+ApplyWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
 
-	if (!am_tablesync_worker())
-	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
-		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+	InitializingApplyWorker = true;
 
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
-		}
-		else
-		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-		}
+	SetupApplyOrSyncWorker(worker_slot);
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-								 MySubscription->name,
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-								 "?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
+	InitializingApplyWorker = false;
 
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+	run_apply_worker();
 
 	proc_exit(0);
 }
@@ -4724,7 +4690,7 @@ ApplyWorkerMain(Datum main_arg)
  * After error recovery, disable the subscription in a new transaction
  * and exit cleanly.
  */
-static void
+void
 DisableSubscriptionAndExit(void)
 {
 	/*
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 39588da79f..bbd71d0b42 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -18,6 +18,7 @@ extern PGDLLIMPORT volatile sig_atomic_t ParallelApplyMessagePending;
 
 extern void ApplyWorkerMain(Datum main_arg);
 extern void ParallelApplyWorkerMain(Datum main_arg);
+extern void TablesyncWorkerMain(Datum main_arg);
 
 extern bool IsLogicalWorker(void);
 extern bool IsLogicalParallelApplyWorker(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 343e781896..fdbc1183f2 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -19,6 +19,7 @@
 #include "datatype/timestamp.h"
 #include "miscadmin.h"
 #include "replication/logicalrelation.h"
+#include "replication/walreceiver.h"
 #include "storage/buffile.h"
 #include "storage/fileset.h"
 #include "storage/lock.h"
@@ -243,7 +244,6 @@ extern int	logicalrep_sync_worker_count(Oid subid);
 
 extern void ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 											   char *originname, Size szoriginname);
-extern char *LogicalRepSyncTableStart(XLogRecPtr *origin_startpos);
 
 extern bool AllTablesyncsReady(void);
 extern void UpdateTwoPhaseState(Oid suboid, char new_state);
@@ -265,7 +265,17 @@ extern void maybe_reread_subscription(void);
 
 extern void stream_cleanup_files(Oid subid, TransactionId xid);
 
-extern void InitializeApplyWorker(void);
+extern void set_stream_options(WalRcvStreamOptions *options,
+							   char *slotname,
+							   XLogRecPtr *origin_startpos);
+
+extern void start_apply(XLogRecPtr origin_startpos);
+
+extern void InitializeLogRepWorker(void);
+
+extern void SetupApplyOrSyncWorker(int worker_slot);
+
+extern void DisableSubscriptionAndExit(void);
 
 extern void store_flush_position(XLogRecPtr remote_lsn, XLogRecPtr local_lsn);
 
@@ -305,6 +315,7 @@ extern void pa_decr_and_wait_stream_block(void);
 extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
 						   XLogRecPtr remote_lsn);
 
+
 #define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)
 
 static inline bool
-- 
2.25.1

v23-0003-Reuse-connection-when-tablesync-workers-change-t.patchapplication/octet-stream; name=v23-0003-Reuse-connection-when-tablesync-workers-change-t.patchDownload
From 84206c931d64ceb97985c83457040405a2b10e86 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:13:52 +0300
Subject: [PATCH v23 3/3] Reuse connection when tablesync workers change the
 target

Previously tablesync workers establish new connections when it changes the syncing
table, but this might have additional overhead. This patch allows the existing
connection to be reused.

As for the publisher node, this patch allows to reuse logical walsender processes
after the streaming is done once.
---
 src/backend/replication/logical/launcher.c  |  1 +
 src/backend/replication/logical/tablesync.c | 60 ++++++++++++++-------
 src/backend/replication/logical/worker.c    | 18 ++++---
 src/backend/replication/walsender.c         |  6 +++
 src/include/replication/worker_internal.h   |  3 ++
 5 files changed, 61 insertions(+), 27 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 25dd06b8af..657e446eaf 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
 	worker->relsync_completed = false;
+	worker->slotnum = slot;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 63b5bed88a..45e753b189 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -151,16 +151,6 @@ finish_sync_worker(bool reuse_worker)
 		pgstat_report_stat(true);
 	}
 
-	/*
-	 * Disconnect from the publisher otherwise reusing the sync worker can
-	 * error due to exceeding max_wal_senders.
-	 */
-	if (LogRepWorkerWalRcvConn != NULL)
-	{
-		walrcv_disconnect(LogRepWorkerWalRcvConn);
-		LogRepWorkerWalRcvConn = NULL;
-	}
-
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
@@ -1268,6 +1258,27 @@ ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
 			 relid, GetSystemIdentifier());
 }
 
+/*
+ * Determine the application_name for tablesync workers.
+ *
+ * Previously, the replication slot name was used as application_name. Since
+ * it's possible to reuse tablesync workers now, a tablesync worker can handle
+ * several different replication slots during its lifetime. Therefore, we
+ * cannot use the slot name as application_name anymore. Instead, the slot
+ * number of the tablesync worker is used as a part of the application_name.
+ *
+ * FIXME: if the tablesync worker starts to reuse the replication slot during
+ * synchronization, we should again use the replication slot name as
+ * application_name.
+ */
+static void
+ApplicationNameForTablesync(Oid suboid, int worker_slot,
+							char *application_name, Size szapp)
+{
+	snprintf(application_name, szapp, "pg_%u_sync_%i_" UINT64_FORMAT, suboid,
+			 worker_slot, GetSystemIdentifier());
+}
+
 /*
  * Start syncing the table in the sync worker.
  *
@@ -1329,15 +1340,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 									slotname,
 									NAMEDATALEN);
 
-	/*
-	 * Here we use the slot name instead of the subscription name as the
-	 * application_name, so that it is different from the leader apply worker,
-	 * so that synchronous replication can distinguish them.
-	 */
-	LogRepWorkerWalRcvConn =
-		walrcv_connect(MySubscription->conninfo, true,
-					   must_use_password,
-					   slotname, &err);
+	/* Connect to the publisher if haven't done so already. */
+	if (LogRepWorkerWalRcvConn == NULL)
+	{
+		char application_name[NAMEDATALEN];
+
+		/*
+		 * The application_name must differ from the subscription name (used by
+		 * the leader apply worker) because synchronous replication has to be
+		 * able to distinguish this worker from the leader apply worker.
+		 */
+		ApplicationNameForTablesync(MySubscription->oid,
+									MyLogicalRepWorker->slotnum,
+									application_name,
+									NAMEDATALEN);
+		LogRepWorkerWalRcvConn =
+			walrcv_connect(MySubscription->conninfo, true,
+						   must_use_password,
+						   application_name, &err);
+	}
+
 	if (LogRepWorkerWalRcvConn == NULL)
 		ereport(ERROR,
 				(errcode(ERRCODE_CONNECTION_FAILURE),
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 60d9c9a5a2..c0c13a4122 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3494,20 +3494,22 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	ErrorContextCallback errcallback;
 
 	/*
-	 * Init the ApplyMessageContext which we clean up after each replication
-	 * protocol message.
+	 * Init the ApplyMessageContext if needed. This context is cleaned up
+	 * after each replication protocol message.
 	 */
-	ApplyMessageContext = AllocSetContextCreate(ApplyContext,
-												"ApplyMessageContext",
-												ALLOCSET_DEFAULT_SIZES);
+	if (!ApplyMessageContext)
+		ApplyMessageContext = AllocSetContextCreate(ApplyContext,
+													"ApplyMessageContext",
+													ALLOCSET_DEFAULT_SIZES);
 
 	/*
 	 * This memory context is used for per-stream data when the streaming mode
 	 * is enabled. This context is reset on each stream stop.
 	 */
-	LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
-													"LogicalStreamingContext",
-													ALLOCSET_DEFAULT_SIZES);
+	if (!LogicalStreamingContext)
+		LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
+														"LogicalStreamingContext",
+														ALLOCSET_DEFAULT_SIZES);
 
 	/* mark as idle, before starting to loop */
 	pgstat_report_activity(STATE_IDLE, NULL);
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index d27ef2985d..1d2a87cdcd 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1808,6 +1808,12 @@ exec_replication_command(const char *cmd_string)
 		case T_CreateReplicationSlotCmd:
 			cmdtag = "CREATE_REPLICATION_SLOT";
 			set_ps_display(cmdtag);
+
+			/*
+			 * Reset flags because reusing tablesync workers can mean
+			 * this is the second time here.
+			 */
+			streamingDoneSending = streamingDoneReceiving = false;
 			CreateReplicationSlot((CreateReplicationSlotCmd *) cmd_node);
 			EndReplicationCommand(cmdtag);
 			break;
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 39b1721dee..b88ff6a646 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -39,6 +39,9 @@ typedef struct LogicalRepWorker
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
+	/* Slot number of this worker. */
+	int			slotnum;
+
 	/* Pointer to proc array. NULL if not running. */
 	PGPROC	   *proc;
 
-- 
2.25.1

#139Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Melih Mutlu (#138)
3 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi,

PFA an updated version with some of the earlier reviews addressed.
Forgot to include them in the previous email.

Thanks,
--
Melih Mutlu
Microsoft

Attachments:

v24-0003-Reuse-connection-when-tablesync-workers-change-t.patchapplication/octet-stream; name=v24-0003-Reuse-connection-when-tablesync-workers-change-t.patchDownload
From 6c449c8c44fc21398d23e68aeb58783ff3fe3550 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:13:52 +0300
Subject: [PATCH v24 3/3] Reuse connection when tablesync workers change the
 target

Previously tablesync workers establish new connections when it changes the syncing
table, but this might have additional overhead. This patch allows the existing
connection to be reused.

As for the publisher node, this patch allows to reuse logical walsender processes
after the streaming is done once.
---
 src/backend/replication/logical/launcher.c  |  1 +
 src/backend/replication/logical/tablesync.c | 60 ++++++++++++++-------
 src/backend/replication/logical/worker.c    | 18 ++++---
 src/backend/replication/walsender.c         |  6 +++
 src/include/replication/worker_internal.h   |  3 ++
 5 files changed, 61 insertions(+), 27 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 25dd06b8af..b19437f9d0 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
 	worker->relsync_completed = false;
+	worker->slot_number = slot;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 2ce3ae67c7..a49b67243e 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -151,16 +151,6 @@ finish_sync_worker(bool reuse_worker)
 		pgstat_report_stat(true);
 	}
 
-	/*
-	 * Disconnect from the publisher otherwise reusing the sync worker can
-	 * error due to exceeding max_wal_senders.
-	 */
-	if (LogRepWorkerWalRcvConn != NULL)
-	{
-		walrcv_disconnect(LogRepWorkerWalRcvConn);
-		LogRepWorkerWalRcvConn = NULL;
-	}
-
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
@@ -1264,6 +1254,27 @@ ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
 			 relid, GetSystemIdentifier());
 }
 
+/*
+ * Determine the application_name for tablesync workers.
+ *
+ * Previously, the replication slot name was used as application_name. Since
+ * it's possible to reuse tablesync workers now, a tablesync worker can handle
+ * several different replication slots during its lifetime. Therefore, we
+ * cannot use the slot name as application_name anymore. Instead, the slot
+ * number of the tablesync worker is used as a part of the application_name.
+ *
+ * XXX: if the tablesync worker starts to reuse the replication slot during
+ * synchronization, we should again use the replication slot name as
+ * application_name.
+ */
+static void
+ApplicationNameForTablesync(Oid suboid, int worker_slot,
+							char *application_name, Size szapp)
+{
+	snprintf(application_name, szapp, "pg_%u_sync_%i_" UINT64_FORMAT, suboid,
+			 worker_slot, GetSystemIdentifier());
+}
+
 /*
  * Start syncing the table in the sync worker.
  *
@@ -1325,15 +1336,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 									slotname,
 									NAMEDATALEN);
 
-	/*
-	 * Here we use the slot name instead of the subscription name as the
-	 * application_name, so that it is different from the leader apply worker,
-	 * so that synchronous replication can distinguish them.
-	 */
-	LogRepWorkerWalRcvConn =
-		walrcv_connect(MySubscription->conninfo, true,
-					   must_use_password,
-					   slotname, &err);
+	/* Connect to the publisher if haven't done so already. */
+	if (LogRepWorkerWalRcvConn == NULL)
+	{
+		char application_name[NAMEDATALEN];
+
+		/*
+		 * The application_name must differ from the subscription name (used by
+		 * the leader apply worker) because synchronous replication has to be
+		 * able to distinguish this worker from the leader apply worker.
+		 */
+		ApplicationNameForTablesync(MySubscription->oid,
+									MyLogicalRepWorker->slot_number,
+									application_name,
+									NAMEDATALEN);
+		LogRepWorkerWalRcvConn =
+			walrcv_connect(MySubscription->conninfo, true,
+						   must_use_password,
+						   application_name, &err);
+	}
+
 	if (LogRepWorkerWalRcvConn == NULL)
 		ereport(ERROR,
 				(errcode(ERRCODE_CONNECTION_FAILURE),
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 02c04ccafd..958b32b458 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3494,20 +3494,22 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	ErrorContextCallback errcallback;
 
 	/*
-	 * Init the ApplyMessageContext which we clean up after each replication
-	 * protocol message.
+	 * Init the ApplyMessageContext if needed. This context is cleaned up
+	 * after each replication protocol message.
 	 */
-	ApplyMessageContext = AllocSetContextCreate(ApplyContext,
-												"ApplyMessageContext",
-												ALLOCSET_DEFAULT_SIZES);
+	if (!ApplyMessageContext)
+		ApplyMessageContext = AllocSetContextCreate(ApplyContext,
+													"ApplyMessageContext",
+													ALLOCSET_DEFAULT_SIZES);
 
 	/*
 	 * This memory context is used for per-stream data when the streaming mode
 	 * is enabled. This context is reset on each stream stop.
 	 */
-	LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
-													"LogicalStreamingContext",
-													ALLOCSET_DEFAULT_SIZES);
+	if (!LogicalStreamingContext)
+		LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
+														"LogicalStreamingContext",
+														ALLOCSET_DEFAULT_SIZES);
 
 	/* mark as idle, before starting to loop */
 	pgstat_report_activity(STATE_IDLE, NULL);
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index d27ef2985d..1d2a87cdcd 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1808,6 +1808,12 @@ exec_replication_command(const char *cmd_string)
 		case T_CreateReplicationSlotCmd:
 			cmdtag = "CREATE_REPLICATION_SLOT";
 			set_ps_display(cmdtag);
+
+			/*
+			 * Reset flags because reusing tablesync workers can mean
+			 * this is the second time here.
+			 */
+			streamingDoneSending = streamingDoneReceiving = false;
 			CreateReplicationSlot((CreateReplicationSlotCmd *) cmd_node);
 			EndReplicationCommand(cmdtag);
 			break;
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 39b1721dee..94169e07b4 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -39,6 +39,9 @@ typedef struct LogicalRepWorker
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
+	/* Slot number of this worker. */
+	int			slot_number;
+
 	/* Pointer to proc array. NULL if not running. */
 	PGPROC	   *proc;
 
-- 
2.25.1

v24-0002-Reuse-Tablesync-Workers.patchapplication/octet-stream; name=v24-0002-Reuse-Tablesync-Workers.patchDownload
From 2c7bd1aeb55db8627c3cabeef150c4497a1c7f28 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:04:46 +0300
Subject: [PATCH v24 2/3] Reuse Tablesync Workers

Before this patch, tablesync workers were capable of syncing only one
table. For each table, a new sync worker was launched and that worker would
exit when done processing the table.

Now, tablesync workers are not limited to processing only one
table. When done, they can move to processing another table in
the same subscription.

If there is a table that needs to be synced, an available tablesync
worker picks up that table and syncs it. Each tablesync worker
continues to pick new tables to sync until there are no tables left
requiring synchronization. If there was no available worker to
process the table, then a new tablesync worker will be launched,
provided the number of tablesync workers for the subscription does not
exceed max_sync_workers_per_subscription.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/replication/logical/launcher.c  |   1 +
 src/backend/replication/logical/tablesync.c | 117 ++++++++++++++++++--
 src/backend/replication/logical/worker.c    |  45 +++++++-
 src/include/replication/worker_internal.h   |   2 +
 4 files changed, 149 insertions(+), 16 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e231fa7f95..25dd06b8af 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -440,6 +440,7 @@ retry:
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
+	worker->relsync_completed = false;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index ff859e0910..2ce3ae67c7 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -134,10 +134,12 @@ static StringInfo copybuf = NULL;
 
 /*
  * Exit routine for synchronization worker.
+ *
+ * If reuse_worker is false, at the conclusion of this function the worker
+ * process will exit.
  */
 static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+finish_sync_worker(bool reuse_worker)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -149,21 +151,38 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
+	/*
+	 * Disconnect from the publisher otherwise reusing the sync worker can
+	 * error due to exceeding max_wal_senders.
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		walrcv_disconnect(LogRepWorkerWalRcvConn);
+		LogRepWorkerWalRcvConn = NULL;
+	}
+
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
 	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
+	if (reuse_worker)
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\" will be reused to sync table \"%s\" with relid %u.",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
+	else
+		ereport(LOG,
+				(errmsg("logical replication table synchronization worker for subscription \"%s\" has finished",
+						MySubscription->name)));
 	CommitTransactionCommand();
 
 	/* Find the leader apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
 
 	/* Stop gracefully */
-	proc_exit(0);
+	if (!reuse_worker)
+		proc_exit(0);
 }
 
 /*
@@ -383,7 +402,15 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 */
 		replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
+		/* Sync worker has completed synchronization of the current table. */
+		MyLogicalRepWorker->relsync_completed = true;
+
+		ereport(LOG,
+				(errmsg("logical replication table synchronization for subscription \"%s\", relation \"%s\" with relid %u has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
+		CommitTransactionCommand();
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -1288,7 +1315,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		case SUBREL_STATE_SYNCDONE:
 		case SUBREL_STATE_READY:
 		case SUBREL_STATE_UNKNOWN:
-			finish_sync_worker();	/* doesn't return */
+			finish_sync_worker(false);	/* doesn't return */
 	}
 
 	/* Calculate the name of the tablesync slot. */
@@ -1646,6 +1673,8 @@ run_tablesync_worker()
 	char	   *slotname = NULL;
 	WalRcvStreamOptions options;
 
+	MyLogicalRepWorker->relsync_completed = false;
+
 	start_table_sync(&origin_startpos, &slotname);
 
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
@@ -1668,12 +1697,78 @@ void
 TablesyncWorkerMain(Datum main_arg)
 {
 	int			worker_slot = DatumGetInt32(main_arg);
+	bool 		done = false;
 
 	SetupApplyOrSyncWorker(worker_slot);
 
-	run_tablesync_worker();
+	/*
+	 * The loop where worker does its job. It loops until there is no relation
+	 * left to sync.
+	 */
+	for (;!done;)
+	{
+		run_tablesync_worker();
+
+		if (IsTransactionState())
+			CommitTransactionCommand();
+
+		if (MyLogicalRepWorker->relsync_completed)
+		{
+			List	   *rstates;
+			ListCell   *lc;
+
+			/*
+			 * This tablesync worker is 'done' unless another table that needs
+			 * syncing is found.
+			 */
+			done = true;
+
+			/* This transaction will be committed by finish_sync_worker. */
+			StartTransactionCommand();
+
+			/*
+			 * Check if there is any table whose relation state is still INIT.
+			 * If a table in INIT state is found, the worker will not be
+			 * finished, it will be reused instead.
+			 */
+			rstates = GetSubscriptionRelations(MySubscription->oid, true);
+
+			foreach(lc, rstates)
+			{
+				SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
+
+				if (rstate->state == SUBREL_STATE_SYNCDONE)
+					continue;
+
+				/*
+				 * Take exclusive lock to prevent any other sync worker from
+				 * picking the same table.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
+
+				/*
+				 * Pick the table for the next run if it is not already picked
+				 * up by another worker.
+				 */
+				if (!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+				{
+					/* Update worker state for the next table */
+					MyLogicalRepWorker->relid = rstate->relid;
+					MyLogicalRepWorker->relstate = rstate->state;
+					MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+					LWLockRelease(LogicalRepWorkerLock);
+
+					/* Found a table for next iteration */
+					finish_sync_worker(true);
+					done = false;
+					break;
+				}
+				LWLockRelease(LogicalRepWorkerLock);
+			}
+		}
+	}
 
-	finish_sync_worker();
+	finish_sync_worker(false);
 }
 
 /*
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 8ed6fa7acf..02c04ccafd 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3621,12 +3621,30 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				if (am_tablesync_worker())
+				{
+					/*
+					 * apply_dispatch() may have gone into apply_handle_commit()
+					 * which can call process_syncing_tables_for_sync.
+					 *
+					 * process_syncing_tables_for_sync decides whether the sync of
+					 * the current table is completed. If it is completed,
+					 * streaming must be already ended. So, we can break the loop.
+					 */
+					if (MyLogicalRepWorker->relsync_completed)
+					{
+						endofstream = true;
+						break;
+					}
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
 
 		/* confirm all writes so far */
-		send_feedback(last_received, false, false);
+		if (!MyLogicalRepWorker->relsync_completed)
+			send_feedback(last_received, false, false);
 
 		if (!in_remote_transaction && !in_streamed_transaction)
 		{
@@ -3640,6 +3658,18 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+
+			if (am_tablesync_worker())
+			{
+				/*
+				 * If relsync_completed is true, this means that the tablesync
+				 * worker is done with synchronization. Streaming has already been
+				 * ended by process_syncing_tables_for_sync. We should move to the
+				 * next table if needed, or exit.
+				 */
+				if (MyLogicalRepWorker->relsync_completed)
+					endofstream = true;
+			}
 		}
 
 		/* Cleanup the memory. */
@@ -3742,8 +3772,12 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * End streaming here for only apply workers. Ending streaming for
+	 * tablesync workers is deferred until the worker exits its main loop.
+	 */
+	if (!am_tablesync_worker())
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
 }
 
 /*
@@ -4617,9 +4651,10 @@ InitializeLogRepWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has started",
+				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" with relid %u has started",
 						MySubscription->name,
-						get_rel_name(MyLogicalRepWorker->relid))));
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid)));
 	else
 		ereport(LOG,
 				(errmsg("logical replication apply worker for subscription \"%s\" has started",
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index fdbc1183f2..39b1721dee 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -56,6 +56,8 @@ typedef struct LogicalRepWorker
 	char		relstate;
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
+	bool		relsync_completed; /* has tablesync finished syncing
+									* the assigned table? */
 
 	/*
 	 * Used to create the changes and subxact files for the streaming
-- 
2.25.1

v24-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchapplication/octet-stream; name=v24-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchDownload
From 638b60024fce0e14dbbd6572369e66d32f9d5d8f Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Mon, 5 Jun 2023 15:04:41 +0300
Subject: [PATCH v24 1/3] Refactor to split Apply and Tablesync Workers

Both apply and tablesync workers were using ApplyWorkerMain() as entry
point. As the name implies, ApplyWorkerMain() should be considered as
the main function for apply workers. Tablesync worker's path was hidden
and does not have enough in common to share the same main function with
apply worker.

Also, most of the code shared by both worker types is already combined
in LogicalRepApplyLoop(). There is no need to combine the rest in
ApplyWorkerMain() anymore.

This patch introduces TablesyncWorkerMain() as a new entry point for
tablesync workers. This aims to increase code readability and help to
the upcoming reuse tablesync worker improvements.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/postmaster/bgworker.c             |   3 +
 .../replication/logical/applyparallelworker.c |   2 +-
 src/backend/replication/logical/launcher.c    |  32 +-
 src/backend/replication/logical/tablesync.c   |  94 ++++-
 src/backend/replication/logical/worker.c      | 380 ++++++++----------
 src/include/replication/logicalworker.h       |   1 +
 src/include/replication/worker_internal.h     |  15 +-
 7 files changed, 303 insertions(+), 224 deletions(-)

diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 5b4bd71694..505e38376c 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -131,6 +131,9 @@ static const struct
 	},
 	{
 		"ParallelApplyWorkerMain", ParallelApplyWorkerMain
+	},
+	{
+		"TablesyncWorkerMain", TablesyncWorkerMain
 	}
 };
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 6fb96148f4..1d4e83c4c1 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -942,7 +942,7 @@ ParallelApplyWorkerMain(Datum main_arg)
 	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
 		MyLogicalRepWorker->reply_time = 0;
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 542af7d863..e231fa7f95 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -459,24 +459,30 @@ retry:
 	snprintf(bgw.bgw_library_name, MAXPGPATH, "postgres");
 
 	if (is_parallel_apply_worker)
+	{
 		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ParallelApplyWorkerMain");
-	else
-		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
-
-	if (OidIsValid(relid))
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication worker for subscription %u sync %u", subid, relid);
-	else if (is_parallel_apply_worker)
+				 "logical replication parallel apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
+	}
+	else if (OidIsValid(relid))
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "TablesyncWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication parallel apply worker for subscription %u", subid);
+				 "logical replication tablesync worker for subscription %u sync %u",
+				 subid,
+				 relid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication tablesync worker");
+	}
 	else
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication apply worker for subscription %u", subid);
-
-	if (is_parallel_apply_worker)
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
-	else
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication worker");
+				 "logical replication apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication apply worker");
+	}
 
 	bgw.bgw_restart_time = BGW_NEVER_RESTART;
 	bgw.bgw_notify_pid = MyProcPid;
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 6d461654ab..ff859e0910 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -104,17 +104,21 @@
 #include "nodes/makefuncs.h"
 #include "parser/parse_relation.h"
 #include "pgstat.h"
+#include "postmaster/interrupt.h"
 #include "replication/logicallauncher.h"
 #include "replication/logicalrelation.h"
+#include "replication/logicalworker.h"
 #include "replication/walreceiver.h"
 #include "replication/worker_internal.h"
 #include "replication/slot.h"
 #include "replication/origin.h"
 #include "storage/ipc.h"
 #include "storage/lmgr.h"
+#include "tcop/tcopprot.h"
 #include "utils/acl.h"
 #include "utils/array.h"
 #include "utils/builtins.h"
+#include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
 #include "utils/rls.h"
@@ -1241,7 +1245,7 @@ ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
  *
  * The returned slot name is palloc'ed in current memory context.
  */
-char *
+static char *
 LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 {
 	char	   *slotname;
@@ -1584,6 +1588,94 @@ FetchTableStates(bool *started_tx)
 	return has_subrels;
 }
 
+/*
+ * Execute the initial sync with error handling. Disable the subscription,
+ * if it's required.
+ *
+ * Allocate the slot name in long-lived context on return. Note that we don't
+ * handle FATAL errors which are probably because of system resource error and
+ * are not repeatable.
+ */
+static void
+start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+{
+	char	   *syncslotname = NULL;
+
+	Assert(am_tablesync_worker());
+
+	PG_TRY();
+	{
+		/* Call initial sync. */
+		syncslotname = LogicalRepSyncTableStart(origin_startpos);
+	}
+	PG_CATCH();
+	{
+		if (MySubscription->disableonerr)
+			DisableSubscriptionAndExit();
+		else
+		{
+			/*
+			 * Report the worker failed during table synchronization. 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();
+
+	/* allocate slot name in long-lived context */
+	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
+	pfree(syncslotname);
+}
+
+/*
+ * Runs the tablesync worker.
+ *
+ * It starts syncing tables. After a successful sync, sets streaming options
+ * and starts streaming to catchup.
+ */
+static void
+run_tablesync_worker()
+{
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *slotname = NULL;
+	WalRcvStreamOptions options;
+
+	start_table_sync(&origin_startpos, &slotname);
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   sizeof(originname));
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(&options, slotname, &origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+
+	/* Start applying changes to catchup. */
+	start_apply(origin_startpos);
+}
+
+/* Logical Replication Tablesync worker entry point */
+void
+TablesyncWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
+
+	SetupApplyOrSyncWorker(worker_slot);
+
+	run_tablesync_worker();
+
+	finish_sync_worker();
+}
+
 /*
  * If the subscription has no tables then return false.
  *
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 832b1cf764..8ed6fa7acf 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -396,8 +396,6 @@ static void stream_close_file(void);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
-static void DisableSubscriptionAndExit(void);
-
 static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
 static void apply_handle_insert_internal(ApplyExecutionData *edata,
 										 ResultRelInfo *relinfo,
@@ -4327,6 +4325,57 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+/*
+ * Sets streaming options including replication slot name and origin start
+ * position. Workers need these options for logical replication.
+ */
+void
+set_stream_options(WalRcvStreamOptions *options,
+				   char *slotname,
+				   XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4361,24 +4410,18 @@ TwoPhaseTransactionGid(Oid subid, TransactionId xid, char *gid, int szgid)
 }
 
 /*
- * Execute the initial sync with error handling. Disable the subscription,
- * if it's required.
+ * Common function to run the apply loop with error handling. Disable the
+ * subscription, if necessary.
  *
- * Allocate the slot name in long-lived context on return. Note that we don't
- * handle FATAL errors which are probably because of system resource error and
- * are not repeatable.
+ * Note that we don't handle FATAL errors which are probably because
+ * of system resource error and are not repeatable.
  */
-static void
-start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+void
+start_apply(XLogRecPtr origin_startpos)
 {
-	char	   *syncslotname = NULL;
-
-	Assert(am_tablesync_worker());
-
 	PG_TRY();
 	{
-		/* Call initial sync. */
-		syncslotname = LogicalRepSyncTableStart(origin_startpos);
+		LogicalRepApplyLoop(origin_startpos);
 	}
 	PG_CATCH();
 	{
@@ -4387,65 +4430,132 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 		else
 		{
 			/*
-			 * Report the worker failed during table synchronization. Abort
-			 * the current transaction so that the stats message is sent in an
+			 * Report the worker failed while applying changes. Abort the
+			 * current transaction so that the stats message is sent in an
 			 * idle state.
 			 */
 			AbortOutOfAnyTransaction();
-			pgstat_report_subscription_error(MySubscription->oid, false);
+			pgstat_report_subscription_error(MySubscription->oid, !am_tablesync_worker());
 
 			PG_RE_THROW();
 		}
 	}
 	PG_END_TRY();
-
-	/* allocate slot name in long-lived context */
-	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
-	pfree(syncslotname);
 }
 
 /*
- * Run the apply loop with error handling. Disable the subscription,
- * if necessary.
+ * Runs the leader apply worker.
  *
- * Note that we don't handle FATAL errors which are probably because
- * of system resource error and are not repeatable.
+ * It sets up replication origin, streaming options and then starts streaming.
  */
 static void
-start_apply(XLogRecPtr origin_startpos)
+run_apply_worker()
 {
-	PG_TRY();
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *slotname = NULL;
+	WalRcvStreamOptions options;
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+	bool		must_use_password;
+
+	slotname = MySubscription->slotname;
+
+	/*
+	 * This shouldn't happen if the subscription is enabled, but guard
+	 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
+	 * crash if slot is NULL.)
+	 */
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("subscription has no replication slot set")));
+
+	/* Setup replication origin tracking. */
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, sizeof(originname));
+	StartTransactionCommand();
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	origin_startpos = replorigin_session_get_progress(false);
+
+	/* Is the use of a password mandatory? */
+	must_use_password = MySubscription->passwordrequired &&
+		!superuser_arg(MySubscription->owner);
+
+	/* Note that the superuser_arg call can access the DB */
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											must_use_password,
+											MySubscription->name, &err);
+
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+					errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+	 * We don't really use the output identify_system for anything but it
+	 * does some initializations on the upstream so let's still call it.
+	 */
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(&options, slotname, &origin_startpos);
+
+	/*
+		* Even when the two_phase mode is requested by the user, it remains
+		* as the tri-state PENDING until all tablesyncs have reached READY
+		* state. Only then, can it become ENABLED.
+		*
+		* Note: If the subscription has no tables then leave the state as
+		* PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+		* work.
+		*/
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
 	{
-		LogicalRepApplyLoop(origin_startpos);
+		/* Start streaming with two_phase enabled */
+		options.proto.logical.twophase = true;
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
 	}
-	PG_CATCH();
+	else
 	{
-		if (MySubscription->disableonerr)
-			DisableSubscriptionAndExit();
-		else
-		{
-			/*
-			 * Report the worker failed while applying changes. Abort the
-			 * current transaction so that the stats message is sent in an
-			 * idle state.
-			 */
-			AbortOutOfAnyTransaction();
-			pgstat_report_subscription_error(MySubscription->oid, !am_tablesync_worker());
-
-			PG_RE_THROW();
-		}
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
 	}
-	PG_END_TRY();
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+
+	/* Run the main loop. */
+	start_apply(origin_startpos);
 }
 
 /*
- * Common initialization for leader apply worker and parallel apply worker.
+ * Common initialization for leader apply worker, parallel apply worker and
+ * tablesync worker.
  *
  * Initialize the database connection, in-memory subscription and necessary
  * config options.
  */
 void
-InitializeApplyWorker(void)
+InitializeLogRepWorker(void)
 {
 	MemoryContext oldctx;
 
@@ -4518,22 +4628,15 @@ InitializeApplyWorker(void)
 	CommitTransactionCommand();
 }
 
-/* Logical Replication Apply worker entry point */
+/* Common function to setup the leader apply or tablesync worker. */
 void
-ApplyWorkerMain(Datum main_arg)
+SetupApplyOrSyncWorker(int worker_slot)
 {
-	int			worker_slot = DatumGetInt32(main_arg);
-	char		originname[NAMEDATALEN];
-	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
-	char	   *myslotname = NULL;
-	WalRcvStreamOptions options;
-	int			server_version;
-
-	InitializingApplyWorker = true;
-
 	/* Attach to slot */
 	logicalrep_worker_attach(worker_slot);
 
+	Assert(am_tablesync_worker() || am_leader_apply_worker());
+
 	/* Setup signal handling */
 	pqsignal(SIGHUP, SignalHandlerForConfigReload);
 	pqsignal(SIGTERM, die);
@@ -4551,79 +4654,12 @@ ApplyWorkerMain(Datum main_arg)
 	/* Load the libpq-specific functions */
 	load_file("libpqwalreceiver", false);
 
-	InitializeApplyWorker();
-
-	InitializingApplyWorker = false;
+	InitializeLogRepWorker();
 
 	/* Connect to the origin and start the replication. */
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-		bool		must_use_password;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-
-		/* Is the use of a password mandatory? */
-		must_use_password = MySubscription->passwordrequired &&
-			!superuser_arg(MySubscription->owner);
-
-		/* Note that the superuser_arg call can access the DB */
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												must_use_password,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
 	 * the subscription relation state.
@@ -4631,91 +4667,21 @@ ApplyWorkerMain(Datum main_arg)
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
+}
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
-
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
-
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-
-	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
-	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
+/* Logical Replication Apply worker entry point */
+void
+ApplyWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
 
-	if (!am_tablesync_worker())
-	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
-		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+	InitializingApplyWorker = true;
 
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
-		}
-		else
-		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-		}
+	SetupApplyOrSyncWorker(worker_slot);
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-								 MySubscription->name,
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-								 "?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
+	InitializingApplyWorker = false;
 
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+	run_apply_worker();
 
 	proc_exit(0);
 }
@@ -4724,7 +4690,7 @@ ApplyWorkerMain(Datum main_arg)
  * After error recovery, disable the subscription in a new transaction
  * and exit cleanly.
  */
-static void
+void
 DisableSubscriptionAndExit(void)
 {
 	/*
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 39588da79f..bbd71d0b42 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -18,6 +18,7 @@ extern PGDLLIMPORT volatile sig_atomic_t ParallelApplyMessagePending;
 
 extern void ApplyWorkerMain(Datum main_arg);
 extern void ParallelApplyWorkerMain(Datum main_arg);
+extern void TablesyncWorkerMain(Datum main_arg);
 
 extern bool IsLogicalWorker(void);
 extern bool IsLogicalParallelApplyWorker(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 343e781896..fdbc1183f2 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -19,6 +19,7 @@
 #include "datatype/timestamp.h"
 #include "miscadmin.h"
 #include "replication/logicalrelation.h"
+#include "replication/walreceiver.h"
 #include "storage/buffile.h"
 #include "storage/fileset.h"
 #include "storage/lock.h"
@@ -243,7 +244,6 @@ extern int	logicalrep_sync_worker_count(Oid subid);
 
 extern void ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 											   char *originname, Size szoriginname);
-extern char *LogicalRepSyncTableStart(XLogRecPtr *origin_startpos);
 
 extern bool AllTablesyncsReady(void);
 extern void UpdateTwoPhaseState(Oid suboid, char new_state);
@@ -265,7 +265,17 @@ extern void maybe_reread_subscription(void);
 
 extern void stream_cleanup_files(Oid subid, TransactionId xid);
 
-extern void InitializeApplyWorker(void);
+extern void set_stream_options(WalRcvStreamOptions *options,
+							   char *slotname,
+							   XLogRecPtr *origin_startpos);
+
+extern void start_apply(XLogRecPtr origin_startpos);
+
+extern void InitializeLogRepWorker(void);
+
+extern void SetupApplyOrSyncWorker(int worker_slot);
+
+extern void DisableSubscriptionAndExit(void);
 
 extern void store_flush_position(XLogRecPtr remote_lsn, XLogRecPtr local_lsn);
 
@@ -305,6 +315,7 @@ extern void pa_decr_and_wait_stream_block(void);
 extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
 						   XLogRecPtr remote_lsn);
 
+
 #define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)
 
 static inline bool
-- 
2.25.1

#140Amit Kapila
amit.kapila16@gmail.com
In reply to: Melih Mutlu (#139)
1 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Wed, Aug 2, 2023 at 4:09 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

PFA an updated version with some of the earlier reviews addressed.
Forgot to include them in the previous email.

It is always better to explicitly tell which reviews are addressed but
anyway, I have done some minor cleanup in the 0001 patch including
removing includes which didn't seem necessary, modified a few
comments, and ran pgindent. I also thought of modifying some variable
names based on suggestions by Peter Smith in an email [1]/messages/by-id/CAHut+Ps3Du9JFmhecWY8+VFD11VLOkSmB36t_xWHHQJNMpdA-A@mail.gmail.com but didn't
find many of them any better than the current ones so modified just a
few of those. If you guys are okay with this then let's commit it and
then we can focus more on the remaining patches.

[1]: /messages/by-id/CAHut+Ps3Du9JFmhecWY8+VFD11VLOkSmB36t_xWHHQJNMpdA-A@mail.gmail.com

--
With Regards,
Amit Kapila.

Attachments:

v25-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchapplication/octet-stream; name=v25-0001-Refactor-to-split-Apply-and-Tablesync-Workers.patchDownload
From 7dbdf5e21c057cfdfa6ee85ff1efe35341f0badd Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Mon, 5 Jun 2023 15:04:41 +0300
Subject: [PATCH v25] Refactor to split Apply and Tablesync Workers

Both apply and tablesync workers were using ApplyWorkerMain() as entry
point. As the name implies, ApplyWorkerMain() should be considered as
the main function for apply workers. Tablesync worker's path was hidden
and does not have enough in common to share the same main function with
apply worker.

Also, most of the code shared by both worker types is already combined
in LogicalRepApplyLoop(). There is no need to combine the rest in
ApplyWorkerMain() anymore.

This patch introduces TablesyncWorkerMain() as a new entry point for
tablesync workers. This aims to increase code readability and help to
the upcoming reuse tablesync worker improvements.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/postmaster/bgworker.c             |   3 +
 .../replication/logical/applyparallelworker.c |   2 +-
 src/backend/replication/logical/launcher.c    |  32 +-
 src/backend/replication/logical/tablesync.c   |  91 ++++-
 src/backend/replication/logical/worker.c      | 380 ++++++++----------
 src/include/replication/logicalworker.h       |   1 +
 src/include/replication/worker_internal.h     |  14 +-
 7 files changed, 299 insertions(+), 224 deletions(-)

diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 5b4bd71694..505e38376c 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -131,6 +131,9 @@ static const struct
 	},
 	{
 		"ParallelApplyWorkerMain", ParallelApplyWorkerMain
+	},
+	{
+		"TablesyncWorkerMain", TablesyncWorkerMain
 	}
 };
 
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 6fb96148f4..1d4e83c4c1 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -942,7 +942,7 @@ ParallelApplyWorkerMain(Datum main_arg)
 	MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
 		MyLogicalRepWorker->reply_time = 0;
 
-	InitializeApplyWorker();
+	InitializeLogRepWorker();
 
 	InitializingApplyWorker = false;
 
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 542af7d863..e231fa7f95 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -459,24 +459,30 @@ retry:
 	snprintf(bgw.bgw_library_name, MAXPGPATH, "postgres");
 
 	if (is_parallel_apply_worker)
+	{
 		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ParallelApplyWorkerMain");
-	else
-		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
-
-	if (OidIsValid(relid))
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication worker for subscription %u sync %u", subid, relid);
-	else if (is_parallel_apply_worker)
+				 "logical replication parallel apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
+	}
+	else if (OidIsValid(relid))
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "TablesyncWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication parallel apply worker for subscription %u", subid);
+				 "logical replication tablesync worker for subscription %u sync %u",
+				 subid,
+				 relid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication tablesync worker");
+	}
 	else
+	{
+		snprintf(bgw.bgw_function_name, BGW_MAXLEN, "ApplyWorkerMain");
 		snprintf(bgw.bgw_name, BGW_MAXLEN,
-				 "logical replication apply worker for subscription %u", subid);
-
-	if (is_parallel_apply_worker)
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication parallel worker");
-	else
-		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication worker");
+				 "logical replication apply worker for subscription %u",
+				 subid);
+		snprintf(bgw.bgw_type, BGW_MAXLEN, "logical replication apply worker");
+	}
 
 	bgw.bgw_restart_time = BGW_NEVER_RESTART;
 	bgw.bgw_notify_pid = MyProcPid;
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 6d461654ab..651a775065 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -106,6 +106,7 @@
 #include "pgstat.h"
 #include "replication/logicallauncher.h"
 #include "replication/logicalrelation.h"
+#include "replication/logicalworker.h"
 #include "replication/walreceiver.h"
 #include "replication/worker_internal.h"
 #include "replication/slot.h"
@@ -1241,7 +1242,7 @@ ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
  *
  * The returned slot name is palloc'ed in current memory context.
  */
-char *
+static char *
 LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 {
 	char	   *slotname;
@@ -1584,6 +1585,94 @@ FetchTableStates(bool *started_tx)
 	return has_subrels;
 }
 
+/*
+ * Execute the initial sync with error handling. Disable the subscription,
+ * if it's required.
+ *
+ * Allocate the slot name in long-lived context on return. Note that we don't
+ * handle FATAL errors which are probably because of system resource error and
+ * are not repeatable.
+ */
+static void
+start_table_sync(XLogRecPtr *origin_startpos, char **slotname)
+{
+	char	   *sync_slotname = NULL;
+
+	Assert(am_tablesync_worker());
+
+	PG_TRY();
+	{
+		/* Call initial sync. */
+		sync_slotname = LogicalRepSyncTableStart(origin_startpos);
+	}
+	PG_CATCH();
+	{
+		if (MySubscription->disableonerr)
+			DisableSubscriptionAndExit();
+		else
+		{
+			/*
+			 * Report the worker failed during table synchronization. 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();
+
+	/* allocate slot name in long-lived context */
+	*slotname = MemoryContextStrdup(ApplyContext, sync_slotname);
+	pfree(sync_slotname);
+}
+
+/*
+ * Runs the tablesync worker.
+ *
+ * It starts syncing tables. After a successful sync, sets streaming options
+ * and starts streaming to catchup with apply worker.
+ */
+static void
+run_tablesync_worker()
+{
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *slotname = NULL;
+	WalRcvStreamOptions options;
+
+	start_table_sync(&origin_startpos, &slotname);
+
+	ReplicationOriginNameForLogicalRep(MySubscription->oid,
+									   MyLogicalRepWorker->relid,
+									   originname,
+									   sizeof(originname));
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(&options, slotname, &origin_startpos);
+
+	walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+
+	/* Apply the changes till we catchup with the apply worker. */
+	start_apply(origin_startpos);
+}
+
+/* Logical Replication Tablesync worker entry point */
+void
+TablesyncWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
+
+	SetupApplyOrSyncWorker(worker_slot);
+
+	run_tablesync_worker();
+
+	finish_sync_worker();
+}
+
 /*
  * If the subscription has no tables then return false.
  *
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 832b1cf764..a9f5fa7dfc 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -396,8 +396,6 @@ static void stream_close_file(void);
 
 static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
 
-static void DisableSubscriptionAndExit(void);
-
 static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
 static void apply_handle_insert_internal(ApplyExecutionData *edata,
 										 ResultRelInfo *relinfo,
@@ -4327,6 +4325,57 @@ stream_open_and_write_change(TransactionId xid, char action, StringInfo s)
 	stream_stop_internal(xid);
 }
 
+/*
+ * Sets streaming options including replication slot name and origin start
+ * position. Workers need these options for logical replication.
+ */
+void
+set_stream_options(WalRcvStreamOptions *options,
+				   char *slotname,
+				   XLogRecPtr *origin_startpos)
+{
+	int			server_version;
+
+	options->logical = true;
+	options->startpoint = *origin_startpos;
+	options->slotname = slotname;
+
+	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
+	options->proto.logical.proto_version =
+		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
+		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
+		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
+		LOGICALREP_PROTO_VERSION_NUM;
+
+	options->proto.logical.publication_names = MySubscription->publications;
+	options->proto.logical.binary = MySubscription->binary;
+
+	/*
+	 * Assign the appropriate option value for streaming option according to
+	 * the 'streaming' mode and the publisher's ability to support that mode.
+	 */
+	if (server_version >= 160000 &&
+		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
+	{
+		options->proto.logical.streaming_str = "parallel";
+		MyLogicalRepWorker->parallel_apply = true;
+	}
+	else if (server_version >= 140000 &&
+			 MySubscription->stream != LOGICALREP_STREAM_OFF)
+	{
+		options->proto.logical.streaming_str = "on";
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+	else
+	{
+		options->proto.logical.streaming_str = NULL;
+		MyLogicalRepWorker->parallel_apply = false;
+	}
+
+	options->proto.logical.twophase = false;
+	options->proto.logical.origin = pstrdup(MySubscription->origin);
+}
+
 /*
  * Cleanup the memory for subxacts and reset the related variables.
  */
@@ -4361,24 +4410,18 @@ TwoPhaseTransactionGid(Oid subid, TransactionId xid, char *gid, int szgid)
 }
 
 /*
- * Execute the initial sync with error handling. Disable the subscription,
- * if it's required.
+ * Common function to run the apply loop with error handling. Disable the
+ * subscription, if necessary.
  *
- * Allocate the slot name in long-lived context on return. Note that we don't
- * handle FATAL errors which are probably because of system resource error and
- * are not repeatable.
+ * Note that we don't handle FATAL errors which are probably because
+ * of system resource error and are not repeatable.
  */
-static void
-start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
+void
+start_apply(XLogRecPtr origin_startpos)
 {
-	char	   *syncslotname = NULL;
-
-	Assert(am_tablesync_worker());
-
 	PG_TRY();
 	{
-		/* Call initial sync. */
-		syncslotname = LogicalRepSyncTableStart(origin_startpos);
+		LogicalRepApplyLoop(origin_startpos);
 	}
 	PG_CATCH();
 	{
@@ -4387,65 +4430,132 @@ start_table_sync(XLogRecPtr *origin_startpos, char **myslotname)
 		else
 		{
 			/*
-			 * Report the worker failed during table synchronization. Abort
-			 * the current transaction so that the stats message is sent in an
+			 * Report the worker failed while applying changes. Abort the
+			 * current transaction so that the stats message is sent in an
 			 * idle state.
 			 */
 			AbortOutOfAnyTransaction();
-			pgstat_report_subscription_error(MySubscription->oid, false);
+			pgstat_report_subscription_error(MySubscription->oid, !am_tablesync_worker());
 
 			PG_RE_THROW();
 		}
 	}
 	PG_END_TRY();
-
-	/* allocate slot name in long-lived context */
-	*myslotname = MemoryContextStrdup(ApplyContext, syncslotname);
-	pfree(syncslotname);
 }
 
 /*
- * Run the apply loop with error handling. Disable the subscription,
- * if necessary.
+ * Runs the leader apply worker.
  *
- * Note that we don't handle FATAL errors which are probably because
- * of system resource error and are not repeatable.
+ * It sets up replication origin, streaming options and then starts streaming.
  */
 static void
-start_apply(XLogRecPtr origin_startpos)
+run_apply_worker()
 {
-	PG_TRY();
+	char		originname[NAMEDATALEN];
+	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
+	char	   *slotname = NULL;
+	WalRcvStreamOptions options;
+	RepOriginId originid;
+	TimeLineID	startpointTLI;
+	char	   *err;
+	bool		must_use_password;
+
+	slotname = MySubscription->slotname;
+
+	/*
+	 * This shouldn't happen if the subscription is enabled, but guard against
+	 * DDL bugs or manual catalog changes.  (libpqwalreceiver will crash if
+	 * slot is NULL.)
+	 */
+	if (!slotname)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+				 errmsg("subscription has no replication slot set")));
+
+	/* Setup replication origin tracking. */
+	ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
+									   originname, sizeof(originname));
+	StartTransactionCommand();
+	originid = replorigin_by_name(originname, true);
+	if (!OidIsValid(originid))
+		originid = replorigin_create(originname);
+	replorigin_session_setup(originid, 0);
+	replorigin_session_origin = originid;
+	origin_startpos = replorigin_session_get_progress(false);
+
+	/* Is the use of a password mandatory? */
+	must_use_password = MySubscription->passwordrequired &&
+		!superuser_arg(MySubscription->owner);
+
+	/* Note that the superuser_arg call can access the DB */
+	CommitTransactionCommand();
+
+	LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
+											must_use_password,
+											MySubscription->name, &err);
+
+	if (LogRepWorkerWalRcvConn == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+				 errmsg("could not connect to the publisher: %s", err)));
+
+	/*
+	 * We don't really use the output identify_system for anything but it does
+	 * some initializations on the upstream so let's still call it.
+	 */
+	(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
+
+	set_apply_error_context_origin(originname);
+
+	set_stream_options(&options, slotname, &origin_startpos);
+
+	/*
+	 * Even when the two_phase mode is requested by the user, it remains as
+	 * the tri-state PENDING until all tablesyncs have reached READY state.
+	 * Only then, can it become ENABLED.
+	 *
+	 * Note: If the subscription has no tables then leave the state as
+	 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
+	 * work.
+	 */
+	if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
+		AllTablesyncsReady())
 	{
-		LogicalRepApplyLoop(origin_startpos);
+		/* Start streaming with two_phase enabled */
+		options.proto.logical.twophase = true;
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+
+		StartTransactionCommand();
+		UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
+		MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
+		CommitTransactionCommand();
 	}
-	PG_CATCH();
+	else
 	{
-		if (MySubscription->disableonerr)
-			DisableSubscriptionAndExit();
-		else
-		{
-			/*
-			 * Report the worker failed while applying changes. Abort the
-			 * current transaction so that the stats message is sent in an
-			 * idle state.
-			 */
-			AbortOutOfAnyTransaction();
-			pgstat_report_subscription_error(MySubscription->oid, !am_tablesync_worker());
-
-			PG_RE_THROW();
-		}
+		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
 	}
-	PG_END_TRY();
+
+	ereport(DEBUG1,
+			(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
+							 MySubscription->name,
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
+							 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
+							 "?")));
+
+	/* Run the main loop. */
+	start_apply(origin_startpos);
 }
 
 /*
- * Common initialization for leader apply worker and parallel apply worker.
+ * Common initialization for leader apply worker, parallel apply worker and
+ * tablesync worker.
  *
  * Initialize the database connection, in-memory subscription and necessary
  * config options.
  */
 void
-InitializeApplyWorker(void)
+InitializeLogRepWorker(void)
 {
 	MemoryContext oldctx;
 
@@ -4518,22 +4628,15 @@ InitializeApplyWorker(void)
 	CommitTransactionCommand();
 }
 
-/* Logical Replication Apply worker entry point */
+/* Common function to setup the leader apply or tablesync worker. */
 void
-ApplyWorkerMain(Datum main_arg)
+SetupApplyOrSyncWorker(int worker_slot)
 {
-	int			worker_slot = DatumGetInt32(main_arg);
-	char		originname[NAMEDATALEN];
-	XLogRecPtr	origin_startpos = InvalidXLogRecPtr;
-	char	   *myslotname = NULL;
-	WalRcvStreamOptions options;
-	int			server_version;
-
-	InitializingApplyWorker = true;
-
 	/* Attach to slot */
 	logicalrep_worker_attach(worker_slot);
 
+	Assert(am_tablesync_worker() || am_leader_apply_worker());
+
 	/* Setup signal handling */
 	pqsignal(SIGHUP, SignalHandlerForConfigReload);
 	pqsignal(SIGTERM, die);
@@ -4551,79 +4654,12 @@ ApplyWorkerMain(Datum main_arg)
 	/* Load the libpq-specific functions */
 	load_file("libpqwalreceiver", false);
 
-	InitializeApplyWorker();
-
-	InitializingApplyWorker = false;
+	InitializeLogRepWorker();
 
 	/* Connect to the origin and start the replication. */
 	elog(DEBUG1, "connecting to publisher using connection string \"%s\"",
 		 MySubscription->conninfo);
 
-	if (am_tablesync_worker())
-	{
-		start_table_sync(&origin_startpos, &myslotname);
-
-		ReplicationOriginNameForLogicalRep(MySubscription->oid,
-										   MyLogicalRepWorker->relid,
-										   originname,
-										   sizeof(originname));
-		set_apply_error_context_origin(originname);
-	}
-	else
-	{
-		/* This is the leader apply worker */
-		RepOriginId originid;
-		TimeLineID	startpointTLI;
-		char	   *err;
-		bool		must_use_password;
-
-		myslotname = MySubscription->slotname;
-
-		/*
-		 * This shouldn't happen if the subscription is enabled, but guard
-		 * against DDL bugs or manual catalog changes.  (libpqwalreceiver will
-		 * crash if slot is NULL.)
-		 */
-		if (!myslotname)
-			ereport(ERROR,
-					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-					 errmsg("subscription has no replication slot set")));
-
-		/* Setup replication origin tracking. */
-		StartTransactionCommand();
-		ReplicationOriginNameForLogicalRep(MySubscription->oid, InvalidOid,
-										   originname, sizeof(originname));
-		originid = replorigin_by_name(originname, true);
-		if (!OidIsValid(originid))
-			originid = replorigin_create(originname);
-		replorigin_session_setup(originid, 0);
-		replorigin_session_origin = originid;
-		origin_startpos = replorigin_session_get_progress(false);
-
-		/* Is the use of a password mandatory? */
-		must_use_password = MySubscription->passwordrequired &&
-			!superuser_arg(MySubscription->owner);
-
-		/* Note that the superuser_arg call can access the DB */
-		CommitTransactionCommand();
-
-		LogRepWorkerWalRcvConn = walrcv_connect(MySubscription->conninfo, true,
-												must_use_password,
-												MySubscription->name, &err);
-		if (LogRepWorkerWalRcvConn == NULL)
-			ereport(ERROR,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("could not connect to the publisher: %s", err)));
-
-		/*
-		 * We don't really use the output identify_system for anything but it
-		 * does some initializations on the upstream so let's still call it.
-		 */
-		(void) walrcv_identify_system(LogRepWorkerWalRcvConn, &startpointTLI);
-
-		set_apply_error_context_origin(originname);
-	}
-
 	/*
 	 * Setup callback for syscache so that we know when something changes in
 	 * the subscription relation state.
@@ -4631,91 +4667,21 @@ ApplyWorkerMain(Datum main_arg)
 	CacheRegisterSyscacheCallback(SUBSCRIPTIONRELMAP,
 								  invalidate_syncing_table_states,
 								  (Datum) 0);
+}
 
-	/* Build logical replication streaming options. */
-	options.logical = true;
-	options.startpoint = origin_startpos;
-	options.slotname = myslotname;
-
-	server_version = walrcv_server_version(LogRepWorkerWalRcvConn);
-	options.proto.logical.proto_version =
-		server_version >= 160000 ? LOGICALREP_PROTO_STREAM_PARALLEL_VERSION_NUM :
-		server_version >= 150000 ? LOGICALREP_PROTO_TWOPHASE_VERSION_NUM :
-		server_version >= 140000 ? LOGICALREP_PROTO_STREAM_VERSION_NUM :
-		LOGICALREP_PROTO_VERSION_NUM;
-
-	options.proto.logical.publication_names = MySubscription->publications;
-	options.proto.logical.binary = MySubscription->binary;
-
-	/*
-	 * Assign the appropriate option value for streaming option according to
-	 * the 'streaming' mode and the publisher's ability to support that mode.
-	 */
-	if (server_version >= 160000 &&
-		MySubscription->stream == LOGICALREP_STREAM_PARALLEL)
-	{
-		options.proto.logical.streaming_str = "parallel";
-		MyLogicalRepWorker->parallel_apply = true;
-	}
-	else if (server_version >= 140000 &&
-			 MySubscription->stream != LOGICALREP_STREAM_OFF)
-	{
-		options.proto.logical.streaming_str = "on";
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-	else
-	{
-		options.proto.logical.streaming_str = NULL;
-		MyLogicalRepWorker->parallel_apply = false;
-	}
-
-	options.proto.logical.twophase = false;
-	options.proto.logical.origin = pstrdup(MySubscription->origin);
+/* Logical Replication Apply worker entry point */
+void
+ApplyWorkerMain(Datum main_arg)
+{
+	int			worker_slot = DatumGetInt32(main_arg);
 
-	if (!am_tablesync_worker())
-	{
-		/*
-		 * Even when the two_phase mode is requested by the user, it remains
-		 * as the tri-state PENDING until all tablesyncs have reached READY
-		 * state. Only then, can it become ENABLED.
-		 *
-		 * Note: If the subscription has no tables then leave the state as
-		 * PENDING, which allows ALTER SUBSCRIPTION ... REFRESH PUBLICATION to
-		 * work.
-		 */
-		if (MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING &&
-			AllTablesyncsReady())
-		{
-			/* Start streaming with two_phase enabled */
-			options.proto.logical.twophase = true;
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
+	InitializingApplyWorker = true;
 
-			StartTransactionCommand();
-			UpdateTwoPhaseState(MySubscription->oid, LOGICALREP_TWOPHASE_STATE_ENABLED);
-			MySubscription->twophasestate = LOGICALREP_TWOPHASE_STATE_ENABLED;
-			CommitTransactionCommand();
-		}
-		else
-		{
-			walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-		}
+	SetupApplyOrSyncWorker(worker_slot);
 
-		ereport(DEBUG1,
-				(errmsg_internal("logical replication apply worker for subscription \"%s\" two_phase is %s",
-								 MySubscription->name,
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_DISABLED ? "DISABLED" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_PENDING ? "PENDING" :
-								 MySubscription->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED ? "ENABLED" :
-								 "?")));
-	}
-	else
-	{
-		/* Start normal logical streaming replication. */
-		walrcv_startstreaming(LogRepWorkerWalRcvConn, &options);
-	}
+	InitializingApplyWorker = false;
 
-	/* Run the main loop. */
-	start_apply(origin_startpos);
+	run_apply_worker();
 
 	proc_exit(0);
 }
@@ -4724,7 +4690,7 @@ ApplyWorkerMain(Datum main_arg)
  * After error recovery, disable the subscription in a new transaction
  * and exit cleanly.
  */
-static void
+void
 DisableSubscriptionAndExit(void)
 {
 	/*
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 39588da79f..bbd71d0b42 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -18,6 +18,7 @@ extern PGDLLIMPORT volatile sig_atomic_t ParallelApplyMessagePending;
 
 extern void ApplyWorkerMain(Datum main_arg);
 extern void ParallelApplyWorkerMain(Datum main_arg);
+extern void TablesyncWorkerMain(Datum main_arg);
 
 extern bool IsLogicalWorker(void);
 extern bool IsLogicalParallelApplyWorker(void);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 343e781896..672a7117c0 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -19,6 +19,7 @@
 #include "datatype/timestamp.h"
 #include "miscadmin.h"
 #include "replication/logicalrelation.h"
+#include "replication/walreceiver.h"
 #include "storage/buffile.h"
 #include "storage/fileset.h"
 #include "storage/lock.h"
@@ -243,7 +244,6 @@ extern int	logicalrep_sync_worker_count(Oid subid);
 
 extern void ReplicationOriginNameForLogicalRep(Oid suboid, Oid relid,
 											   char *originname, Size szoriginname);
-extern char *LogicalRepSyncTableStart(XLogRecPtr *origin_startpos);
 
 extern bool AllTablesyncsReady(void);
 extern void UpdateTwoPhaseState(Oid suboid, char new_state);
@@ -265,7 +265,17 @@ extern void maybe_reread_subscription(void);
 
 extern void stream_cleanup_files(Oid subid, TransactionId xid);
 
-extern void InitializeApplyWorker(void);
+extern void set_stream_options(WalRcvStreamOptions *options,
+							   char *slotname,
+							   XLogRecPtr *origin_startpos);
+
+extern void start_apply(XLogRecPtr origin_startpos);
+
+extern void InitializeLogRepWorker(void);
+
+extern void SetupApplyOrSyncWorker(int worker_slot);
+
+extern void DisableSubscriptionAndExit(void);
 
 extern void store_flush_position(XLogRecPtr remote_lsn, XLogRecPtr local_lsn);
 
-- 
2.39.1

#141Peter Smith
smithpb2250@gmail.com
In reply to: Amit Kapila (#140)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Wed, Aug 2, 2023 at 11:19 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

On Wed, Aug 2, 2023 at 4:09 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

PFA an updated version with some of the earlier reviews addressed.
Forgot to include them in the previous email.

It is always better to explicitly tell which reviews are addressed but
anyway, I have done some minor cleanup in the 0001 patch including
removing includes which didn't seem necessary, modified a few
comments, and ran pgindent. I also thought of modifying some variable
names based on suggestions by Peter Smith in an email [1] but didn't
find many of them any better than the current ones so modified just a
few of those. If you guys are okay with this then let's commit it and
then we can focus more on the remaining patches.

I checked the latest patch v25-0001.

LGTM.

~~

BTW, I have re-tested many cases of HEAD versus HEAD+v25-0001 (using
current test scripts previously mentioned in this thread). Because
v25-0001 is only a refactoring patch we expect that the results should
be the same as for HEAD, and that is what I observed.

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#142Amit Kapila
amit.kapila16@gmail.com
In reply to: Peter Smith (#141)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Thu, Aug 3, 2023 at 9:35 AM Peter Smith <smithpb2250@gmail.com> wrote:

On Wed, Aug 2, 2023 at 11:19 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

On Wed, Aug 2, 2023 at 4:09 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

PFA an updated version with some of the earlier reviews addressed.
Forgot to include them in the previous email.

It is always better to explicitly tell which reviews are addressed but
anyway, I have done some minor cleanup in the 0001 patch including
removing includes which didn't seem necessary, modified a few
comments, and ran pgindent. I also thought of modifying some variable
names based on suggestions by Peter Smith in an email [1] but didn't
find many of them any better than the current ones so modified just a
few of those. If you guys are okay with this then let's commit it and
then we can focus more on the remaining patches.

I checked the latest patch v25-0001.

LGTM.

Thanks, I have pushed 0001. Let's focus on the remaining patches.

--
With Regards,
Amit Kapila.

#143Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#139)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Melih,

Now that v25-0001 has been pushed, can you please rebase the remaining patches?

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#144Peter Smith
smithpb2250@gmail.com
In reply to: Peter Smith (#143)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Just to clarify my previous post, I meant we will need new v26* patches

v24-0001 -> not needed because v25-0001 pushed
v24-0002 -> v26-0001
v24-0003 -> v26-0002

Show quoted text

On Thu, Aug 3, 2023 at 6:19 PM Peter Smith <smithpb2250@gmail.com> wrote:

Hi Melih,

Now that v25-0001 has been pushed, can you please rebase the remaining patches?

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#145Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Amit Kapila (#142)
2 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi,

Amit Kapila <amit.kapila16@gmail.com>, 3 Ağu 2023 Per, 09:22 tarihinde şunu
yazdı:

On Thu, Aug 3, 2023 at 9:35 AM Peter Smith <smithpb2250@gmail.com> wrote:

I checked the latest patch v25-0001.

LGTM.

Thanks, I have pushed 0001. Let's focus on the remaining patches.

Thanks!

Peter Smith <smithpb2250@gmail.com>, 3 Ağu 2023 Per, 12:06 tarihinde şunu
yazdı:

Just to clarify my previous post, I meant we will need new v26* patches

Right. I attached the v26 as you asked.

Thanks,
--
Melih Mutlu
Microsoft

Attachments:

v26-0001-Reuse-Tablesync-Workers.patchapplication/octet-stream; name=v26-0001-Reuse-Tablesync-Workers.patchDownload
From 344720c8c7b976e211239e551c244e66f6f7491c Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:04:46 +0300
Subject: [PATCH v26 1/2] Reuse Tablesync Workers

Before this patch, tablesync workers were capable of syncing only one
table. For each table, a new sync worker was launched and that worker would
exit when done processing the table.

Now, tablesync workers are not limited to processing only one
table. When done, they can move to processing another table in
the same subscription.

If there is a table that needs to be synced, an available tablesync
worker picks up that table and syncs it. Each tablesync worker
continues to pick new tables to sync until there are no tables left
requiring synchronization. If there was no available worker to
process the table, then a new tablesync worker will be launched,
provided the number of tablesync workers for the subscription does not
exceed max_sync_workers_per_subscription.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/replication/logical/launcher.c  |   1 +
 src/backend/replication/logical/tablesync.c | 117 ++++++++++++++++++--
 src/backend/replication/logical/worker.c    |  49 ++++++--
 src/include/replication/worker_internal.h   |   2 +
 4 files changed, 151 insertions(+), 18 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e231fa7f95..25dd06b8af 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -440,6 +440,7 @@ retry:
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
+	worker->relsync_completed = false;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 651a775065..31d4a7db15 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -131,10 +131,12 @@ static StringInfo copybuf = NULL;
 
 /*
  * Exit routine for synchronization worker.
+ *
+ * If reuse_worker is false, at the conclusion of this function the worker
+ * process will exit.
  */
 static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+finish_sync_worker(bool reuse_worker)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -146,21 +148,38 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
+	/*
+	 * Disconnect from the publisher otherwise reusing the sync worker can
+	 * error due to exceeding max_wal_senders.
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		walrcv_disconnect(LogRepWorkerWalRcvConn);
+		LogRepWorkerWalRcvConn = NULL;
+	}
+
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
 	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
+	if (reuse_worker)
+		ereport(LOG,
+				errmsg("logical replication table synchronization worker for subscription \"%s\" will be reused to sync table \"%s\" with relid %u.",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid));
+	else
+		ereport(LOG,
+				errmsg("logical replication table synchronization worker for subscription \"%s\" has finished",
+						MySubscription->name));
 	CommitTransactionCommand();
 
 	/* Find the leader apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
 
 	/* Stop gracefully */
-	proc_exit(0);
+	if (!reuse_worker)
+		proc_exit(0);
 }
 
 /*
@@ -380,7 +399,15 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 */
 		replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
+		/* Sync worker has completed synchronization of the current table. */
+		MyLogicalRepWorker->relsync_completed = true;
+
+		ereport(LOG,
+				errmsg("logical replication table synchronization for subscription \"%s\", relation \"%s\" with relid %u has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid));
+		CommitTransactionCommand();
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -1285,7 +1312,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		case SUBREL_STATE_SYNCDONE:
 		case SUBREL_STATE_READY:
 		case SUBREL_STATE_UNKNOWN:
-			finish_sync_worker();	/* doesn't return */
+			finish_sync_worker(false);	/* doesn't return */
 	}
 
 	/* Calculate the name of the tablesync slot. */
@@ -1643,6 +1670,8 @@ run_tablesync_worker()
 	char	   *slotname = NULL;
 	WalRcvStreamOptions options;
 
+	MyLogicalRepWorker->relsync_completed = false;
+
 	start_table_sync(&origin_startpos, &slotname);
 
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
@@ -1665,12 +1694,78 @@ void
 TablesyncWorkerMain(Datum main_arg)
 {
 	int			worker_slot = DatumGetInt32(main_arg);
+	bool 		done = false;
 
 	SetupApplyOrSyncWorker(worker_slot);
 
-	run_tablesync_worker();
+	/*
+	 * The loop where worker does its job. It loops until there is no relation
+	 * left to sync.
+	 */
+	for (;!done;)
+	{
+		run_tablesync_worker();
+
+		if (IsTransactionState())
+			CommitTransactionCommand();
+
+		if (MyLogicalRepWorker->relsync_completed)
+		{
+			List	   *rstates;
+			ListCell   *lc;
+
+			/*
+			 * This tablesync worker is 'done' unless another table that needs
+			 * syncing is found.
+			 */
+			done = true;
+
+			/* This transaction will be committed by finish_sync_worker. */
+			StartTransactionCommand();
+
+			/*
+			 * Check if there is any table whose relation state is still INIT.
+			 * If a table in INIT state is found, the worker will not be
+			 * finished, it will be reused instead.
+			 */
+			rstates = GetSubscriptionRelations(MySubscription->oid, true);
+
+			foreach(lc, rstates)
+			{
+				SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
+
+				if (rstate->state == SUBREL_STATE_SYNCDONE)
+					continue;
+
+				/*
+				 * Take exclusive lock to prevent any other sync worker from
+				 * picking the same table.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
+
+				/*
+				 * Pick the table for the next run if it is not already picked
+				 * up by another worker.
+				 */
+				if (!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+				{
+					/* Update worker state for the next table */
+					MyLogicalRepWorker->relid = rstate->relid;
+					MyLogicalRepWorker->relstate = rstate->state;
+					MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+					LWLockRelease(LogicalRepWorkerLock);
+
+					/* Found a table for next iteration */
+					finish_sync_worker(true);
+					done = false;
+					break;
+				}
+				LWLockRelease(LogicalRepWorkerLock);
+			}
+		}
+	}
 
-	finish_sync_worker();
+	finish_sync_worker(false);
 }
 
 /*
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index a9f5fa7dfc..32087a00b9 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3621,12 +3621,30 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				if (am_tablesync_worker())
+				{
+					/*
+					 * apply_dispatch() may have gone into apply_handle_commit()
+					 * which can call process_syncing_tables_for_sync.
+					 *
+					 * process_syncing_tables_for_sync decides whether the sync of
+					 * the current table is completed. If it is completed,
+					 * streaming must be already ended. So, we can break the loop.
+					 */
+					if (MyLogicalRepWorker->relsync_completed)
+					{
+						endofstream = true;
+						break;
+					}
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
 
 		/* confirm all writes so far */
-		send_feedback(last_received, false, false);
+		if (!MyLogicalRepWorker->relsync_completed)
+			send_feedback(last_received, false, false);
 
 		if (!in_remote_transaction && !in_streamed_transaction)
 		{
@@ -3640,6 +3658,18 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+
+			if (am_tablesync_worker())
+			{
+				/*
+				 * If relsync_completed is true, this means that the tablesync
+				 * worker is done with synchronization. Streaming has already been
+				 * ended by process_syncing_tables_for_sync. We should move to the
+				 * next table if needed, or exit.
+				 */
+				if (MyLogicalRepWorker->relsync_completed)
+					endofstream = true;
+			}
 		}
 
 		/* Cleanup the memory. */
@@ -3742,8 +3772,12 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * End streaming here for only apply workers. Ending streaming for
+	 * tablesync workers is deferred until the worker exits its main loop.
+	 */
+	if (!am_tablesync_worker())
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
 }
 
 /*
@@ -4617,13 +4651,14 @@ InitializeLogRepWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has started",
+				errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" with relid %u has started",
 						MySubscription->name,
-						get_rel_name(MyLogicalRepWorker->relid))));
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid));
 	else
 		ereport(LOG,
-				(errmsg("logical replication apply worker for subscription \"%s\" has started",
-						MySubscription->name)));
+				errmsg("logical replication apply worker for subscription \"%s\" has started",
+						MySubscription->name));
 
 	CommitTransactionCommand();
 }
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 672a7117c0..10bea1d533 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -56,6 +56,8 @@ typedef struct LogicalRepWorker
 	char		relstate;
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
+	bool		relsync_completed; /* has tablesync finished syncing
+									* the assigned table? */
 
 	/*
 	 * Used to create the changes and subxact files for the streaming
-- 
2.25.1

v26-0002-Reuse-connection-when-tablesync-workers-change-t.patchapplication/octet-stream; name=v26-0002-Reuse-connection-when-tablesync-workers-change-t.patchDownload
From 9e691523e67ff63f730513a9fe051be49bf70473 Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:13:52 +0300
Subject: [PATCH v26 2/2] Reuse connection when tablesync workers change the
 target

Previously tablesync workers establish new connections when it changes the syncing
table, but this might have additional overhead. This patch allows the existing
connection to be reused.

As for the publisher node, this patch allows to reuse logical walsender processes
after the streaming is done once.
---
 src/backend/replication/logical/launcher.c  |  1 +
 src/backend/replication/logical/tablesync.c | 60 ++++++++++++++-------
 src/backend/replication/logical/worker.c    | 18 ++++---
 src/backend/replication/walsender.c         |  6 +++
 src/include/replication/worker_internal.h   |  3 ++
 5 files changed, 61 insertions(+), 27 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 25dd06b8af..b19437f9d0 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
 	worker->relsync_completed = false;
+	worker->slot_number = slot;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 31d4a7db15..ec2f67d879 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -148,16 +148,6 @@ finish_sync_worker(bool reuse_worker)
 		pgstat_report_stat(true);
 	}
 
-	/*
-	 * Disconnect from the publisher otherwise reusing the sync worker can
-	 * error due to exceeding max_wal_senders.
-	 */
-	if (LogRepWorkerWalRcvConn != NULL)
-	{
-		walrcv_disconnect(LogRepWorkerWalRcvConn);
-		LogRepWorkerWalRcvConn = NULL;
-	}
-
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
@@ -1261,6 +1251,27 @@ ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
 			 relid, GetSystemIdentifier());
 }
 
+/*
+ * Determine the application_name for tablesync workers.
+ *
+ * Previously, the replication slot name was used as application_name. Since
+ * it's possible to reuse tablesync workers now, a tablesync worker can handle
+ * several different replication slots during its lifetime. Therefore, we
+ * cannot use the slot name as application_name anymore. Instead, the slot
+ * number of the tablesync worker is used as a part of the application_name.
+ *
+ * XXX: if the tablesync worker starts to reuse the replication slot during
+ * synchronization, we should again use the replication slot name as
+ * application_name.
+ */
+static void
+ApplicationNameForTablesync(Oid suboid, int worker_slot,
+							char *application_name, Size szapp)
+{
+	snprintf(application_name, szapp, "pg_%u_sync_%i_" UINT64_FORMAT, suboid,
+			 worker_slot, GetSystemIdentifier());
+}
+
 /*
  * Start syncing the table in the sync worker.
  *
@@ -1322,15 +1333,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 									slotname,
 									NAMEDATALEN);
 
-	/*
-	 * Here we use the slot name instead of the subscription name as the
-	 * application_name, so that it is different from the leader apply worker,
-	 * so that synchronous replication can distinguish them.
-	 */
-	LogRepWorkerWalRcvConn =
-		walrcv_connect(MySubscription->conninfo, true,
-					   must_use_password,
-					   slotname, &err);
+	/* Connect to the publisher if haven't done so already. */
+	if (LogRepWorkerWalRcvConn == NULL)
+	{
+		char application_name[NAMEDATALEN];
+
+		/*
+		 * The application_name must differ from the subscription name (used by
+		 * the leader apply worker) because synchronous replication has to be
+		 * able to distinguish this worker from the leader apply worker.
+		 */
+		ApplicationNameForTablesync(MySubscription->oid,
+									MyLogicalRepWorker->slot_number,
+									application_name,
+									NAMEDATALEN);
+		LogRepWorkerWalRcvConn =
+			walrcv_connect(MySubscription->conninfo, true,
+						   must_use_password,
+						   application_name, &err);
+	}
+
 	if (LogRepWorkerWalRcvConn == NULL)
 		ereport(ERROR,
 				(errcode(ERRCODE_CONNECTION_FAILURE),
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 32087a00b9..c49334c284 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3494,20 +3494,22 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	ErrorContextCallback errcallback;
 
 	/*
-	 * Init the ApplyMessageContext which we clean up after each replication
-	 * protocol message.
+	 * Init the ApplyMessageContext if needed. This context is cleaned up
+	 * after each replication protocol message.
 	 */
-	ApplyMessageContext = AllocSetContextCreate(ApplyContext,
-												"ApplyMessageContext",
-												ALLOCSET_DEFAULT_SIZES);
+	if (!ApplyMessageContext)
+		ApplyMessageContext = AllocSetContextCreate(ApplyContext,
+													"ApplyMessageContext",
+													ALLOCSET_DEFAULT_SIZES);
 
 	/*
 	 * This memory context is used for per-stream data when the streaming mode
 	 * is enabled. This context is reset on each stream stop.
 	 */
-	LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
-													"LogicalStreamingContext",
-													ALLOCSET_DEFAULT_SIZES);
+	if (!LogicalStreamingContext)
+		LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
+														"LogicalStreamingContext",
+														ALLOCSET_DEFAULT_SIZES);
 
 	/* mark as idle, before starting to loop */
 	pgstat_report_activity(STATE_IDLE, NULL);
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index d27ef2985d..1d2a87cdcd 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1808,6 +1808,12 @@ exec_replication_command(const char *cmd_string)
 		case T_CreateReplicationSlotCmd:
 			cmdtag = "CREATE_REPLICATION_SLOT";
 			set_ps_display(cmdtag);
+
+			/*
+			 * Reset flags because reusing tablesync workers can mean
+			 * this is the second time here.
+			 */
+			streamingDoneSending = streamingDoneReceiving = false;
 			CreateReplicationSlot((CreateReplicationSlotCmd *) cmd_node);
 			EndReplicationCommand(cmdtag);
 			break;
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 10bea1d533..47fa7fbd55 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -39,6 +39,9 @@ typedef struct LogicalRepWorker
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
+	/* Slot number of this worker. */
+	int			slot_number;
+
 	/* Pointer to proc array. NULL if not running. */
 	PGPROC	   *proc;
 
-- 
2.25.1

#146Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#145)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

FWIW, I confirmed that my review comments for v22* have all been
addressed in the latest v26* patches.

Thanks!

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#147Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#145)
4 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Melih.

Now that the design#1 ERRORs have been fixed, we returned to doing
performance measuring of the design#1 patch versus HEAD.

Unfortunately, we observed that under some particular conditions
(large transactions of 1000 inserts/tx for a busy apply worker, 100
empty tables to be synced) the performance was worse with the design#1
patch applied.

~~

RESULTS

Below are some recent measurements (for 100 empty tables to be synced
when apply worker is already busy). We vary the size of the published
transaction for the "busy" table, and you can see that for certain
large transaction sizes (1000 and 2000 inserts/tx) the design#1
performance was worse than HEAD:

~

The publisher "busy" table does commit every 10 inserts:
2w 4w 8w 16w
HEAD 3945 1138 1166 1205
HEAD+v24-0002 3559 886 355 490
%improvement 10% 22% 70% 59%

~

The publisher "busy" table does commit every 100 inserts:
2w 4w 8w 16w
HEAD 2363 1357 1354 1355
HEAD+v24-0002 2077 1358 762 756
%improvement 12% 0% 44% 44%

~

Publisher "busy" table does commit every 1000 inserts:
2w 4w 8w 16w
HEAD 11898 5855 1868 1631
HEAD+v24-0002 21905 8254 3531 1626
%improvement -84% -41% -89% 0%

^ Note - design#1 was slower than HEAD here

~

Publisher "busy" table does commit every 2000 inserts:
2w 4w 8w 16w
HEAD 21740 7109 3454 1703
HEAD+v24-0002 21585 10877 4779 2293
%improvement 1% -53% -38% -35%

^ Note - design#1 was slower than HEAD here

~

The publisher "busy" table does commit every 5000 inserts:
2w 4w 8w 16w
HEAD 36094 18105 8595 3567
HEAD+v24-0002 36305 18199 8151 3710
%improvement -1% -1% 5% -4%

~

The publisher "busy" table does commit every 10000 inserts:
2w 4w 8w 16w
HEAD 38077 18406 9426 5559
HEAD+v24-0002 36763 18027 8896 4166
%improvement 3% 2% 6% 25%

------

TEST SCRIPTS

The "busy apply" test scripts are basically the same as already posted
[1]: /messages/by-id/CAHut+PuNVNK2+A+R6eV8rKPNBHemCFE4NDtEYfpXbYr6SsvvBg@mail.gmail.com

------
[1]: /messages/by-id/CAHut+PuNVNK2+A+R6eV8rKPNBHemCFE4NDtEYfpXbYr6SsvvBg@mail.gmail.com

Kind Regards,
Peter Smith.
Fujitsu Australia

Attachments:

do_one_test_SUB.shtext/x-sh; charset=US-ASCII; name=do_one_test_SUB.shDownload
do_one_test_PUB.shtext/x-sh; charset=US-ASCII; name=do_one_test_PUB.shDownload
do_one_test_setup.shtext/x-sh; charset=US-ASCII; name=do_one_test_setup.shDownload
testrun.shtext/x-sh; charset=US-ASCII; name=testrun.shDownload
#148Zhijie Hou (Fujitsu)
houzj.fnst@fujitsu.com
In reply to: Melih Mutlu (#145)
RE: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Thursday, August 3, 2023 7:30 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Right. I attached the v26 as you asked. 

Thanks for posting the patches.
 
While reviewing the patch, I noticed one rare case that it's possible that there
are two table sync worker for the same table in the same time.

The patch relies on LogicalRepWorkerLock to prevent concurrent access, but the
apply worker will start a new worker after releasing the lock. So, at the point[1]
where the lock is released and the new table sync worker has not been started,
it seems possible that another old table sync worker will be reused for the
same table.

/* Now safe to release the LWLock */
LWLockRelease(LogicalRepWorkerLock);
*[1]
/*
* If there are free sync worker slot(s), start a new sync
* worker for the table.
*/
if (nsyncworkers < max_sync_workers_per_subscription)
...
logicalrep_worker_launch(MyLogicalRepWorker->dbid,

I can reproduce it by using gdb.

Steps:
1. set max_sync_workers_per_subscription to 1 and setup pub/sub which publishes
two tables(table A and B).
2. when the table sync worker for the table A started, use gdb to block it
before being reused for another table.
3. set max_sync_workers_per_subscription to 2 and use gdb to block the apply
worker at the point after releasing the LogicalRepWorkerLock and before
starting another table sync worker for table B.
4. release the blocked table sync worker, then we can see the table sync worker
is also reused for table B.
5. release the apply worker, then we can see the apply worker will start
another table sync worker for the same table(B).

I think it would be better to prevent this case from happening as this case
will give some unexpected ERROR or LOG. Note that I haven't checked if it would
cause worse problems like duplicate copy or others.

Best Regards,
Hou zj

#149vignesh C
vignesh21@gmail.com
In reply to: Peter Smith (#147)
2 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Melih,

Here is a patch to help in getting the execution at various phases
like: a) replication slot creation time, b) Wal reading c) Number of
WAL records read d) subscription relation state change etc
Couple of observation while we tested with this patch:
1) We noticed that the patch takes more time for finding the decoding
start point.
2) Another observation was that the number of XLOG records read for
identify the consistent point was significantly high with the v26_0001
patch.

HEAD
postgres=# select avg(counttime)/1000 "avgtime(ms)",
median(counttime)/1000 "median(ms)", min(counttime)/1000
"mintime(ms)", max(counttime)/1000 "maxtime(ms)", logtype from test
group by logtype;
avgtime(ms) | median(ms) | mintime(ms) |
maxtime(ms) | logtype
------------------------+------------------------+-------------+-------------+--------------------------
0.00579245283018867920 | 0.00200000000000000000 | 0 |
1 | SNAPSHOT_BUILD
1.2246811320754717 | 0.98550000000000000000 | 0 |
37 | LOGICAL_SLOT_CREATION
171.0863283018867920 | 183.9120000000000000 | 0 |
408 | FIND_DECODING_STARTPOINT
2.0699433962264151 | 1.4380000000000000 | 1 |
49 | INIT_DECODING_CONTEXT
(4 rows)

HEAD + v26-0001 patch
postgres=# select avg(counttime)/1000 "avgtime(ms)",
median(counttime)/1000 "median(ms)", min(counttime)/1000
"mintime(ms)", max(counttime)/1000 "maxtime(ms)", logtype from test
group by logtype;
avgtime(ms) | median(ms) | mintime(ms) |
maxtime(ms) | logtype
------------------------+------------------------+-------------+-------------+--------------------------
0.00588113207547169810 | 0.00500000000000000000 | 0 |
0 | SNAPSHOT_BUILD
1.1270962264150943 | 1.1000000000000000 | 0 |
2 | LOGICAL_SLOT_CREATION
301.1745528301886790 | 410.4870000000000000 | 0 |
427 | FIND_DECODING_STARTPOINT
1.4814660377358491 | 1.4530000000000000 | 1 |
9 | INIT_DECODING_CONTEXT
(4 rows)

In the above FIND_DECODING_STARTPOINT is very much higher with V26-0001 patch.

HEAD
FIND_DECODING_XLOG_RECORD_COUNT
- average = 2762
- median = 3362

HEAD + reuse worker patch(v26_0001 patch)
Where FIND_DECODING_XLOG_RECORD_COUNT
- average = 4105
- median = 5345

Similarly Number of xlog records read is higher with v26_0001 patch.

Steps to calculate the timing:
-- first collect the necessary LOG from subscriber's log.
cat *.log | grep -E
'(LOGICAL_SLOT_CREATION|INIT_DECODING_CONTEXT|FIND_DECODING_STARTPOINT|SNAPSHOT_BUILD|FIND_DECODING_XLOG_RECORD_COUNT|LOGICAL_XLOG_READ|LOGICAL_DECODE_PROCESS_RECORD|LOGICAL_WAIT_TRANSACTION)'

grep.dat

create table testv26(logtime varchar, pid varchar, level varchar,
space varchar, logtype varchar, counttime int);
-- then copy these datas into db table to count the avg number.
COPY testv26 FROM '/home/logs/grep.dat' DELIMITER ' ';

-- Finally, use the SQL to analyze the data:
select avg(counttime)/1000 "avgtime(ms)", logtype from testv26 group by logtype;

--- To get the number of xlog records read:
select avg(counttime) from testv26 where logtype
='FIND_DECODING_XLOG_RECORD_COUNT' and counttime != 1;

Thanks to Peter and Hou-san who helped in finding these out. We are
parallely analysing this, @Melih Mutlu posting this information so
that it might help you too in analysing this issue.

Regards,
Vignesh

Attachments:

v1-0001-count-state-change-time.patchtext/x-patch; charset=US-ASCII; name=v1-0001-count-state-change-time.patchDownload
From b755cab38ff76e9f63304b2d8f344cb098ca6a33 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 4 Aug 2023 17:57:29 +0800
Subject: [PATCH v1 1/2] count state change time

---
 src/backend/replication/logical/tablesync.c | 28 +++++++++++++++++++++
 1 file changed, 28 insertions(+)

diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 651a775065..0d9298f7b3 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -123,6 +123,10 @@
 #include "utils/syscache.h"
 #include "utils/usercontext.h"
 
+static TimestampTz start = 0;
+static long		secs = 0;
+static int			microsecs = 0;
+
 static bool table_states_valid = false;
 static List *table_states_not_ready = NIL;
 static bool FetchTableStates(bool *started_tx);
@@ -338,6 +342,11 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		ReplicationSlotDropAtPubNode(LogRepWorkerWalRcvConn, syncslotname, false);
 
 		CommitTransactionCommand();
+
+		TimestampDifference(start, GetCurrentTimestamp(), &secs, &microsecs);
+		elog(LOG, "SUBREL_STATE_SYNCDONE %d", ((int) secs * 1000000 + microsecs));
+		start = GetCurrentTimestamp();
+
 		pgstat_report_stat(false);
 
 		/*
@@ -1258,6 +1267,8 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	bool		must_use_password;
 	bool		run_as_owner;
 
+	start = GetCurrentTimestamp();
+
 	/* Check the state of the table synchronization. */
 	StartTransactionCommand();
 	relstate = GetSubscriptionRelState(MyLogicalRepWorker->subid,
@@ -1361,6 +1372,10 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 	MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
 	SpinLockRelease(&MyLogicalRepWorker->relmutex);
 
+	TimestampDifference(start, GetCurrentTimestamp(), &secs, &microsecs);
+	elog(LOG, "SUBREL_STATE_DATASYNC %d", ((int) secs * 1000000 + microsecs));
+	start = GetCurrentTimestamp();
+
 	/* Update the state and make it visible to others. */
 	StartTransactionCommand();
 	UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
@@ -1404,6 +1419,10 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 					   slotname, false /* permanent */ , false /* two_phase */ ,
 					   CRS_USE_SNAPSHOT, origin_startpos);
 
+	TimestampDifference(start, GetCurrentTimestamp(), &secs, &microsecs);
+	elog(LOG, "WALRCV_CREATE_SLOT %d", ((int) secs * 1000000 + microsecs));
+	start = GetCurrentTimestamp();
+
 	/*
 	 * Setup replication origin tracking. The purpose of doing this before the
 	 * copy is to avoid doing the copy again due to any error in setting up
@@ -1502,6 +1521,10 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 
 	CommitTransactionCommand();
 
+	TimestampDifference(start, GetCurrentTimestamp(), &secs, &microsecs);
+	elog(LOG, "SUBREL_STATE_FINISHEDCOPY %d", ((int) secs * 1000000 + microsecs));
+	start = GetCurrentTimestamp();
+
 copy_table_done:
 
 	elog(DEBUG1,
@@ -1521,6 +1544,11 @@ copy_table_done:
 	 * then return to let LogicalRepApplyLoop do it.
 	 */
 	wait_for_worker_state_change(SUBREL_STATE_CATCHUP);
+
+	TimestampDifference(start, GetCurrentTimestamp(), &secs, &microsecs);
+	elog(LOG, "SUBREL_STATE_CATCHUP %d", ((int) secs * 1000000 + microsecs));
+	start = GetCurrentTimestamp();
+
 	return slotname;
 }
 
-- 
2.34.1

v1-0002-Logs-to-measure-creation-of-replication-slot-brea.patchtext/x-patch; charset=US-ASCII; name=v1-0002-Logs-to-measure-creation-of-replication-slot-brea.patchDownload
From 0b9b3944b572165072e00cafb0bbc8f5a80554be Mon Sep 17 00:00:00 2001
From: Vignesh C <vignesh21@gmail.com>
Date: Mon, 7 Aug 2023 12:26:20 +0530
Subject: [PATCH v1 2/2] Logs to measure creation of replication slot breakup.

Logs to measure creation of replication slot breakup.
---
 src/backend/replication/logical/logical.c   | 27 +++++++++++++++-
 src/backend/replication/logical/snapbuild.c | 14 +++++++-
 src/backend/replication/walsender.c         | 36 +++++++++++++++++++++
 3 files changed, 75 insertions(+), 2 deletions(-)

diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 41243d0187..aa195c4aa9 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -630,11 +630,20 @@ void
 DecodingContextFindStartpoint(LogicalDecodingContext *ctx)
 {
 	ReplicationSlot *slot = ctx->slot;
+	int count = 0;
+
+	instr_time	start;
+	instr_time	elapsed;
+	instr_time	total_read;
+	instr_time	total_decode;
+
+	INSTR_TIME_SET_ZERO(total_read);
+	INSTR_TIME_SET_ZERO(total_decode);
 
 	/* Initialize from where to start reading WAL. */
 	XLogBeginRead(ctx->reader, slot->data.restart_lsn);
 
-	elog(DEBUG1, "searching for logical decoding starting point, starting at %X/%X",
+	elog(LOG, "searching for logical decoding starting point, starting at %X/%X",
 		 LSN_FORMAT_ARGS(slot->data.restart_lsn));
 
 	/* Wait for a consistent starting point */
@@ -642,16 +651,29 @@ DecodingContextFindStartpoint(LogicalDecodingContext *ctx)
 	{
 		XLogRecord *record;
 		char	   *err = NULL;
+		count++;
+
+		INSTR_TIME_SET_CURRENT(start);
 
 		/* the read_page callback waits for new WAL */
 		record = XLogReadRecord(ctx->reader, &err);
+		INSTR_TIME_SET_CURRENT(elapsed);
+		INSTR_TIME_SUBTRACT(elapsed, start);
+		INSTR_TIME_ADD(total_read, elapsed);
+
 		if (err)
 			elog(ERROR, "could not find logical decoding starting point: %s", err);
 		if (!record)
 			elog(ERROR, "could not find logical decoding starting point");
 
+		INSTR_TIME_SET_CURRENT(start);
+
 		LogicalDecodingProcessRecord(ctx, ctx->reader);
 
+		INSTR_TIME_SET_CURRENT(elapsed);
+		INSTR_TIME_SUBTRACT(elapsed, start);
+		INSTR_TIME_ADD(total_decode, elapsed);
+
 		/* only continue till we found a consistent spot */
 		if (DecodingContextReady(ctx))
 			break;
@@ -664,6 +686,9 @@ DecodingContextFindStartpoint(LogicalDecodingContext *ctx)
 	if (slot->data.two_phase)
 		slot->data.two_phase_at = ctx->reader->EndRecPtr;
 	SpinLockRelease(&slot->mutex);
+	elog(LOG, "LOGICAL_XLOG_READ %ld", INSTR_TIME_GET_MICROSEC(total_read));
+	elog(LOG, "LOGICAL_DECODE_PROCESS_RECORD %ld", INSTR_TIME_GET_MICROSEC(total_decode));
+	elog(LOG, "FIND_DECODING_XLOG_RECORD_COUNT %d", count);
 }
 
 /*
diff --git a/src/backend/replication/logical/snapbuild.c b/src/backend/replication/logical/snapbuild.c
index 843ceba840..2ce302a597 100644
--- a/src/backend/replication/logical/snapbuild.c
+++ b/src/backend/replication/logical/snapbuild.c
@@ -1209,6 +1209,8 @@ SnapBuildXidHasCatalogChanges(SnapBuild *builder, TransactionId xid,
  * -----------------------------------
  */
 
+extern instr_time	total_wait;
+
 /*
  * Process a running xacts record, and use its information to first build a
  * historic snapshot and later to release resources that aren't needed
@@ -1227,8 +1229,18 @@ SnapBuildProcessRunningXacts(SnapBuild *builder, XLogRecPtr lsn, xl_running_xact
 	 */
 	if (builder->state < SNAPBUILD_CONSISTENT)
 	{
+		instr_time	start;
+		instr_time	elapsed;
+		bool result;
+
+		INSTR_TIME_SET_CURRENT(start);
+		result = SnapBuildFindSnapshot(builder, lsn, running);
+		INSTR_TIME_SET_CURRENT(elapsed);
+		INSTR_TIME_SUBTRACT(elapsed, start);
+		INSTR_TIME_ADD(total_wait, elapsed);
+
 		/* returns false if there's no point in performing cleanup just yet */
-		if (!SnapBuildFindSnapshot(builder, lsn, running))
+		if (!result)
 			return;
 	}
 	else
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index d27ef2985d..13831bdc6f 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -197,6 +197,10 @@ static volatile sig_atomic_t replication_active = false;
 
 static LogicalDecodingContext *logical_decoding_ctx = NULL;
 
+static TimestampTz start = 0;
+static long		secs = 0;
+static int			microsecs = 0;
+
 /* A sample associating a WAL location with the time it was written. */
 typedef struct
 {
@@ -1034,6 +1038,8 @@ parseCreateReplSlotOptions(CreateReplicationSlotCmd *cmd,
 	}
 }
 
+instr_time	total_wait;
+
 /*
  * Create a new replication slot.
  */
@@ -1052,6 +1058,15 @@ CreateReplicationSlot(CreateReplicationSlotCmd *cmd)
 	Datum		values[4];
 	bool		nulls[4] = {0};
 
+	instr_time	begin;
+	instr_time	elapsed;
+	instr_time	total_create;
+
+	INSTR_TIME_SET_ZERO(total_create);
+	INSTR_TIME_SET_ZERO(total_wait);
+
+	INSTR_TIME_SET_CURRENT(begin);
+
 	Assert(!MyReplicationSlot);
 
 	parseCreateReplSlotOptions(cmd, &reserve_wal, &snapshot_action, &two_phase);
@@ -1083,6 +1098,12 @@ CreateReplicationSlot(CreateReplicationSlotCmd *cmd)
 		LogicalDecodingContext *ctx;
 		bool		need_full_snapshot = false;
 
+		INSTR_TIME_SET_CURRENT(elapsed);
+		INSTR_TIME_SUBTRACT(elapsed, begin);
+		INSTR_TIME_ADD(total_create, elapsed);
+
+		elog(LOG, "LOGICAL_SLOT_CREATION %ld", INSTR_TIME_GET_MICROSEC(total_create));
+
 		/*
 		 * Do options check early so that we can bail before calling the
 		 * DecodingContextFindStartpoint which can take long time.
@@ -1131,6 +1152,7 @@ CreateReplicationSlot(CreateReplicationSlotCmd *cmd)
 			need_full_snapshot = true;
 		}
 
+		start = GetCurrentTimestamp();
 		ctx = CreateInitDecodingContext(cmd->plugin, NIL, need_full_snapshot,
 										InvalidXLogRecPtr,
 										XL_ROUTINE(.page_read = logical_read_xlog_page,
@@ -1139,6 +1161,10 @@ CreateReplicationSlot(CreateReplicationSlotCmd *cmd)
 										WalSndPrepareWrite, WalSndWriteData,
 										WalSndUpdateProgress);
 
+		TimestampDifference(start, GetCurrentTimestamp(), &secs, &microsecs);
+		elog(LOG, "INIT_DECODING_CONTEXT %d", ((int) secs * 1000000 + microsecs));
+		start = GetCurrentTimestamp();
+
 		/*
 		 * Signal that we don't need the timeout mechanism. We're just
 		 * creating the replication slot and don't yet accept feedback
@@ -1151,6 +1177,12 @@ CreateReplicationSlot(CreateReplicationSlotCmd *cmd)
 		/* build initial snapshot, might take a while */
 		DecodingContextFindStartpoint(ctx);
 
+		TimestampDifference(start, GetCurrentTimestamp(), &secs, &microsecs);
+		elog(LOG, "FIND_DECODING_STARTPOINT %d", ((int) secs * 1000000 + microsecs));
+		start = GetCurrentTimestamp();
+
+		elog(LOG, "LOGICAL_WAIT_TRANSACTION %ld", INSTR_TIME_GET_MICROSEC(total_wait));
+
 		/*
 		 * Export or use the snapshot if we've been asked to do so.
 		 *
@@ -1169,6 +1201,10 @@ CreateReplicationSlot(CreateReplicationSlotCmd *cmd)
 			RestoreTransactionSnapshot(snap, MyProc);
 		}
 
+		TimestampDifference(start, GetCurrentTimestamp(), &secs, &microsecs);
+		elog(LOG, "SNAPSHOT_BUILD %d", ((int) secs * 1000000 + microsecs));
+		start = GetCurrentTimestamp();
+
 		/* don't need the decoding context anymore */
 		FreeDecodingContext(ctx);
 
-- 
2.34.1

#150Amit Kapila
amit.kapila16@gmail.com
In reply to: Zhijie Hou (Fujitsu) (#148)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Wed, Aug 9, 2023 at 8:28 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:

On Thursday, August 3, 2023 7:30 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Right. I attached the v26 as you asked.

Thanks for posting the patches.

While reviewing the patch, I noticed one rare case that it's possible that there
are two table sync worker for the same table in the same time.

The patch relies on LogicalRepWorkerLock to prevent concurrent access, but the
apply worker will start a new worker after releasing the lock. So, at the point[1]
where the lock is released and the new table sync worker has not been started,
it seems possible that another old table sync worker will be reused for the
same table.

/* Now safe to release the LWLock */
LWLockRelease(LogicalRepWorkerLock);
*[1]
/*
* If there are free sync worker slot(s), start a new sync
* worker for the table.
*/
if (nsyncworkers < max_sync_workers_per_subscription)
...
logicalrep_worker_launch(MyLogicalRepWorker->dbid,

Yeah, this is a problem. I think one idea to solve this is by
extending the lock duration till we launch the tablesync worker but we
should also consider changing this locking scheme such that there is a
better way to indicate that for a particular rel, tablesync is in
progress. Currently, the code in TablesyncWorkerMain() also acquires
the lock in exclusive mode even though the tablesync for a rel is in
progress which I guess could easily heart us for larger values of
max_logical_replication_workers. So, that could be another motivation
to think for a different locking scheme.

--
With Regards,
Amit Kapila.

#151Peter Smith
smithpb2250@gmail.com
In reply to: vignesh C (#149)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Melih,

FYI -- The same testing was repeated but this time PG was configured
to say synchronous_commit=on. Other factors and scripts were the same
as before --- busy apply, 5 runs, 4 workers, 1000 inserts/tx, 100
empty tables, etc.

There are still more xlog records seen for the v26 patch, but now the
v26 performance was better than HEAD.

RESULTS (synchronous_commit=on)
---------------------------------------------------

Xlog Counts

HEAD
postgres=# select avg(counttime) "avg", median(counttime) "median",
min(counttime) "min", max(counttime) "max", logtype from test_head
group by logtype;
avg | median | min | max |
logtype
-----------------------+-----------------------+-----+------+-----------
-----------------------+-----------------------+-----+------+-----------
-----------------------+-----------------------+-----+------+-----------
1253.7509433962264151 | 1393.0000000000000000 | 1 | 2012 |
FIND_DECODING_XLOG_RECORD_COUNT
(1 row)

HEAD+v26-0001
postgres=# select avg(counttime) "avg", median(counttime) "median",
min(counttime) "min", max(counttime) "max", logtype from test_v26
group by logtype;
avg | median | min | max |
logtype
-----------------------+-----------------------+-----+------+-----------
-----------------------+-----------------------+-----+------+-----------
-----------------------+-----------------------+-----+------+-----------
1278.4075471698113208 | 1423.5000000000000000 | 1 | 2015 |
FIND_DECODING_XLOG_RECORD_COUNT
(1 row)

~~~~~~

Performance

HEAD
[peter@localhost res_0809_vignesh_timing_sync_head]$ cat *.dat_SUB |
grep RESULT | grep -v duration | awk '{print $3}'
4014.266
3892.089
4195.318
3571.862
4312.183

HEAD+v26-0001
[peter@localhost res_0809_vignesh_timing_sync_v260001]$ cat *.dat_SUB
| grep RESULT | grep -v duration | awk '{print $3}'
3326.627
3213.028
3433.611
3299.803
3258.821

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#152Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Peter Smith (#151)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Peter and Vignesh,

Peter Smith <smithpb2250@gmail.com>, 7 Ağu 2023 Pzt, 09:25 tarihinde şunu
yazdı:

Hi Melih.

Now that the design#1 ERRORs have been fixed, we returned to doing
performance measuring of the design#1 patch versus HEAD.

Thanks a lot for taking the time to benchmark the patch. It's really
helpful.

Publisher "busy" table does commit every 1000 inserts:

2w 4w 8w 16w
HEAD 11898 5855 1868 1631
HEAD+v24-0002 21905 8254 3531 1626
%improvement -84% -41% -89% 0%

^ Note - design#1 was slower than HEAD here

~

Publisher "busy" table does commit every 2000 inserts:
2w 4w 8w 16w
HEAD 21740 7109 3454 1703
HEAD+v24-0002 21585 10877 4779 2293
%improvement 1% -53% -38% -35%

I assume you meant HEAD+v26-0002 and not v24. I wanted to quickly reproduce
these two cases where the patch was significantly worse. Interestingly my
results are a bit different than yours.

Publisher "busy" table does commit every 1000 inserts:
2w 4w 8w 16w
HEAD 22405 10335 5008 3304
HEAD+v26 19954 8037 4068 2761
%improvement 1% 2% 2% 1%

Publisher "busy" table does commit every 2000 inserts:
2w 4w 8w 16w
HEAD 33122 14220 7251 4279
HEAD+v26 34248 16213 7356 3914
%improvement 0% -1% 0% 1%

If I'm not doing something wrong in testing (or maybe the patch doesn't
perform reliable yet for some reason), I don't see a drastic change in
performance. But I guess the patch is supposed to perform better than HEAD
in these both cases anyway. right?. I would expect the performance of the
patch to converge to HEAD's performance with large tables. But I'm not sure
what to expect when apply worker is busy with large transactions.

However, I need to investigate a bit more what Vignesh shared earlier [1]/messages/by-id/CALDaNm1TA068E2niJFUR9ig+Yz3-ank=j5=j-2UocbzaDnQPrA@mail.gmail.com.
It makes sense that those issues can cause this problem here.

It just takes a bit of time for me to figure out these things, but I'm
working on it.

[1]: /messages/by-id/CALDaNm1TA068E2niJFUR9ig+Yz3-ank=j5=j-2UocbzaDnQPrA@mail.gmail.com
/messages/by-id/CALDaNm1TA068E2niJFUR9ig+Yz3-ank=j5=j-2UocbzaDnQPrA@mail.gmail.com

Thanks,
--
Melih Mutlu
Microsoft

#153Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#152)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Fri, Aug 11, 2023 at 12:54 AM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Hi Peter and Vignesh,

Peter Smith <smithpb2250@gmail.com>, 7 Ağu 2023 Pzt, 09:25 tarihinde şunu yazdı:

Hi Melih.

Now that the design#1 ERRORs have been fixed, we returned to doing
performance measuring of the design#1 patch versus HEAD.

Thanks a lot for taking the time to benchmark the patch. It's really helpful.

Publisher "busy" table does commit every 1000 inserts:
2w 4w 8w 16w
HEAD 11898 5855 1868 1631
HEAD+v24-0002 21905 8254 3531 1626
%improvement -84% -41% -89% 0%

^ Note - design#1 was slower than HEAD here

~

Publisher "busy" table does commit every 2000 inserts:
2w 4w 8w 16w
HEAD 21740 7109 3454 1703
HEAD+v24-0002 21585 10877 4779 2293
%improvement 1% -53% -38% -35%

I assume you meant HEAD+v26-0002 and not v24. I wanted to quickly reproduce these two cases where the patch was significantly worse. Interestingly my results are a bit different than yours.

No, I meant what I wrote there. When I ran the tests the HEAD included
the v25-0001 refactoring patch, but v26 did not yet exist.

For now, we are only performance testing the first
"Reuse-Tablesyc-Workers" patch, but not yet including the second patch
("Reuse connection when...").

Note that those "Reuse-Tablesyc-Workers" patches v24-0002 and v26-0001
are equivalent because there are only cosmetic log message differences
between them.
So, my testing was with HEAD+v24-0002 (but not including v24-0003).
Your same testing should be with HEAD+v26-0001 (but not including v26-0002).

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#154vignesh C
vignesh21@gmail.com
In reply to: vignesh C (#149)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Wed, 9 Aug 2023 at 09:51, vignesh C <vignesh21@gmail.com> wrote:

Hi Melih,

Here is a patch to help in getting the execution at various phases
like: a) replication slot creation time, b) Wal reading c) Number of
WAL records read d) subscription relation state change etc
Couple of observation while we tested with this patch:
1) We noticed that the patch takes more time for finding the decoding
start point.
2) Another observation was that the number of XLOG records read for
identify the consistent point was significantly high with the v26_0001
patch.

HEAD
postgres=# select avg(counttime)/1000 "avgtime(ms)",
median(counttime)/1000 "median(ms)", min(counttime)/1000
"mintime(ms)", max(counttime)/1000 "maxtime(ms)", logtype from test
group by logtype;
avgtime(ms) | median(ms) | mintime(ms) |
maxtime(ms) | logtype
------------------------+------------------------+-------------+-------------+--------------------------
0.00579245283018867920 | 0.00200000000000000000 | 0 |
1 | SNAPSHOT_BUILD
1.2246811320754717 | 0.98550000000000000000 | 0 |
37 | LOGICAL_SLOT_CREATION
171.0863283018867920 | 183.9120000000000000 | 0 |
408 | FIND_DECODING_STARTPOINT
2.0699433962264151 | 1.4380000000000000 | 1 |
49 | INIT_DECODING_CONTEXT
(4 rows)

HEAD + v26-0001 patch
postgres=# select avg(counttime)/1000 "avgtime(ms)",
median(counttime)/1000 "median(ms)", min(counttime)/1000
"mintime(ms)", max(counttime)/1000 "maxtime(ms)", logtype from test
group by logtype;
avgtime(ms) | median(ms) | mintime(ms) |
maxtime(ms) | logtype
------------------------+------------------------+-------------+-------------+--------------------------
0.00588113207547169810 | 0.00500000000000000000 | 0 |
0 | SNAPSHOT_BUILD
1.1270962264150943 | 1.1000000000000000 | 0 |
2 | LOGICAL_SLOT_CREATION
301.1745528301886790 | 410.4870000000000000 | 0 |
427 | FIND_DECODING_STARTPOINT
1.4814660377358491 | 1.4530000000000000 | 1 |
9 | INIT_DECODING_CONTEXT
(4 rows)

In the above FIND_DECODING_STARTPOINT is very much higher with V26-0001 patch.

HEAD
FIND_DECODING_XLOG_RECORD_COUNT
- average = 2762
- median = 3362

HEAD + reuse worker patch(v26_0001 patch)
Where FIND_DECODING_XLOG_RECORD_COUNT
- average = 4105
- median = 5345

Similarly Number of xlog records read is higher with v26_0001 patch.

Steps to calculate the timing:
-- first collect the necessary LOG from subscriber's log.
cat *.log | grep -E
'(LOGICAL_SLOT_CREATION|INIT_DECODING_CONTEXT|FIND_DECODING_STARTPOINT|SNAPSHOT_BUILD|FIND_DECODING_XLOG_RECORD_COUNT|LOGICAL_XLOG_READ|LOGICAL_DECODE_PROCESS_RECORD|LOGICAL_WAIT_TRANSACTION)'

grep.dat

create table testv26(logtime varchar, pid varchar, level varchar,
space varchar, logtype varchar, counttime int);
-- then copy these datas into db table to count the avg number.
COPY testv26 FROM '/home/logs/grep.dat' DELIMITER ' ';

-- Finally, use the SQL to analyze the data:
select avg(counttime)/1000 "avgtime(ms)", logtype from testv26 group by logtype;

--- To get the number of xlog records read:
select avg(counttime) from testv26 where logtype
='FIND_DECODING_XLOG_RECORD_COUNT' and counttime != 1;

Thanks to Peter and Hou-san who helped in finding these out. We are
parallely analysing this, @Melih Mutlu posting this information so
that it might help you too in analysing this issue.

I analysed further on why it needs to read a larger number of XLOG
records in some cases while creating the replication slot, here are my
thoughts:
Note: Tablesync worker needs to connect to the publisher and create
consistent point for the slots by reading the XLOG records. This
requires that all the open transactions and the transactions that are
created while creating consistent point should be committed.
I feel the creation of slots is better in few cases in Head because:
Publisher | Subscriber
------------------------------------------------------------
Begin txn1 transaction |
Insert 1..1000 records |
Commit |
Begin txn2 transaction |
Insert 1..1000 records | Apply worker applies transaction txn1
| Start tablesync table t2
| create consistent point in
| publisher before transaction txn3 is
| started
commit | We just need to wait till
| transaction txn2 is finished.
Begin txn3 transaction |
Insert 1..1000 records |
commit |

In V26, this is happening in some cases:
Publisher | Subscriber
------------------------------------------------------------
Begin txn1 transaction |
Insert 1..1000 records |
Commit |
Begin txn2 transaction |
Insert 1..1000 records | Apply worker applies transaction txn1
| Start tablesync table t2
commit | Create consistent point
Begin txn3 transaction | (since transaction txn2 is committed
| and txn3 is started, we will
| need to wait
| for transaction txn3 to be
| committed)
Insert 1..1000 records |
commit |

This is because In HEAD the tablesync worker will be started after one
commit, so we are able to create the consistent point before a new
transaction is started in some cases.
Create slot will be fastest if the tablesync worker is able to connect
to the publisher and create a consistent point before the new
transaction is started. The probability of this is better in HEAD for
this scenario as the new tablesync worker is started after commit and
the tablesync worker in HEAD has a better time window(because the
current transaction has just started) before another new transaction
is started. This probability is slightly lower with the V26 version.
I felt this issue is purely a timing issue in a few cases because of
the timing of the new transactions being created while creating the
slot.

Regards,
Vignesh

#155vignesh C
vignesh21@gmail.com
In reply to: vignesh C (#154)
1 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Fri, 11 Aug 2023 at 16:26, vignesh C <vignesh21@gmail.com> wrote:

On Wed, 9 Aug 2023 at 09:51, vignesh C <vignesh21@gmail.com> wrote:

Hi Melih,

Here is a patch to help in getting the execution at various phases
like: a) replication slot creation time, b) Wal reading c) Number of
WAL records read d) subscription relation state change etc
Couple of observation while we tested with this patch:
1) We noticed that the patch takes more time for finding the decoding
start point.
2) Another observation was that the number of XLOG records read for
identify the consistent point was significantly high with the v26_0001
patch.

HEAD
postgres=# select avg(counttime)/1000 "avgtime(ms)",
median(counttime)/1000 "median(ms)", min(counttime)/1000
"mintime(ms)", max(counttime)/1000 "maxtime(ms)", logtype from test
group by logtype;
avgtime(ms) | median(ms) | mintime(ms) |
maxtime(ms) | logtype
------------------------+------------------------+-------------+-------------+--------------------------
0.00579245283018867920 | 0.00200000000000000000 | 0 |
1 | SNAPSHOT_BUILD
1.2246811320754717 | 0.98550000000000000000 | 0 |
37 | LOGICAL_SLOT_CREATION
171.0863283018867920 | 183.9120000000000000 | 0 |
408 | FIND_DECODING_STARTPOINT
2.0699433962264151 | 1.4380000000000000 | 1 |
49 | INIT_DECODING_CONTEXT
(4 rows)

HEAD + v26-0001 patch
postgres=# select avg(counttime)/1000 "avgtime(ms)",
median(counttime)/1000 "median(ms)", min(counttime)/1000
"mintime(ms)", max(counttime)/1000 "maxtime(ms)", logtype from test
group by logtype;
avgtime(ms) | median(ms) | mintime(ms) |
maxtime(ms) | logtype
------------------------+------------------------+-------------+-------------+--------------------------
0.00588113207547169810 | 0.00500000000000000000 | 0 |
0 | SNAPSHOT_BUILD
1.1270962264150943 | 1.1000000000000000 | 0 |
2 | LOGICAL_SLOT_CREATION
301.1745528301886790 | 410.4870000000000000 | 0 |
427 | FIND_DECODING_STARTPOINT
1.4814660377358491 | 1.4530000000000000 | 1 |
9 | INIT_DECODING_CONTEXT
(4 rows)

In the above FIND_DECODING_STARTPOINT is very much higher with V26-0001 patch.

HEAD
FIND_DECODING_XLOG_RECORD_COUNT
- average = 2762
- median = 3362

HEAD + reuse worker patch(v26_0001 patch)
Where FIND_DECODING_XLOG_RECORD_COUNT
- average = 4105
- median = 5345

Similarly Number of xlog records read is higher with v26_0001 patch.

Steps to calculate the timing:
-- first collect the necessary LOG from subscriber's log.
cat *.log | grep -E
'(LOGICAL_SLOT_CREATION|INIT_DECODING_CONTEXT|FIND_DECODING_STARTPOINT|SNAPSHOT_BUILD|FIND_DECODING_XLOG_RECORD_COUNT|LOGICAL_XLOG_READ|LOGICAL_DECODE_PROCESS_RECORD|LOGICAL_WAIT_TRANSACTION)'

grep.dat

create table testv26(logtime varchar, pid varchar, level varchar,
space varchar, logtype varchar, counttime int);
-- then copy these datas into db table to count the avg number.
COPY testv26 FROM '/home/logs/grep.dat' DELIMITER ' ';

-- Finally, use the SQL to analyze the data:
select avg(counttime)/1000 "avgtime(ms)", logtype from testv26 group by logtype;

--- To get the number of xlog records read:
select avg(counttime) from testv26 where logtype
='FIND_DECODING_XLOG_RECORD_COUNT' and counttime != 1;

Thanks to Peter and Hou-san who helped in finding these out. We are
parallely analysing this, @Melih Mutlu posting this information so
that it might help you too in analysing this issue.

I analysed further on why it needs to read a larger number of XLOG
records in some cases while creating the replication slot, here are my
thoughts:
Note: Tablesync worker needs to connect to the publisher and create
consistent point for the slots by reading the XLOG records. This
requires that all the open transactions and the transactions that are
created while creating consistent point should be committed.
I feel the creation of slots is better in few cases in Head because:
Publisher | Subscriber
------------------------------------------------------------
Begin txn1 transaction |
Insert 1..1000 records |
Commit |
Begin txn2 transaction |
Insert 1..1000 records | Apply worker applies transaction txn1
| Start tablesync table t2
| create consistent point in
| publisher before transaction txn3 is
| started
commit | We just need to wait till
| transaction txn2 is finished.
Begin txn3 transaction |
Insert 1..1000 records |
commit |

In V26, this is happening in some cases:
Publisher | Subscriber
------------------------------------------------------------
Begin txn1 transaction |
Insert 1..1000 records |
Commit |
Begin txn2 transaction |
Insert 1..1000 records | Apply worker applies transaction txn1
| Start tablesync table t2
commit | Create consistent point
Begin txn3 transaction | (since transaction txn2 is committed
| and txn3 is started, we will
| need to wait
| for transaction txn3 to be
| committed)
Insert 1..1000 records |
commit |

This is because In HEAD the tablesync worker will be started after one
commit, so we are able to create the consistent point before a new
transaction is started in some cases.
Create slot will be fastest if the tablesync worker is able to connect
to the publisher and create a consistent point before the new
transaction is started. The probability of this is better in HEAD for
this scenario as the new tablesync worker is started after commit and
the tablesync worker in HEAD has a better time window(because the
current transaction has just started) before another new transaction
is started. This probability is slightly lower with the V26 version.
I felt this issue is purely a timing issue in a few cases because of
the timing of the new transactions being created while creating the
slot.

I used the following steps to analyse this issue:
Logs can be captured by applying the patches at [1]/messages/by-id/CALDaNm1TA068E2niJFUR9ig+Yz3-ank=j5=j-2UocbzaDnQPrA@mail.gmail.com.

-- first collect the necessary information about from publisher's log
from the execution of HEAD:
cat *.log | grep FIND_DECODING_XLOG_RECORD_COUNT > grep_head.dat

-- first collect the necessary information about from publisher's log
from the execution of v26:
cat *.log | grep FIND_DECODING_XLOG_RECORD_COUNT > grep_v26.dat

-- then copy these datas into HEAD's db table to count the avg number.
COPY test_head FROM '/home/logs/grep_head.dat' DELIMITER ' ';

-- then copy these datas into the v26 db table to count the avg number.
COPY test_v26 FROM '/home/logs/grep_v26.dat' DELIMITER ' ';

Find the average of XLOG records read in HEAD:
postgres=# select avg(counttime) from test_head where logtype
='FIND_DECODING_XLOG_RECORD_COUNT' and counttime != 1;
avg
-----------------------
1394.1100000000000000
(1 row)

Find the average of XLOG records read in V26:
postgres=# select avg(counttime) from test_v26 where logtype
='FIND_DECODING_XLOG_RECORD_COUNT' and counttime != 1;
avg
-----------------------
1900.4100000000000000
(1 row)

When analysing why create replication slot needs to read more records
in a few cases, I found a very interesting observation. I found that
with HEAD about 29% (29 out of 100 tables) of tables could find the
consistent point by reading the WAL records up to the next subsequent
COMMIT, whereas with V26 patch only 5% of tables could find the
consistent point by reading the WAL records up to next subsequent
commit. In these cases V26 patch had to read another transaction of
approximately > 1000 WAL records to reach the consistent point which
results in an increase of average for more records to be read with V26
version. For these I got the start lsn and consistent lsn from the log
files by matching the corresponding FIND_DECODING_XLOG_RECORD_COUNT, I
did a waldump of the WAL file and searched the records between start
lsn and consistent LSN in the WAL dump and confirmed that only one
COMMIT record had to be read to reach the consistent point. Details of
this information from the log of HEAD and V26 is attached.

The number of tables required to read less than 1 commit can be found
by the following:
-- I checked for 1000 WAL records because we are having 1000 inserts
in each transaction.
select count(counttime) from test_head where logtype
='FIND_DECODING_XLOG_RECORD_COUNT' and counttime < 1000;
count
-------
29
(1 row)

select count(counttime) from test_v26 where logtype
='FIND_DECODING_XLOG_RECORD_COUNT' and counttime < 1000;
count
-------
5
(1 row)

Apart from these there were other instances where the V26 had to read
more COMMIT record in few cases.
The above is happening because as mentioned in [2]/messages/by-id/CALDaNm2k2z3Hpa3Omb_tpxWkyHnUvsWjJMbqDs-2uD2eLzemJQ@mail.gmail.com. i.e. in HEAD the
tablesync worker will be started after one commit, so we are able to
create the consistent point before a new transaction is started in
some cases. Create slot will be fastest if the tablesync worker is
able to connect to the publisher and create a consistent point before
the new transaction is started. The probability of this is better in
HEAD for this scenario as the new tablesync worker is started after
commit and the tablesync worker in HEAD has a better time
window(because the current transaction has just started) before
another new transaction is started. This probability is slightly
lower with the V26 version. I felt this issue is purely a timing issue
in a few cases because of the timing of the new transactions being
created while creating the slot.
Since this is purely a timing issue as explained above in a few cases
because of the timing of the new transactions being created while
creating the slot, I felt we can ignore this.

[1]: /messages/by-id/CALDaNm1TA068E2niJFUR9ig+Yz3-ank=j5=j-2UocbzaDnQPrA@mail.gmail.com
[2]: /messages/by-id/CALDaNm2k2z3Hpa3Omb_tpxWkyHnUvsWjJMbqDs-2uD2eLzemJQ@mail.gmail.com

Regards,
Vignesh

Attachments:

log_information_of_consistent_point_1_commit.txttext/plain; charset=US-ASCII; name=log_information_of_consistent_point_1_commit.txtDownload
#156Melih Mutlu
m.melihmutlu@gmail.com
In reply to: Peter Smith (#153)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Peter,

Peter Smith <smithpb2250@gmail.com>, 11 Ağu 2023 Cum, 01:26 tarihinde şunu
yazdı:

No, I meant what I wrote there. When I ran the tests the HEAD included
the v25-0001 refactoring patch, but v26 did not yet exist.

For now, we are only performance testing the first
"Reuse-Tablesyc-Workers" patch, but not yet including the second patch
("Reuse connection when...").

Note that those "Reuse-Tablesyc-Workers" patches v24-0002 and v26-0001
are equivalent because there are only cosmetic log message differences
between them.

Ok, that's fair.

So, my testing was with HEAD+v24-0002 (but not including v24-0003).
Your same testing should be with HEAD+v26-0001 (but not including
v26-0002).

That's actually what I did. I should have been more clear about what I
included in my previous email.With v26-0002, results are noticeably better
anyway.
I just rerun the test again against HEAD, HEAD+v26-0001 and additionally
HEAD+v26-0001+v26-0002 this time, for better comparison.

Here are my results with the same scripts you shared earlier (I obviously
only changed the number of inserts before each commit. ).
Note that this is when synchronous_commit = off.

100 inserts/tx
+-------------+-------+------+------+------+
|             | 2w    | 4w   | 8w   | 16w  |
+-------------+-------+------+------+------+
| v26-0002    | 10421 | 6472 | 6656 | 6566 |
+-------------+-------+------+------+------+
| improvement | 31%   | 12%  | 0%   | 5%   |
+-------------+-------+------+------+------+
| v26-0001    | 14585 | 7386 | 7129 | 7274 |
+-------------+-------+------+------+------+
| improvement | 9%    | 5%   | 12%  | 7%   |
+-------------+-------+------+------+------+
| HEAD        | 16130 | 7785 | 8147 | 7827 |
+-------------+-------+------+------+------+
1000 inserts/tx
+-------------+-------+------+------+------+
|             | 2w    | 4w   | 8w   | 16w  |
+-------------+-------+------+------+------+
| v26-0002    | 13796 | 6848 | 5942 | 6315 |
+-------------+-------+------+------+------+
| improvement | 9%    | 7%   | 10%  | 8%   |
+-------------+-------+------+------+------+
| v26-0001    | 14685 | 7325 | 6675 | 6719 |
+-------------+-------+------+------+------+
| improvement | 3%    | 0%   | 0%   | 2%   |
+-------------+-------+------+------+------+
| HEAD        | 15118 | 7354 | 6644 | 6890 |
+-------------+-------+------+------+------+
2000 inserts/tx
+-------------+-------+-------+------+------+
|             | 2w    | 4w    | 8w   | 16w  |
+-------------+-------+-------+------+------+
| v26-0002    | 22442 | 9944  | 6034 | 5829 |
+-------------+-------+-------+------+------+
| improvement | 5%    | 2%    | 4%   | 10%  |
+-------------+-------+-------+------+------+
| v26-0001    | 23632 | 10164 | 6311 | 6480 |
+-------------+-------+-------+------+------+
| improvement | 0%    | 0%    | 0%   | 0%   |
+-------------+-------+-------+------+------+
| HEAD        | 23667 | 10157 | 6285 | 6470 |
+-------------+-------+-------+------+------+
5000 inserts/tx
+-------------+-------+-------+-------+------+
|             | 2w    | 4w    | 8w    | 16w  |
+-------------+-------+-------+-------+------+
| v26-0002    | 41443 | 21385 | 10832 | 6146 |
+-------------+-------+-------+-------+------+
| improvement | 0%    | 0%    | 1%    | 16%  |
+-------------+-------+-------+-------+------+
| v26-0001    | 41293 | 21226 | 10814 | 6158 |
+-------------+-------+-------+-------+------+
| improvement | 0%    | 1%    | 1%    | 15%  |
+-------------+-------+-------+-------+------+
| HEAD        | 41503 | 21466 | 10943 | 7292 |
+-------------+-------+-------+-------+------+

Again, I couldn't reproduce the cases where you saw significantly degraded
performance. I wonder if I'm missing something. Did you do anything not
included in the test scripts you shared? Do you think v26-0001 will
perform 84% worse than HEAD, if you try again? I just want to be sure that
it was not a random thing.
Interestingly, I also don't see an improvement in above results as big as
in your results when inserts/tx ratio is smaller. Even though it certainly
is improved in such cases.

Thanks,
--
Melih Mutlu
Microsoft

#157Amit Kapila
amit.kapila16@gmail.com
In reply to: Melih Mutlu (#156)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Fri, Aug 11, 2023 at 7:15 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Peter Smith <smithpb2250@gmail.com>, 11 Ağu 2023 Cum, 01:26 tarihinde şunu yazdı:

No, I meant what I wrote there. When I ran the tests the HEAD included
the v25-0001 refactoring patch, but v26 did not yet exist.

For now, we are only performance testing the first
"Reuse-Tablesyc-Workers" patch, but not yet including the second patch
("Reuse connection when...").

Note that those "Reuse-Tablesyc-Workers" patches v24-0002 and v26-0001
are equivalent because there are only cosmetic log message differences
between them.

Ok, that's fair.

So, my testing was with HEAD+v24-0002 (but not including v24-0003).
Your same testing should be with HEAD+v26-0001 (but not including v26-0002).

That's actually what I did. I should have been more clear about what I included in my previous email.With v26-0002, results are noticeably better anyway.
I just rerun the test again against HEAD, HEAD+v26-0001 and additionally HEAD+v26-0001+v26-0002 this time, for better comparison.

Here are my results with the same scripts you shared earlier (I obviously only changed the number of inserts before each commit. ).
Note that this is when synchronous_commit = off.

100 inserts/tx
+-------------+-------+------+------+------+
|             | 2w    | 4w   | 8w   | 16w  |
+-------------+-------+------+------+------+
| v26-0002    | 10421 | 6472 | 6656 | 6566 |
+-------------+-------+------+------+------+
| improvement | 31%   | 12%  | 0%   | 5%   |
+-------------+-------+------+------+------+
| v26-0001    | 14585 | 7386 | 7129 | 7274 |
+-------------+-------+------+------+------+
| improvement | 9%    | 5%   | 12%  | 7%   |
+-------------+-------+------+------+------+
| HEAD        | 16130 | 7785 | 8147 | 7827 |
+-------------+-------+------+------+------+
1000 inserts/tx
+-------------+-------+------+------+------+
|             | 2w    | 4w   | 8w   | 16w  |
+-------------+-------+------+------+------+
| v26-0002    | 13796 | 6848 | 5942 | 6315 |
+-------------+-------+------+------+------+
| improvement | 9%    | 7%   | 10%  | 8%   |
+-------------+-------+------+------+------+
| v26-0001    | 14685 | 7325 | 6675 | 6719 |
+-------------+-------+------+------+------+
| improvement | 3%    | 0%   | 0%   | 2%   |
+-------------+-------+------+------+------+
| HEAD        | 15118 | 7354 | 6644 | 6890 |
+-------------+-------+------+------+------+
2000 inserts/tx
+-------------+-------+-------+------+------+
|             | 2w    | 4w    | 8w   | 16w  |
+-------------+-------+-------+------+------+
| v26-0002    | 22442 | 9944  | 6034 | 5829 |
+-------------+-------+-------+------+------+
| improvement | 5%    | 2%    | 4%   | 10%  |
+-------------+-------+-------+------+------+
| v26-0001    | 23632 | 10164 | 6311 | 6480 |
+-------------+-------+-------+------+------+
| improvement | 0%    | 0%    | 0%   | 0%   |
+-------------+-------+-------+------+------+
| HEAD        | 23667 | 10157 | 6285 | 6470 |
+-------------+-------+-------+------+------+
5000 inserts/tx
+-------------+-------+-------+-------+------+
|             | 2w    | 4w    | 8w    | 16w  |
+-------------+-------+-------+-------+------+
| v26-0002    | 41443 | 21385 | 10832 | 6146 |
+-------------+-------+-------+-------+------+
| improvement | 0%    | 0%    | 1%    | 16%  |
+-------------+-------+-------+-------+------+
| v26-0001    | 41293 | 21226 | 10814 | 6158 |
+-------------+-------+-------+-------+------+
| improvement | 0%    | 1%    | 1%    | 15%  |
+-------------+-------+-------+-------+------+
| HEAD        | 41503 | 21466 | 10943 | 7292 |
+-------------+-------+-------+-------+------+

Again, I couldn't reproduce the cases where you saw significantly degraded performance.

I am not surprised to see that you don't see regression because as per
Vignesh's analysis, this is purely a timing issue where sometimes
after the patch the slot creation can take more time because there is
a constant inflow of transactions on the publisher. I think we are
seeing it because this workload is predominantly just creating and
destroying slots. We can probably improve it later as discussed
earlier by using a single for multiple copies (especially for small
tables) or something like that.

--
With Regards,
Amit Kapila.

#158Amit Kapila
amit.kapila16@gmail.com
In reply to: Amit Kapila (#150)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Thu, Aug 10, 2023 at 10:15 AM Amit Kapila <amit.kapila16@gmail.com> wrote:

On Wed, Aug 9, 2023 at 8:28 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:

On Thursday, August 3, 2023 7:30 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Right. I attached the v26 as you asked.

Thanks for posting the patches.

While reviewing the patch, I noticed one rare case that it's possible that there
are two table sync worker for the same table in the same time.

The patch relies on LogicalRepWorkerLock to prevent concurrent access, but the
apply worker will start a new worker after releasing the lock. So, at the point[1]
where the lock is released and the new table sync worker has not been started,
it seems possible that another old table sync worker will be reused for the
same table.

/* Now safe to release the LWLock */
LWLockRelease(LogicalRepWorkerLock);
*[1]
/*
* If there are free sync worker slot(s), start a new sync
* worker for the table.
*/
if (nsyncworkers < max_sync_workers_per_subscription)
...
logicalrep_worker_launch(MyLogicalRepWorker->dbid,

Yeah, this is a problem. I think one idea to solve this is by
extending the lock duration till we launch the tablesync worker but we
should also consider changing this locking scheme such that there is a
better way to indicate that for a particular rel, tablesync is in
progress. Currently, the code in TablesyncWorkerMain() also acquires
the lock in exclusive mode even though the tablesync for a rel is in
progress which I guess could easily heart us for larger values of
max_logical_replication_workers. So, that could be another motivation
to think for a different locking scheme.

Yet another problem is that currently apply worker maintains a hash
table for 'last_start_times' to avoid restarting the tablesync worker
immediately upon error. The same functionality is missing while
reusing the table sync worker. One possibility is to use a shared hash
table to remember start times but I think it may depend on what we
decide to solve the previous problem reported by Hou-San.

--
With Regards,
Amit Kapila.

#159vignesh C
vignesh21@gmail.com
In reply to: Amit Kapila (#150)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Thu, 10 Aug 2023 at 10:16, Amit Kapila <amit.kapila16@gmail.com> wrote:

On Wed, Aug 9, 2023 at 8:28 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:

On Thursday, August 3, 2023 7:30 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Right. I attached the v26 as you asked.

Thanks for posting the patches.

While reviewing the patch, I noticed one rare case that it's possible that there
are two table sync worker for the same table in the same time.

The patch relies on LogicalRepWorkerLock to prevent concurrent access, but the
apply worker will start a new worker after releasing the lock. So, at the point[1]
where the lock is released and the new table sync worker has not been started,
it seems possible that another old table sync worker will be reused for the
same table.

/* Now safe to release the LWLock */
LWLockRelease(LogicalRepWorkerLock);
*[1]
/*
* If there are free sync worker slot(s), start a new sync
* worker for the table.
*/
if (nsyncworkers < max_sync_workers_per_subscription)
...
logicalrep_worker_launch(MyLogicalRepWorker->dbid,

Yeah, this is a problem. I think one idea to solve this is by
extending the lock duration till we launch the tablesync worker but we
should also consider changing this locking scheme such that there is a
better way to indicate that for a particular rel, tablesync is in
progress. Currently, the code in TablesyncWorkerMain() also acquires
the lock in exclusive mode even though the tablesync for a rel is in
progress which I guess could easily heart us for larger values of
max_logical_replication_workers. So, that could be another motivation
to think for a different locking scheme.

There are couple of ways in which this issue can be solved:
Approach #1) check that the reuse worker has not picked up this table
for table sync from logicalrep_worker_launch while holding a lock on
LogicalRepWorkerLock, if the reuse worker has already picked it up for
processing, simply ignore it and return, nothing has to be done by the
launcher in this case.
Approach #2) a) Applyworker to create a shared memory of all the
relations that need to be synced, b) tablesync worker to take a lock
on this shared memory and pick the next table to be
processed(tablesync worker need not get the subscription relations
again and again) c) tablesync worker to update the status in shared
memory for the relation(since the lock is held there will be no
concurrency issues), also mark the start time in the shared memory,
this will help in not to restart the failed table before
wal_retrieve_retry_interval has expired d) tablesync worker to sync
the table e) subscription relation will be marked as ready and the
tablesync worker to remove the entry from shared memory f) Applyworker
will periodically synchronize the shared memory relations to keep it
in sync with the fetched subscription relation tables g) when apply
worker exits, the shared memory will be cleared.

Approach #2) will also help in solving the other issue reported by Amit at [1]/messages/by-id/CAA4eK1KyHfVOVeio28p8CHDnuyKuej78cj_7U9mHAB4ictVQwQ@mail.gmail.com.
I felt we can use Approach #2 to solve the problem as it solves both
the reported issues and also there is an added advantage where the
re-use table sync worker need not scan the pg_subscription_rel to get
the non-ready table for every run, instead we can use the list
prepared by apply worker.
Thoughts?

[1]: /messages/by-id/CAA4eK1KyHfVOVeio28p8CHDnuyKuej78cj_7U9mHAB4ictVQwQ@mail.gmail.com

Regards,
Vignesh

#160Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#145)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Here is another review comment about patch v26-0001.

The tablesync worker processes include the 'relid' as part of their
name. See launcher.c:

snprintf(bgw.bgw_name, BGW_MAXLEN,
"logical replication tablesync worker for subscription %u sync %u",
subid,
relid);

~~

And if that worker is "reused" by v26-0001 to process another relation
there is a LOG

if (reuse_worker)
ereport(LOG,
errmsg("logical replication table synchronization worker for
subscription \"%s\" will be reused to sync table \"%s\" with relid
%u.",
MySubscription->name,
get_rel_name(MyLogicalRepWorker->relid),
MyLogicalRepWorker->relid));

AFAICT, when being "reused" the original process name remains
unchanged, and so I think it will continue to appear to any user
looking at it that the tablesync process is just taking a very long
time handling the original 'relid'.

Won't the stale process name cause confusion to the users?

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#161Peter Smith
smithpb2250@gmail.com
In reply to: Melih Mutlu (#156)
4 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Fri, Aug 11, 2023 at 11:45 PM Melih Mutlu <m.melihmutlu@gmail.com> wrote:

Again, I couldn't reproduce the cases where you saw significantly degraded performance. I wonder if I'm missing something. Did you do anything not included in the test scripts you shared? Do you think v26-0001 will perform 84% worse than HEAD, if you try again? I just want to be sure that it was not a random thing.
Interestingly, I also don't see an improvement in above results as big as in your results when inserts/tx ratio is smaller. Even though it certainly is improved in such cases.

TEST ENVIRONMENTS

I am running the tests on a high-spec machine:

-- NOTE: Nobody else is using this machine during our testing, so
there are no unexpected influences messing up the results.

Linix

Architecture: x86_64
CPU(s): 120
Thread(s) per core: 2
Core(s) per socket: 15

total used free shared buff/cache available
Mem: 755G 5.7G 737G 49M 12G 748G
Swap: 4.0G 0B 4.0G

~~~

The results I am seeing are not random. HEAD+v26-0001 is consistently
worse than HEAD but only for some settings. With these settings, I see
bad results (i.e. worse than HEAD) consistently every time using the
dedicated test machine.

Hou-san also reproduced bad results using a different high-spec machine

Vignesh also reproduced bad results using just his laptop but in his
case, it did *not* occur every time. As discussed elsewhere the
problem is timing-related, so sometimes you may be lucky and sometimes
not.

~

I expect you are running everything correctly, but if you are using
just a laptop (like Vignesh) then like him you might need to try
multiple times before you can hit the problem happening in your
environment.

Anyway, in case there is some other reason you are not seeing the bad
results I have re-attached scripts and re-described every step below.

======

BUILDING

-- NOTE: I have a very minimal configuration without any
optimization/debug flags etc. See config.log

$ ./configure --prefix=/home/peter/pg_oss

-- NOTE: Of course, make sure to be running using the correct Postgres:

echo 'set environment variables for OSS work'
export PATH=/home/peter/pg_oss/bin:$PATH

-- NOTE: Be sure to do git stash or whatever so don't accidentally
build a patched version thinking it is the HEAD version
-- NOTE: Be sure to do a full clean build and apply (or don't apply
v26-0001) according to the test you wish to run.

STEPS
1. sudo make clean
2. make
3. sudo make install

======

SCRIPTS & STEPS

SCRIPTS
testrun.sh
do_one_test_setup.sh
do_one_test_PUB.sh
do_one_test_SUB.sh

---

STEPS

Step-1. Edit the testrun.sh

tables=( 100 )
workers=( 2 4 8 16 )
size="0"
prefix="0816headbusy" <-- edit to differentiate each test run

~

Step-2. Edit the do_one_test_PUB.sh
IF commit_counter = 1000 THEN <-- edit this if needed. I wanted 1000
inserts/tx so nothing to do

~

Step-3: Check nothing else is running. If yes, then clean it up
[peter@localhost testing_busy]$ ps -eaf | grep postgres
peter 111924 100103 0 19:31 pts/0 00:00:00 grep --color=auto postgres

~

Step-4: Run the tests
[peter@localhost testing_busy]$ ./testrun.sh
num_tables=100, size=0, num_workers=2, run #1 <-- check the echo
matched the config you set in the Setp-1
waiting for server to shut down.... done
server stopped
waiting for server to shut down.... done
server stopped
num_tables=100, size=0, num_workers=2, run #2
waiting for server to shut down.... done
server stopped
waiting for server to shut down.... done
server stopped
num_tables=100, size=0, num_workers=2, run #3
...

~

Step-5: Sanity check
When the test completes the current folder will be full of .log and .dat* files.
Check for sanity that no errors happened

[peter@localhost testing_busy]$ cat *.log | grep ERROR
[peter@localhost testing_busy]$

~

Step-6: Collect the results
The results are output (by the do_one_test_SUB.sh) into the *.dat_SUB files
Use grep to extract them

[peter@localhost testing_busy]$ cat 0816headbusy_100t_0_2w_*.dat_SUB |
grep RESULT | grep -v duration | awk '{print $3}'
11742.019
12157.355
11773.807
11582.981
12220.962
12546.325
12210.713
12614.892
12015.489
13527.05

Repeat grep for other files:
$ cat 0816headbusy_100t_0_4w_*.dat_SUB | grep RESULT | grep -v
duration | awk '{print $3}'
$ cat 0816headbusy_100t_0_8w_*.dat_SUB | grep RESULT | grep -v
duration | awk '{print $3}'
$ cat 0816headbusy_100t_0_16w_*.dat_SUB | grep RESULT | grep -v
duration | awk '{print $3}'

~

Step-7: Summarise the results
Now I just cut/paste the results from Step-6 into a spreadsheet and
report the median of the runs.

For example, for the above HEAD run, it was:
2w 4w 8w 16w
1 11742 5996 1919 1582
2 12157 5960 1871 1469
3 11774 5926 2101 1571
4 11583 6155 1883 1671
5 12221 6310 1895 1707
6 12546 6166 1900 1470
7 12211 6114 2477 1587
8 12615 6173 2610 1715
9 12015 5869 2110 1673
10 13527 5913 2144 1227
Median 12184 6055 2010 1584

~

Step-8: REPEAT
-- repeat all above for different size transactions (editing do_one_test_PUB.sh)
-- repeat all above after rebuilding again with HEAD+v26-0001

------
Kind Regards,
Peter Smith.
Fujitsu Australia

Attachments:

do_one_test_SUB.shtext/x-sh; charset=US-ASCII; name=do_one_test_SUB.shDownload
testrun.shtext/x-sh; charset=US-ASCII; name=testrun.shDownload
do_one_test_setup.shtext/x-sh; charset=US-ASCII; name=do_one_test_setup.shDownload
do_one_test_PUB.shtext/x-sh; charset=US-ASCII; name=do_one_test_PUB.shDownload
#162Peter Smith
smithpb2250@gmail.com
In reply to: Peter Smith (#161)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi Melih,

Last week we revisited your implementation of design#2. Vignesh rebased it,
and then made a few other changes.

PSA v28*

The patch changes include:
* changed the logic slightly by setting recv_immediately(new variable), if
this variable is set the main apply worker loop will not wait in this case.
* setting the relation state to ready immediately if there are no more
incremental changes to be synced.
* receive the incremental changes if applicable and set the relation state
to ready without waiting.
* reuse the worker if the worker is free before trying to start a new table
sync worker
* restarting the tablesync worker only after wal_retrieve_retry_interval

~

FWIW, we just wanted to share with you the performance measurements seen
using this design#2 patch set:

======

RESULTS (not busy tests)

------
10 empty tables
2w 4w 8w 16w
HEAD: 125 119 140 133
HEAD+v28*: 92 93 123 134
%improvement: 27% 22% 12% -1%
------
100 empty tables
2w 4w 8w 16w
HEAD: 1037 843 1109 1155
HEAD+v28*: 591 625 2616 2569
%improvement: 43% 26% -136% -122%
------
1000 empty tables
2w 4w 8w 16w
HEAD: 15874 10047 9919 10338
HEAD+v28*: 33673 12199 9094 9896
%improvement: -112% -21% 8% 4%
------
2000 empty tables
2w 4w 8w 16w
HEAD: 45266 24216 19395 19820
HEAD+v28*: 88043 21550 21668 22607
%improvement: -95% 11% -12% -14%

~~~

Note - the results were varying quite a lot in comparison to the HEAD
e.g. HEAD results are very consistent, but the v28* results observed are not
HEAD 1000 (2w): 15861, 15777, 16007, 15950, 15886, 15740, 15846, 15740,
15908, 15940
v28* 1000 (2w): 34214, 13679, 8792, 33289, 31976, 56071, 57042, 56163,
34058, 11969

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#163Peter Smith
smithpb2250@gmail.com
In reply to: Peter Smith (#162)
4 attachment(s)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Oops - now with attachments

On Mon, Aug 21, 2023 at 5:56 PM Peter Smith <smithpb2250@gmail.com> wrote:

Show quoted text

Hi Melih,

Last week we revisited your implementation of design#2. Vignesh rebased
it, and then made a few other changes.

PSA v28*

The patch changes include:
* changed the logic slightly by setting recv_immediately(new variable), if
this variable is set the main apply worker loop will not wait in this case.
* setting the relation state to ready immediately if there are no more
incremental changes to be synced.
* receive the incremental changes if applicable and set the relation state
to ready without waiting.
* reuse the worker if the worker is free before trying to start a new
table sync worker
* restarting the tablesync worker only after wal_retrieve_retry_interval

~

FWIW, we just wanted to share with you the performance measurements seen
using this design#2 patch set:

======

RESULTS (not busy tests)

------
10 empty tables
2w 4w 8w 16w
HEAD: 125 119 140 133
HEAD+v28*: 92 93 123 134
%improvement: 27% 22% 12% -1%
------
100 empty tables
2w 4w 8w 16w
HEAD: 1037 843 1109 1155
HEAD+v28*: 591 625 2616 2569
%improvement: 43% 26% -136% -122%
------
1000 empty tables
2w 4w 8w 16w
HEAD: 15874 10047 9919 10338
HEAD+v28*: 33673 12199 9094 9896
%improvement: -112% -21% 8% 4%
------
2000 empty tables
2w 4w 8w 16w
HEAD: 45266 24216 19395 19820
HEAD+v28*: 88043 21550 21668 22607
%improvement: -95% 11% -12% -14%

~~~

Note - the results were varying quite a lot in comparison to the HEAD
e.g. HEAD results are very consistent, but the v28* results observed are
not
HEAD 1000 (2w): 15861, 15777, 16007, 15950, 15886, 15740, 15846, 15740,
15908, 15940
v28* 1000 (2w): 34214, 13679, 8792, 33289, 31976, 56071, 57042, 56163,
34058, 11969

------
Kind Regards,
Peter Smith.
Fujitsu Australia

Attachments:

v28-0001-Reuse-Tablesync-Workers.patchapplication/octet-stream; name=v28-0001-Reuse-Tablesync-Workers.patchDownload
From 11dc455c38bf4c82f9b5a602f29a032bc342b74a Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:04:46 +0300
Subject: [PATCH v28 1/4] Reuse Tablesync Workers

Before this patch, tablesync workers were capable of syncing only one
table. For each table, a new sync worker was launched and that worker would
exit when done processing the table.

Now, tablesync workers are not limited to processing only one
table. When done, they can move to processing another table in
the same subscription.

If there is a table that needs to be synced, an available tablesync
worker picks up that table and syncs it. Each tablesync worker
continues to pick new tables to sync until there are no tables left
requiring synchronization. If there was no available worker to
process the table, then a new tablesync worker will be launched,
provided the number of tablesync workers for the subscription does not
exceed max_sync_workers_per_subscription.

Discussion: http://postgr.es/m/CAGPVpCTq=rUDd4JUdaRc1XUWf4BrH2gdSNf3rtOMUGj9rPpfzQ@mail.gmail.com
---
 src/backend/replication/logical/launcher.c  |   1 +
 src/backend/replication/logical/tablesync.c | 117 ++++++++++++++++++--
 src/backend/replication/logical/worker.c    |  49 ++++++--
 src/include/replication/worker_internal.h   |   2 +
 4 files changed, 151 insertions(+), 18 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e231fa7f95..25dd06b8af 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -440,6 +440,7 @@ retry:
 	worker->stream_fileset = NULL;
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
+	worker->relsync_completed = false;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 651a775065..31d4a7db15 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -131,10 +131,12 @@ static StringInfo copybuf = NULL;
 
 /*
  * Exit routine for synchronization worker.
+ *
+ * If reuse_worker is false, at the conclusion of this function the worker
+ * process will exit.
  */
 static void
-pg_attribute_noreturn()
-finish_sync_worker(void)
+finish_sync_worker(bool reuse_worker)
 {
 	/*
 	 * Commit any outstanding transaction. This is the usual case, unless
@@ -146,21 +148,38 @@ finish_sync_worker(void)
 		pgstat_report_stat(true);
 	}
 
+	/*
+	 * Disconnect from the publisher otherwise reusing the sync worker can
+	 * error due to exceeding max_wal_senders.
+	 */
+	if (LogRepWorkerWalRcvConn != NULL)
+	{
+		walrcv_disconnect(LogRepWorkerWalRcvConn);
+		LogRepWorkerWalRcvConn = NULL;
+	}
+
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
 	StartTransactionCommand();
-	ereport(LOG,
-			(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has finished",
-					MySubscription->name,
-					get_rel_name(MyLogicalRepWorker->relid))));
+	if (reuse_worker)
+		ereport(LOG,
+				errmsg("logical replication table synchronization worker for subscription \"%s\" will be reused to sync table \"%s\" with relid %u.",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid));
+	else
+		ereport(LOG,
+				errmsg("logical replication table synchronization worker for subscription \"%s\" has finished",
+						MySubscription->name));
 	CommitTransactionCommand();
 
 	/* Find the leader apply worker and signal it. */
 	logicalrep_worker_wakeup(MyLogicalRepWorker->subid, InvalidOid);
 
 	/* Stop gracefully */
-	proc_exit(0);
+	if (!reuse_worker)
+		proc_exit(0);
 }
 
 /*
@@ -380,7 +399,15 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
 		 */
 		replorigin_drop_by_name(originname, true, false);
 
-		finish_sync_worker();
+		/* Sync worker has completed synchronization of the current table. */
+		MyLogicalRepWorker->relsync_completed = true;
+
+		ereport(LOG,
+				errmsg("logical replication table synchronization for subscription \"%s\", relation \"%s\" with relid %u has finished",
+						MySubscription->name,
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid));
+		CommitTransactionCommand();
 	}
 	else
 		SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -1285,7 +1312,7 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 		case SUBREL_STATE_SYNCDONE:
 		case SUBREL_STATE_READY:
 		case SUBREL_STATE_UNKNOWN:
-			finish_sync_worker();	/* doesn't return */
+			finish_sync_worker(false);	/* doesn't return */
 	}
 
 	/* Calculate the name of the tablesync slot. */
@@ -1643,6 +1670,8 @@ run_tablesync_worker()
 	char	   *slotname = NULL;
 	WalRcvStreamOptions options;
 
+	MyLogicalRepWorker->relsync_completed = false;
+
 	start_table_sync(&origin_startpos, &slotname);
 
 	ReplicationOriginNameForLogicalRep(MySubscription->oid,
@@ -1665,12 +1694,78 @@ void
 TablesyncWorkerMain(Datum main_arg)
 {
 	int			worker_slot = DatumGetInt32(main_arg);
+	bool 		done = false;
 
 	SetupApplyOrSyncWorker(worker_slot);
 
-	run_tablesync_worker();
+	/*
+	 * The loop where worker does its job. It loops until there is no relation
+	 * left to sync.
+	 */
+	for (;!done;)
+	{
+		run_tablesync_worker();
+
+		if (IsTransactionState())
+			CommitTransactionCommand();
+
+		if (MyLogicalRepWorker->relsync_completed)
+		{
+			List	   *rstates;
+			ListCell   *lc;
+
+			/*
+			 * This tablesync worker is 'done' unless another table that needs
+			 * syncing is found.
+			 */
+			done = true;
+
+			/* This transaction will be committed by finish_sync_worker. */
+			StartTransactionCommand();
+
+			/*
+			 * Check if there is any table whose relation state is still INIT.
+			 * If a table in INIT state is found, the worker will not be
+			 * finished, it will be reused instead.
+			 */
+			rstates = GetSubscriptionRelations(MySubscription->oid, true);
+
+			foreach(lc, rstates)
+			{
+				SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
+
+				if (rstate->state == SUBREL_STATE_SYNCDONE)
+					continue;
+
+				/*
+				 * Take exclusive lock to prevent any other sync worker from
+				 * picking the same table.
+				 */
+				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
+
+				/*
+				 * Pick the table for the next run if it is not already picked
+				 * up by another worker.
+				 */
+				if (!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
+				{
+					/* Update worker state for the next table */
+					MyLogicalRepWorker->relid = rstate->relid;
+					MyLogicalRepWorker->relstate = rstate->state;
+					MyLogicalRepWorker->relstate_lsn = rstate->lsn;
+					LWLockRelease(LogicalRepWorkerLock);
+
+					/* Found a table for next iteration */
+					finish_sync_worker(true);
+					done = false;
+					break;
+				}
+				LWLockRelease(LogicalRepWorkerLock);
+			}
+		}
+	}
 
-	finish_sync_worker();
+	finish_sync_worker(false);
 }
 
 /*
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index a20d4c1171..64e965d912 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3621,12 +3621,30 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 					MemoryContextReset(ApplyMessageContext);
 				}
 
+				if (am_tablesync_worker())
+				{
+					/*
+					 * apply_dispatch() may have gone into apply_handle_commit()
+					 * which can call process_syncing_tables_for_sync.
+					 *
+					 * process_syncing_tables_for_sync decides whether the sync of
+					 * the current table is completed. If it is completed,
+					 * streaming must be already ended. So, we can break the loop.
+					 */
+					if (MyLogicalRepWorker->relsync_completed)
+					{
+						endofstream = true;
+						break;
+					}
+				}
+
 				len = walrcv_receive(LogRepWorkerWalRcvConn, &buf, &fd);
 			}
 		}
 
 		/* confirm all writes so far */
-		send_feedback(last_received, false, false);
+		if (!MyLogicalRepWorker->relsync_completed)
+			send_feedback(last_received, false, false);
 
 		if (!in_remote_transaction && !in_streamed_transaction)
 		{
@@ -3640,6 +3658,18 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 
 			/* Process any table synchronization changes. */
 			process_syncing_tables(last_received);
+
+			if (am_tablesync_worker())
+			{
+				/*
+				 * If relsync_completed is true, this means that the tablesync
+				 * worker is done with synchronization. Streaming has already been
+				 * ended by process_syncing_tables_for_sync. We should move to the
+				 * next table if needed, or exit.
+				 */
+				if (MyLogicalRepWorker->relsync_completed)
+					endofstream = true;
+			}
 		}
 
 		/* Cleanup the memory. */
@@ -3742,8 +3772,12 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	error_context_stack = errcallback.previous;
 	apply_error_context_stack = error_context_stack;
 
-	/* All done */
-	walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
+	/*
+	 * End streaming here for only apply workers. Ending streaming for
+	 * tablesync workers is deferred until the worker exits its main loop.
+	 */
+	if (!am_tablesync_worker())
+		walrcv_endstreaming(LogRepWorkerWalRcvConn, &tli);
 }
 
 /*
@@ -4619,13 +4653,14 @@ InitializeLogRepWorker(void)
 
 	if (am_tablesync_worker())
 		ereport(LOG,
-				(errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" has started",
+				errmsg("logical replication table synchronization worker for subscription \"%s\", table \"%s\" with relid %u has started",
 						MySubscription->name,
-						get_rel_name(MyLogicalRepWorker->relid))));
+						get_rel_name(MyLogicalRepWorker->relid),
+						MyLogicalRepWorker->relid));
 	else
 		ereport(LOG,
-				(errmsg("logical replication apply worker for subscription \"%s\" has started",
-						MySubscription->name)));
+				errmsg("logical replication apply worker for subscription \"%s\" has started",
+						MySubscription->name));
 
 	CommitTransactionCommand();
 }
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 672a7117c0..10bea1d533 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -56,6 +56,8 @@ typedef struct LogicalRepWorker
 	char		relstate;
 	XLogRecPtr	relstate_lsn;
 	slock_t		relmutex;
+	bool		relsync_completed; /* has tablesync finished syncing
+									* the assigned table? */
 
 	/*
 	 * Used to create the changes and subxact files for the streaming
-- 
2.34.1

v28-0002-Reuse-connection-when-tablesync-workers-change-t.patchapplication/octet-stream; name=v28-0002-Reuse-connection-when-tablesync-workers-change-t.patchDownload
From 7c3540cd3296018db5fef522379854861e38a21f Mon Sep 17 00:00:00 2001
From: Melih Mutlu <m.melihmutlu@gmail.com>
Date: Tue, 4 Jul 2023 22:13:52 +0300
Subject: [PATCH v28 2/4] Reuse connection when tablesync workers change the
 target

Previously tablesync workers establish new connections when it changes the syncing
table, but this might have additional overhead. This patch allows the existing
connection to be reused.

As for the publisher node, this patch allows to reuse logical walsender processes
after the streaming is done once.
---
 src/backend/replication/logical/launcher.c  |  1 +
 src/backend/replication/logical/tablesync.c | 60 ++++++++++++++-------
 src/backend/replication/logical/worker.c    | 18 ++++---
 src/backend/replication/walsender.c         |  6 +++
 src/include/replication/worker_internal.h   |  3 ++
 5 files changed, 61 insertions(+), 27 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 25dd06b8af..b19437f9d0 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
 	worker->relsync_completed = false;
+	worker->slot_number = slot;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
 	TIMESTAMP_NOBEGIN(worker->last_recv_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 31d4a7db15..ec2f67d879 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -148,16 +148,6 @@ finish_sync_worker(bool reuse_worker)
 		pgstat_report_stat(true);
 	}
 
-	/*
-	 * Disconnect from the publisher otherwise reusing the sync worker can
-	 * error due to exceeding max_wal_senders.
-	 */
-	if (LogRepWorkerWalRcvConn != NULL)
-	{
-		walrcv_disconnect(LogRepWorkerWalRcvConn);
-		LogRepWorkerWalRcvConn = NULL;
-	}
-
 	/* And flush all writes. */
 	XLogFlush(GetXLogWriteRecPtr());
 
@@ -1261,6 +1251,27 @@ ReplicationSlotNameForTablesync(Oid suboid, Oid relid,
 			 relid, GetSystemIdentifier());
 }
 
+/*
+ * Determine the application_name for tablesync workers.
+ *
+ * Previously, the replication slot name was used as application_name. Since
+ * it's possible to reuse tablesync workers now, a tablesync worker can handle
+ * several different replication slots during its lifetime. Therefore, we
+ * cannot use the slot name as application_name anymore. Instead, the slot
+ * number of the tablesync worker is used as a part of the application_name.
+ *
+ * XXX: if the tablesync worker starts to reuse the replication slot during
+ * synchronization, we should again use the replication slot name as
+ * application_name.
+ */
+static void
+ApplicationNameForTablesync(Oid suboid, int worker_slot,
+							char *application_name, Size szapp)
+{
+	snprintf(application_name, szapp, "pg_%u_sync_%i_" UINT64_FORMAT, suboid,
+			 worker_slot, GetSystemIdentifier());
+}
+
 /*
  * Start syncing the table in the sync worker.
  *
@@ -1322,15 +1333,26 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
 									slotname,
 									NAMEDATALEN);
 
-	/*
-	 * Here we use the slot name instead of the subscription name as the
-	 * application_name, so that it is different from the leader apply worker,
-	 * so that synchronous replication can distinguish them.
-	 */
-	LogRepWorkerWalRcvConn =
-		walrcv_connect(MySubscription->conninfo, true,
-					   must_use_password,
-					   slotname, &err);
+	/* Connect to the publisher if haven't done so already. */
+	if (LogRepWorkerWalRcvConn == NULL)
+	{
+		char application_name[NAMEDATALEN];
+
+		/*
+		 * The application_name must differ from the subscription name (used by
+		 * the leader apply worker) because synchronous replication has to be
+		 * able to distinguish this worker from the leader apply worker.
+		 */
+		ApplicationNameForTablesync(MySubscription->oid,
+									MyLogicalRepWorker->slot_number,
+									application_name,
+									NAMEDATALEN);
+		LogRepWorkerWalRcvConn =
+			walrcv_connect(MySubscription->conninfo, true,
+						   must_use_password,
+						   application_name, &err);
+	}
+
 	if (LogRepWorkerWalRcvConn == NULL)
 		ereport(ERROR,
 				(errcode(ERRCODE_CONNECTION_FAILURE),
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 64e965d912..d212f9d543 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3494,20 +3494,22 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 	ErrorContextCallback errcallback;
 
 	/*
-	 * Init the ApplyMessageContext which we clean up after each replication
-	 * protocol message.
+	 * Init the ApplyMessageContext if needed. This context is cleaned up
+	 * after each replication protocol message.
 	 */
-	ApplyMessageContext = AllocSetContextCreate(ApplyContext,
-												"ApplyMessageContext",
-												ALLOCSET_DEFAULT_SIZES);
+	if (!ApplyMessageContext)
+		ApplyMessageContext = AllocSetContextCreate(ApplyContext,
+													"ApplyMessageContext",
+													ALLOCSET_DEFAULT_SIZES);
 
 	/*
 	 * This memory context is used for per-stream data when the streaming mode
 	 * is enabled. This context is reset on each stream stop.
 	 */
-	LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
-													"LogicalStreamingContext",
-													ALLOCSET_DEFAULT_SIZES);
+	if (!LogicalStreamingContext)
+		LogicalStreamingContext = AllocSetContextCreate(ApplyContext,
+														"LogicalStreamingContext",
+														ALLOCSET_DEFAULT_SIZES);
 
 	/* mark as idle, before starting to loop */
 	pgstat_report_activity(STATE_IDLE, NULL);
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index d27ef2985d..1d2a87cdcd 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1808,6 +1808,12 @@ exec_replication_command(const char *cmd_string)
 		case T_CreateReplicationSlotCmd:
 			cmdtag = "CREATE_REPLICATION_SLOT";
 			set_ps_display(cmdtag);
+
+			/*
+			 * Reset flags because reusing tablesync workers can mean
+			 * this is the second time here.
+			 */
+			streamingDoneSending = streamingDoneReceiving = false;
 			CreateReplicationSlot((CreateReplicationSlotCmd *) cmd_node);
 			EndReplicationCommand(cmdtag);
 			break;
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 10bea1d533..47fa7fbd55 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -39,6 +39,9 @@ typedef struct LogicalRepWorker
 	/* Increased every time the slot is taken by new worker. */
 	uint16		generation;
 
+	/* Slot number of this worker. */
+	int			slot_number;
+
 	/* Pointer to proc array. NULL if not running. */
 	PGPROC	   *proc;
 
-- 
2.34.1

v28-0004-Defect-fixes.patchapplication/octet-stream; name=v28-0004-Defect-fixes.patchDownload
From b4419ac9ea4bed647b0bc78eae537810364dee15 Mon Sep 17 00:00:00 2001
From: Vignesh C <vignesh21@gmail.com>
Date: Thu, 17 Aug 2023 20:09:00 +0530
Subject: [PATCH v28 4/4] Defect fixes

The following defects are fixed:
1) Setting table to ready immediately if there are no more incremental
   changes to be synced.
2) Recieve the incremental changes if applicable and setting relation
   state to ready without waiting.
3) Reuse the worker if the worker is free before trying to start a new
   table sync worker
---
 src/backend/replication/logical/launcher.c  |   1 +
 src/backend/replication/logical/tablesync.c | 113 ++++++++++++++------
 src/backend/replication/logical/worker.c    |   6 ++
 src/include/replication/worker_internal.h   |   3 +
 4 files changed, 91 insertions(+), 32 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 27fea8cd6f..7b208d22df 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -473,6 +473,7 @@ retry:
 	worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
 	worker->parallel_apply = is_parallel_apply_worker;
 	worker->relsync_completed = false;
+	worker->recv_immediately = false;
 	worker->slot_number = slot;
 	worker->last_lsn = InvalidXLogRecPtr;
 	TIMESTAMP_NOBEGIN(worker->last_send_time);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index c65f2a64d9..7d45178e2b 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -183,7 +183,8 @@ finish_sync_worker(bool reuse_worker)
  * CATCHUP state to SYNCDONE.
  */
 static bool
-wait_for_relation_state_change(Oid relid, char expected_state)
+wait_for_relation_state_change(Oid relid, char expected_state,
+							   bool check_equal, long timeout)
 {
 	char		state;
 
@@ -201,7 +202,10 @@ wait_for_relation_state_change(Oid relid, char expected_state)
 		if (state == SUBREL_STATE_UNKNOWN)
 			break;
 
-		if (state == expected_state)
+		if (check_equal && state == expected_state)
+			return true;
+
+		if (!check_equal && state != expected_state)
 			return true;
 
 		/* Check if the sync worker is still running and bail if not. */
@@ -214,7 +218,7 @@ wait_for_relation_state_change(Oid relid, char expected_state)
 
 		(void) WaitLatch(MyLatch,
 						 WL_LATCH_SET | WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
-						 1000L, WAIT_EVENT_LOGICAL_SYNC_STATE_CHANGE);
+						 timeout, WAIT_EVENT_LOGICAL_SYNC_STATE_CHANGE);
 
 		ResetLatch(MyLatch);
 	}
@@ -595,13 +599,62 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 					}
 
 					wait_for_relation_state_change(rstate->relid,
-												   SUBREL_STATE_SYNCDONE);
+												   SUBREL_STATE_SYNCDONE, true, 1000L);
+
+					if (current_lsn >= rstate->lsn)
+					{
+						char		originname[NAMEDATALEN];
+
+						rstate->state = SUBREL_STATE_READY;
+						rstate->lsn = current_lsn;
+						if (!started_tx)
+						{
+							StartTransactionCommand();
+							started_tx = true;
+						}
+
+						/*
+						 * Remove the tablesync origin tracking if exists.
+						 *
+						 * There is a chance that the user is concurrently performing
+						 * refresh for the subscription where we remove the table
+						 * state and its origin or the tablesync worker would have
+						 * already removed this origin. We can't rely on tablesync
+						 * worker to remove the origin tracking as if there is any
+						 * error while dropping we won't restart it to drop the
+						 * origin. So passing missing_ok = true.
+						 */
+						ReplicationOriginNameForLogicalRep(MyLogicalRepWorker->subid,
+														rstate->relid,
+														originname,
+														sizeof(originname));
+						replorigin_drop_by_name(originname, true, false);
+
+						/*
+						 * Update the state to READY only after the origin cleanup.
+						 */
+						UpdateSubscriptionRelState(MyLogicalRepWorker->subid,
+												rstate->relid, rstate->state,
+												rstate->lsn);
+					}
+					else
+					{
+						/*
+						 * There are some transaction data that need to be
+						 * synced for the table to reach READY state.
+						 */
+						MyLogicalRepWorker->recv_immediately = true;
+					}
 				}
 				else
 					LWLockRelease(LogicalRepWorkerLock);
 			}
 			else
 			{
+				TimestampTz now = GetCurrentTimestamp();
+				struct tablesync_start_time_mapping *hentry;
+				bool		found;
+
 				/*
 				 * If there is no sync worker for this table yet, count
 				 * running sync workers for this subscription, while we have
@@ -613,33 +666,11 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 				/* Now safe to release the LWLock */
 				LWLockRelease(LogicalRepWorkerLock);
 
-				/*
-				 * If there are free sync worker slot(s), start a new sync
-				 * worker for the table.
-				 */
-				if (nsyncworkers < max_sync_workers_per_subscription)
-				{
-					TimestampTz now = GetCurrentTimestamp();
-					struct tablesync_start_time_mapping *hentry;
-					bool		found;
-
-					hentry = hash_search(last_start_times, &rstate->relid,
-										 HASH_ENTER, &found);
-
-					if (!found ||
-						TimestampDifferenceExceeds(hentry->last_start_time, now,
-												   wal_retrieve_retry_interval))
-					{
-						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
-												 MySubscription->oid,
-												 MySubscription->name,
-												 MyLogicalRepWorker->userid,
-												 rstate->relid,
-												 DSM_HANDLE_INVALID);
-						hentry->last_start_time = now;
-					}
-				}
-				else
+				hentry = hash_search(last_start_times, &rstate->relid,
+										HASH_ENTER, &found);
+				if (!found ||
+					TimestampDifferenceExceeds(hentry->last_start_time, now,
+												wal_retrieve_retry_interval))
 				{
 					/*
 					 * We reached the max_sync_workers_per_subscription limit.
@@ -661,9 +692,27 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						{
 							logicalrep_worker_wakeup_ptr(syncworker);
 						}
+
+						LWLockRelease(LogicalRepWorkerLock);
+
+						wait_for_relation_state_change(rstate->relid,
+													   SUBREL_STATE_INIT, false, 10L);
+						hentry->last_start_time = now;
 					}
+					else if (nsyncworkers < max_sync_workers_per_subscription)
+					{
+						LWLockRelease(LogicalRepWorkerLock);
 
-					LWLockRelease(LogicalRepWorkerLock);
+						logicalrep_worker_launch(MyLogicalRepWorker->dbid,
+												MySubscription->oid,
+												MySubscription->name,
+												MyLogicalRepWorker->userid,
+												rstate->relid,
+												DSM_HANDLE_INVALID);
+						hentry->last_start_time = now;
+					}
+					else
+						LWLockRelease(LogicalRepWorkerLock);
 				}
 			}
 		}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index d212f9d543..e3be91e5a4 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3682,6 +3682,12 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
 		if (endofstream)
 			break;
 
+		if (MyLogicalRepWorker->recv_immediately)
+		{
+			MyLogicalRepWorker->recv_immediately = false;
+			continue;
+		}
+
 		/*
 		 * Wait for more data or latch.  If we have unflushed transactions,
 		 * wake up after WalWriterDelay to see if they've been flushed yet (in
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 0cf2a69fd6..a8d498f5af 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -54,6 +54,9 @@ typedef struct LogicalRepWorker
 	/* Subscription id for the worker. */
 	Oid			subid;
 
+	/* Receive the incremental changes immediately without waiting. */
+	bool		recv_immediately;
+
 	/* Used for initial table synchronization. */
 	Oid			relid;
 	char		relstate;
-- 
2.34.1

v28-0003-apply-worker-assigns-tables.patchapplication/octet-stream; name=v28-0003-apply-worker-assigns-tables.patchDownload
From c3989215cd3c1d1efa790e96ad9fa537a6b8da4e Mon Sep 17 00:00:00 2001
From: Vignesh C <vignesh21@gmail.com>
Date: Thu, 17 Aug 2023 14:09:17 +0530
Subject: [PATCH v28 3/4] apply worker assigns tables

apply worker assigns tables
---
 src/backend/replication/logical/launcher.c  |  32 ++++++
 src/backend/replication/logical/tablesync.c | 110 +++++++++++---------
 src/include/replication/worker_internal.h   |   1 +
 3 files changed, 93 insertions(+), 50 deletions(-)

diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index b19437f9d0..27fea8cd6f 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -297,6 +297,38 @@ logicalrep_workers_find(Oid subid, bool only_running)
 	return res;
 }
 
+/*
+ * Return a logical rep worker in ready state
+ */
+LogicalRepWorker *
+logicalrep_worker_find_syncdone(Oid subid, bool only_running)
+{
+	int			i;
+	LogicalRepWorker *res = NULL;
+
+	Assert(LWLockHeldByMe(LogicalRepWorkerLock));
+
+	/* Search for attached worker for a given subscription id. */
+	for (i = 0; i < max_logical_replication_workers; i++)
+	{
+		LogicalRepWorker *w = &LogicalRepCtx->workers[i];
+
+		/* Skip parallel apply workers. */
+		if (isParallelApplyWorker(w))
+			continue;
+
+		if (w->in_use && w->subid == subid &&
+			w->relstate == SUBREL_STATE_SYNCDONE &&
+			(!only_running || w->proc))
+		{
+			res = w;
+			break;
+		}
+	}
+
+	return res;
+}
+
 /*
  * Start new logical replication background worker, if possible.
  *
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index ec2f67d879..c65f2a64d9 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -241,6 +241,12 @@ wait_for_worker_state_change(char expected_state)
 
 		CHECK_FOR_INTERRUPTS();
 
+		/* No table needs sync anymore. Apply worker wants this sync worker to exit. */
+		if (!OidIsValid(MyLogicalRepWorker->relid))
+		{
+			return false;
+		}
+
 		/*
 		 * Done if already in correct state.  (We assume this fetch is atomic
 		 * enough to not give a misleading answer if we do it with no lock.)
@@ -463,8 +469,28 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 	 */
 	else if (table_states_not_ready == NIL && last_start_times)
 	{
+		List       *workers;
+
 		hash_destroy(last_start_times);
 		last_start_times = NULL;
+
+		/* Let all sync workers exit */
+		LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+		workers = logicalrep_workers_find(MyLogicalRepWorker->subid, false);
+		foreach(lc, workers)
+		{
+			LogicalRepWorker *w = (LogicalRepWorker *) lfirst(lc);
+
+			if (OidIsValid(w->relid))
+			{
+					SpinLockAcquire(&w->relmutex);
+					w->relid = InvalidOid;
+					SpinLockRelease(&w->relmutex);
+					logicalrep_worker_wakeup_ptr(w);
+			}
+		}
+
+		LWLockRelease(LogicalRepWorkerLock);
 	}
 
 	/*
@@ -613,6 +639,32 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
 						hentry->last_start_time = now;
 					}
 				}
+				else
+				{
+					/*
+					 * We reached the max_sync_workers_per_subscription limit.
+					 * Check if there is an existing sync worker waiting for
+					 * new table to sync.
+					 */
+					LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+
+					syncworker = logicalrep_worker_find_syncdone(MyLogicalRepWorker->subid, false);
+					if (syncworker)
+					{
+						SpinLockAcquire(&syncworker->relmutex);
+						syncworker->relid = rstate->relid;
+						syncworker->relstate = rstate->state;
+						syncworker->relstate_lsn = rstate->lsn;
+						SpinLockRelease(&syncworker->relmutex);
+
+						if (syncworker->proc)
+						{
+							logicalrep_worker_wakeup_ptr(syncworker);
+						}
+					}
+
+					LWLockRelease(LogicalRepWorkerLock);
+				}
 			}
 		}
 	}
@@ -1731,59 +1783,17 @@ TablesyncWorkerMain(Datum main_arg)
 		if (IsTransactionState())
 			CommitTransactionCommand();
 
+		finish_sync_worker(true);
+
 		if (MyLogicalRepWorker->relsync_completed)
 		{
-			List	   *rstates;
-			ListCell   *lc;
-
-			/*
-			 * This tablesync worker is 'done' unless another table that needs
-			 * syncing is found.
-			 */
-			done = true;
-
-			/* This transaction will be committed by finish_sync_worker. */
-			StartTransactionCommand();
-
-			/*
-			 * Check if there is any table whose relation state is still INIT.
-			 * If a table in INIT state is found, the worker will not be
-			 * finished, it will be reused instead.
-			 */
-			rstates = GetSubscriptionRelations(MySubscription->oid, true);
-
-			foreach(lc, rstates)
-			{
-				SubscriptionRelState *rstate = (SubscriptionRelState *) lfirst(lc);
-
-				if (rstate->state == SUBREL_STATE_SYNCDONE)
-					continue;
-
-				/*
-				 * Take exclusive lock to prevent any other sync worker from
-				 * picking the same table.
-				 */
-				LWLockAcquire(LogicalRepWorkerLock, LW_EXCLUSIVE);
-
-				/*
-				 * Pick the table for the next run if it is not already picked
-				 * up by another worker.
-				 */
-				if (!logicalrep_worker_find(MySubscription->oid, rstate->relid, false))
-				{
-					/* Update worker state for the next table */
-					MyLogicalRepWorker->relid = rstate->relid;
-					MyLogicalRepWorker->relstate = rstate->state;
-					MyLogicalRepWorker->relstate_lsn = rstate->lsn;
-					LWLockRelease(LogicalRepWorkerLock);
+			/* wait for apply worker to assign a new table with INIT state. */
+			wait_for_worker_state_change(SUBREL_STATE_INIT);
+		}
 
-					/* Found a table for next iteration */
-					finish_sync_worker(true);
-					done = false;
-					break;
-				}
-				LWLockRelease(LogicalRepWorkerLock);
-			}
+		if (!OidIsValid(MyLogicalRepWorker->relid))
+		{
+			break;
 		}
 	}
 
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 47fa7fbd55..0cf2a69fd6 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -237,6 +237,7 @@ 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 LogicalRepWorker * logicalrep_worker_find_syncdone(Oid subid, bool only_running);
 extern bool logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname,
 									 Oid userid, Oid relid,
 									 dsm_handle subworker_dsm);
-- 
2.34.1

#164Shlok Kyal
shlok.kyal.oss@gmail.com
In reply to: Peter Smith (#163)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

Hi,

This patch is not applying on the HEAD. Please rebase and share the
updated patch.

Thanks and Regards
Shlok Kyal

Show quoted text

On Wed, 10 Jan 2024 at 14:55, Peter Smith <smithpb2250@gmail.com> wrote:

Oops - now with attachments

On Mon, Aug 21, 2023 at 5:56 PM Peter Smith <smithpb2250@gmail.com> wrote:

Hi Melih,

Last week we revisited your implementation of design#2. Vignesh rebased it, and then made a few other changes.

PSA v28*

The patch changes include:
* changed the logic slightly by setting recv_immediately(new variable), if this variable is set the main apply worker loop will not wait in this case.
* setting the relation state to ready immediately if there are no more incremental changes to be synced.
* receive the incremental changes if applicable and set the relation state to ready without waiting.
* reuse the worker if the worker is free before trying to start a new table sync worker
* restarting the tablesync worker only after wal_retrieve_retry_interval

~

FWIW, we just wanted to share with you the performance measurements seen using this design#2 patch set:

======

RESULTS (not busy tests)

------
10 empty tables
2w 4w 8w 16w
HEAD: 125 119 140 133
HEAD+v28*: 92 93 123 134
%improvement: 27% 22% 12% -1%
------
100 empty tables
2w 4w 8w 16w
HEAD: 1037 843 1109 1155
HEAD+v28*: 591 625 2616 2569
%improvement: 43% 26% -136% -122%
------
1000 empty tables
2w 4w 8w 16w
HEAD: 15874 10047 9919 10338
HEAD+v28*: 33673 12199 9094 9896
%improvement: -112% -21% 8% 4%
------
2000 empty tables
2w 4w 8w 16w
HEAD: 45266 24216 19395 19820
HEAD+v28*: 88043 21550 21668 22607
%improvement: -95% 11% -12% -14%

~~~

Note - the results were varying quite a lot in comparison to the HEAD
e.g. HEAD results are very consistent, but the v28* results observed are not
HEAD 1000 (2w): 15861, 15777, 16007, 15950, 15886, 15740, 15846, 15740, 15908, 15940
v28* 1000 (2w): 34214, 13679, 8792, 33289, 31976, 56071, 57042, 56163, 34058, 11969

------
Kind Regards,
Peter Smith.
Fujitsu Australia

#165Amit Kapila
amit.kapila16@gmail.com
In reply to: Shlok Kyal (#164)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Wed, Jan 10, 2024 at 2:59 PM Shlok Kyal <shlok.kyal.oss@gmail.com> wrote:

This patch is not applying on the HEAD. Please rebase and share the
updated patch.

IIRC, there were some regressions observed with this patch. So, one
needs to analyze those as well. I think we should mark it as "Returned
with feedback".

--
With Regards,
Amit Kapila.

#166vignesh C
vignesh21@gmail.com
In reply to: Amit Kapila (#165)
Re: [PATCH] Reuse Workers and Replication Slots during Logical Replication

On Wed, 10 Jan 2024 at 15:04, Amit Kapila <amit.kapila16@gmail.com> wrote:

On Wed, Jan 10, 2024 at 2:59 PM Shlok Kyal <shlok.kyal.oss@gmail.com> wrote:

This patch is not applying on the HEAD. Please rebase and share the
updated patch.

IIRC, there were some regressions observed with this patch. So, one
needs to analyze those as well. I think we should mark it as "Returned
with feedback".

Thanks, I have updated the status to "Returned with feedback".
Feel free to post an updated version with the fix for the regression
and start a new entry for the same.

Regards,
Vignesh