From 9339ce3b0ec511f30e21613de12076b35cfb196d Mon Sep 17 00:00:00 2001
From: Shlok Kyal <shlok.kyal.oss@gmail.com>
Date: Mon, 19 May 2025 19:47:04 +0530
Subject: [PATCH v17] Restrict publishing of partitioned table with foreign
 table as its partition

Logical replication of foreign table is not supported and we throw an
error in this case. But when we create a publication on a partitioned
table that has a foreign table as its partition, the initial sync of
such table is successful and we should avoid such cases.

Current Behaviour in HEAD, when publication is created:
1. with publish_via_partition_root = true
The root table is published, and initial data from partitions that are
foreign tables is replicated.

2. with publish_via_partition_root = false and FOR ALL TABLES
All leaf tables except partitions that are foreign tables are published.

3. with publish_via_partition_root = false and
FOR TABLE/ FOR TABLES IN SCHEMA
All leaf tables are published, including initial data from partitions
that are foreign tables.

With this patch we have following behaviour:
1. with publish_via_partition_root = true
An error is thrown when we try to create subscription (with copy_data = true)
on publication with partitioned table with foreign partition.
When copy_data = false, we donot need this check as initial sync is not
performed.
Also we restrict creating/attaching a foreign table as partition of a
partitioned table which is part of any publication.

2. with publish_via_partition_root = false
We skip publishing partitions that are foreign tables. This is done by
avoid adding such partitions in pg_subscription_rel catalog table.

We have introduced two functions 'RelationHasForeignPartition' and
'check_publications_foreign_parts'. In 'RelationHasForeignPartition' we go
through the child nodes of a partition and check if it has a foreign
table. In 'check_publications_foreign_parts' we check if a subscription
with copy_data = true is subscribing to a publication which has partitioned
table with foreign table as its partition.
---
 doc/src/sgml/logical-replication.sgml     |  10 +-
 doc/src/sgml/ref/create_subscription.sgml |   7 ++
 src/backend/catalog/pg_publication.c      |  26 ++--
 src/backend/commands/subscriptioncmds.c   | 100 ++++++++++++++++
 src/backend/commands/tablecmds.c          | 110 +++++++++++++++++
 src/backend/partitioning/partdesc.c       |  32 +++++
 src/bin/pg_upgrade/check.c                |  89 ++++++++++++++
 src/include/partitioning/partdesc.h       |   1 +
 src/test/subscription/Makefile            |   1 +
 src/test/subscription/t/013_partition.pl  | 139 ++++++++++++++++++++++
 10 files changed, 499 insertions(+), 16 deletions(-)

diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index c32e6bc000d..275dd296f18 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2079,10 +2079,18 @@ CONTEXT:  processing remote data for replication origin "pg_16395" during "INSER
 
    <listitem>
     <para>
-     Replication is only supported by tables, including partitioned tables.
+     Replication is only supported for tables, including partitioned tables.
      Attempts to replicate other types of relations, such as views, materialized
      views, or foreign tables, will result in an error.
     </para>
+    <para>
+     Replication is not supported for foreign tables. When foreign tables are
+     used as partitions of partitioned tables, publishing of the partitioned
+     table is only allowed if the <literal>publish_via_partition_root</literal>
+     is set to <literal>false</literal>. In this mode, changes to a partition
+     that is a foreign table are ignored for the purposes of replication, and
+     data contained in them is not included during initial synchronization.
+    </para>
    </listitem>
 
    <listitem>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..9019fe1e384 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -545,6 +545,13 @@ WHERE C.relnamespace = N.oid AND
       PT.pubname IN (&lt;pub-names&gt;);
 </programlisting></para>
 
+   <para>
+    When using the subscription parameter copy_data = true, corresponding
+    publications are checked if it has publish_via_partition_root = true and
+    has a partitioned table with a foreign table as its partition. If this
+    scenario is detected an ERROR is logged to the user.
+   </para>
+
  </refsect1>
 
  <refsect1>
diff --git a/src/backend/catalog/pg_publication.c b/src/backend/catalog/pg_publication.c
index d6f94db5d99..56255dcd5cc 100644
--- a/src/backend/catalog/pg_publication.c
+++ b/src/backend/catalog/pg_publication.c
@@ -304,7 +304,7 @@ check_and_fetch_column_list(Publication *pub, Oid relid, MemoryContext mcxt,
 
 /*
  * Gets the relations based on the publication partition option for a specified
- * relation.
+ * relation. Foreign tables are not included.
  */
 List *
 GetPubPartitionOptionRelations(List *result, PublicationPartOpt pub_partopt,
@@ -313,25 +313,21 @@ GetPubPartitionOptionRelations(List *result, PublicationPartOpt pub_partopt,
 	if (get_rel_relkind(relid) == RELKIND_PARTITIONED_TABLE &&
 		pub_partopt != PUBLICATION_PART_ROOT)
 	{
-		List	   *all_parts = find_all_inheritors(relid, NoLock,
-													NULL);
+		List	   *all_parts = find_all_inheritors(relid, NoLock, NULL);
 
-		if (pub_partopt == PUBLICATION_PART_ALL)
-			result = list_concat(result, all_parts);
-		else if (pub_partopt == PUBLICATION_PART_LEAF)
+		foreach_oid(partOid, all_parts)
 		{
-			ListCell   *lc;
+			char		relkind = get_rel_relkind(partOid);
 
-			foreach(lc, all_parts)
-			{
-				Oid			partOid = lfirst_oid(lc);
+			if (relkind == RELKIND_FOREIGN_TABLE)
+				continue;
 
-				if (get_rel_relkind(partOid) != RELKIND_PARTITIONED_TABLE)
-					result = lappend_oid(result, partOid);
-			}
+			if (pub_partopt == PUBLICATION_PART_LEAF &&
+				relkind == RELKIND_PARTITIONED_TABLE)
+				continue;
+
+			result = lappend_oid(result, partOid);
 		}
-		else
-			Assert(false);
 	}
 	else
 		result = lappend_oid(result, relid);
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 4ff246cd943..83978b77d0c 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -112,6 +112,9 @@ static List *merge_publications(List *oldpublist, List *newpublist, bool addpub,
 static void ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err);
 static void CheckAlterSubOption(Subscription *sub, const char *option,
 								bool slot_needs_update, bool isTopLevel);
+static void check_publications_foreign_parts(WalReceiverConn *wrconn,
+											 List *publications, bool copydata,
+											 char *subname);
 
 
 /*
@@ -723,6 +726,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
 			check_publications(wrconn, publications);
 			check_publications_origin(wrconn, publications, opts.copy_data,
 									  opts.origin, NULL, 0, stmt->subname);
+			check_publications_foreign_parts(wrconn, publications,
+											 opts.copy_data, stmt->subname);
 
 			/*
 			 * Set sync state based on if we were asked to do data copy or
@@ -883,6 +888,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
 		check_publications_origin(wrconn, sub->publications, copy_data,
 								  sub->origin, subrel_local_oids,
 								  subrel_count, sub->name);
+		check_publications_foreign_parts(wrconn, sub->publications, copy_data,
+										 sub->name);
 
 		/*
 		 * Rels that we want to remove from subscription and drop any slots
@@ -2507,3 +2514,96 @@ defGetStreamingMode(DefElem *def)
 					def->defname)));
 	return LOGICALREP_STREAM_OFF;	/* keep compiler quiet */
 }
+
+/*
+ * check_publications_foreign_parts
+ * Check if the publications, on which subscriber is subscribing, publishes any
+ * partitioned table that has a foreign table as its partition and has
+ * publish_via_partition_root set as true. The check is performed only if
+ * copy_data is set as true for the subscription.
+ *
+ * Although DML changes to foreign tables are excluded from publication, the
+ * tablesync worker will still attempt to copy data from foreign table
+ * partitions during initial table synchronization.  To avoid the
+ * inconsistencies that would result, we disallow foreign tables from being
+ * published generally.  However, it's possible for partitioned tables to have
+ * foreign tables as partitions, and we would like to allow publishing those
+ * partitioned tables so that the other partitions are replicated.
+ *
+ * This function is in charge of detecting if publisher with
+ * publish_via_partition_root=true publishes a partitioned table that has a
+ * foreign table as a partition and throw an error if found.
+ *
+ * When publish_via_partition_root is false, each partition published for
+ * replication is listed individually in pg_subscription_rel, and we
+ * don't add partitions that are foreign tables, so this function is not called
+ * for such tables.
+ */
+static void
+check_publications_foreign_parts(WalReceiverConn *wrconn, List *publications,
+								 bool copydata, char *subname)
+{
+	WalRcvExecResult *res;
+	StringInfoData cmd;
+	TupleTableSlot *slot;
+	Oid			tableRow[1] = {TEXTOID};
+	List	   *publist = NIL;
+	int			i;
+
+	if (!copydata)
+		return;
+
+	initStringInfo(&cmd);
+	appendStringInfoString(&cmd,
+						   "SELECT DISTINCT P.pubname AS pubname "
+						   "FROM pg_catalog.pg_publication p, LATERAL "
+						   "pg_get_publication_tables(p.pubname) gpt, LATERAL "
+						   "pg_partition_tree(gpt.relid) gt JOIN pg_catalog.pg_foreign_table ft ON "
+						   "ft.ftrelid = gt.relid WHERE p.pubviaroot = true AND  p.pubname IN (");
+
+	GetPublicationsStr(publications, &cmd, true);
+	appendStringInfoString(&cmd, ")\n");
+
+	res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
+	pfree(cmd.data);
+
+	if (res->status != WALRCV_OK_TUPLES)
+		ereport(ERROR,
+				(errcode(ERRCODE_CONNECTION_FAILURE),
+				 errmsg("could not receive list of replicated tables from the publisher: %s",
+						res->err)));
+
+	/* Process tables. */
+	slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
+	while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
+	{
+		char	   *pubname;
+		bool		isnull;
+
+		pubname = TextDatumGetCString(slot_getattr(slot, 1, &isnull));
+		Assert(!isnull);
+
+		ExecClearTuple(slot);
+		publist = list_append_unique(publist, makeString(pubname));
+	}
+
+	if (publist)
+	{
+		StringInfo	pubnames = makeStringInfo();
+
+		/* Prepare the list of publication(s) for warning message. */
+		GetPublicationsStr(publist, pubnames, false);
+		ereport(ERROR,
+				errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+				errmsg("subscription \"%s\" with copy_data = true cannot subscribe to a publication with publish_via_partition_root = true and publishes partitioned table with foreign table as partition",
+					   subname),
+				errdetail_plural("The subscription is for a publication (%s) with publish_via_partition_root = true, but one or more partitioned tables have foreign tables as partitions.",
+								 "The subscription is for publications (%s) with publish_via_partition_root = true but one or more partitioned tables have foreign tables as partitions.",
+								 list_length(publist), pubnames->data),
+				errhint("Drop the foreign table from the publication or set publish_via_partition_root = false on publication or set copy_data = false."));
+	}
+
+	ExecDropSingleTupleTableSlot(slot);
+
+	walrcv_clear_result(res);
+}
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index b8837f26cb4..5859224900f 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -1133,6 +1133,52 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
 					 errmsg("\"%s\" is not partitioned",
 							RelationGetRelationName(parent))));
 
+		/*
+		 * If we're creating a partition that's a foreign table, verify that
+		 * the parent table is not in a publication with
+		 * publish_via_partition_root enabled.
+		 */
+		if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE)
+		{
+			Oid			schemaid;
+			List	   *puboids;
+			List	   *ancestors;
+
+			/* Start with publications of all tables */
+			puboids = GetAllTablesPublications();
+
+			/* capture all publications that include this relation directly */
+			puboids = list_concat(puboids, GetRelationPublications(parent->rd_id));
+			schemaid = RelationGetNamespace(parent);
+			puboids = list_concat(puboids, GetSchemaPublications(schemaid));
+
+			/* and do the same for its ancestors, if any */
+			ancestors = get_partition_ancestors(parent->rd_id);
+			foreach_oid(ancestor, ancestors)
+			{
+				puboids = list_concat(puboids, GetRelationPublications(ancestor));
+				schemaid = get_rel_namespace(ancestor);
+				puboids = list_concat(puboids, GetSchemaPublications(schemaid));
+			}
+
+			/* Check the publish_via_partition_root bit for each of those */
+			list_sort(puboids, list_oid_cmp);
+			list_deduplicate_oid(puboids);
+			foreach_oid(puboid, puboids)
+			{
+				Publication *pub = GetPublication(puboid);
+
+				if (pub->pubviaroot)
+					ereport(ERROR,
+							errcode(ERRCODE_WRONG_OBJECT_TYPE),
+							errmsg("cannot create foreign table \"%s\" as a partition of \"%s\"",
+								   RelationGetRelationName(rel), RelationGetRelationName(parent)),
+							errdetail("Partitioned table \"%s\" is published with option \"%s\" in publication \"%s\".",
+									  RelationGetRelationName(parent),
+									  "publish_via_partition_root", pub->name));
+			}
+		}
+
 		/*
 		 * The partition constraint of the default partition depends on the
 		 * partition bounds of every other partition. It is possible that
@@ -20322,6 +20368,70 @@ ATExecAttachPartition(List **wqueue, Relation rel, PartitionCmd *cmd,
 				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
 				 errmsg("cannot attach temporary relation of another session as partition")));
 
+	/*
+	 * If the relation to attach is a foreign table, or a partitioned table
+	 * that contains a foreign table as partition, then verify that the parent
+	 * table is not in a publication with publish_via_partition_root enabled.
+	 */
+	if (attachrel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+		(attachrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE &&
+		 RelationHasForeignPartition(attachrel)))
+	{
+		Oid			schemaid;
+		List	   *puboids;
+		List	   *ancestors;
+
+		/* Start with publications of all tables */
+		puboids = GetAllTablesPublications();
+
+		/* capture all publications that include this relation directly */
+		puboids = list_concat(puboids, GetRelationPublications(rel->rd_id));
+		schemaid = RelationGetNamespace(rel);
+		puboids = list_concat(puboids, GetSchemaPublications(schemaid));
+
+		/* and do the same for its ancestors, if any */
+		ancestors = get_partition_ancestors(rel->rd_id);
+		foreach_oid(ancestor, ancestors)
+		{
+			puboids = list_concat(puboids, GetRelationPublications(ancestor));
+			schemaid = get_rel_namespace(ancestor);
+			puboids = list_concat(puboids, GetSchemaPublications(schemaid));
+		}
+
+		/* Now check the publish_via_partition_root bit for each of those */
+		list_sort(puboids, list_oid_cmp);
+		list_deduplicate_oid(puboids);
+		foreach_oid(puboid, puboids)
+		{
+			Publication *pub;
+
+			pub = GetPublication(puboid);
+			if (pub->pubviaroot)
+			{
+				if (attachrel->rd_rel->relkind == RELKIND_FOREIGN_TABLE)
+					ereport(ERROR,
+							(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+							 errmsg("cannot attach foreign table \"%s\" to partition table \"%s\"",
+									RelationGetRelationName(attachrel),
+									RelationGetRelationName(rel)),
+							 errdetail("Partitioned table \"%s\" is published with option \"%s\" in publication \"%s\".",
+									   RelationGetRelationName(rel),
+									   "publish_via_partition_root",
+									   pub->name)));
+				else
+					ereport(ERROR,
+							(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+							 errmsg("cannot attach table \"%s\" with a partition that's a foreign table to partition table \"%s\"",
+									RelationGetRelationName(attachrel),
+									RelationGetRelationName(rel)),
+							 errdetail("Partitioned table \"%s\" is published with option \"%s\" in publication \"%s\".",
+									   RelationGetRelationName(rel),
+									   "publish_via_partition_root",
+									   pub->name)));
+			}
+		}
+	}
+
 	/*
 	 * Check if attachrel has any identity columns or any columns that aren't
 	 * in the parent.
diff --git a/src/backend/partitioning/partdesc.c b/src/backend/partitioning/partdesc.c
index 328b4d450e4..56d176625e6 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -506,3 +506,35 @@ get_default_oid_from_partdesc(PartitionDesc partdesc)
 
 	return InvalidOid;
 }
+
+/*
+ * Return true if the given partitioned table ultimately contains a
+ * partition that is a foreign table, false otherwise.
+ */
+bool
+RelationHasForeignPartition(Relation rel)
+{
+	PartitionDesc pd = RelationGetPartitionDesc(rel, true);
+
+	for (int i = 0; i < pd->nparts; i++)
+	{
+		if (pd->is_leaf[i])
+		{
+			if (get_rel_relkind(pd->oids[i]) == RELKIND_FOREIGN_TABLE)
+				return true;
+		}
+		else
+		{
+			Relation	part;
+			bool		ret;
+
+			part = table_open(pd->oids[i], NoLock);
+			ret = RelationHasForeignPartition(part);
+			table_close(part, NoLock);
+			if (ret)
+				return true;
+		}
+	}
+
+	return false;
+}
diff --git a/src/bin/pg_upgrade/check.c b/src/bin/pg_upgrade/check.c
index fb063a2de42..d2e58c4463a 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -31,6 +31,7 @@ static void check_new_cluster_logical_replication_slots(void);
 static void check_new_cluster_subscription_configuration(void);
 static void check_old_cluster_for_valid_slots(void);
 static void check_old_cluster_subscription_state(void);
+static void check_for_valid_publication(ClusterInfo *cluster);
 
 /*
  * DataTypesUsageChecks - definitions of data type checks for the old cluster
@@ -634,6 +635,8 @@ check_and_dump_old_cluster(void)
 		check_old_cluster_subscription_state();
 	}
 
+	check_for_valid_publication(&old_cluster);
+
 	check_for_data_types_usage(&old_cluster);
 
 	/*
@@ -2273,3 +2276,89 @@ check_old_cluster_subscription_state(void)
 	else
 		check_ok();
 }
+
+/*
+ * Callback function for processing results of query for
+ * check_for_valid_publication()'s UpgradeTask.  If the query returned
+ * any rows (i.e., the check failed), write the details to the report file.
+ */
+static void
+process_pub_check(DbInfo *dbinfo, PGresult *res, void *arg)
+{
+	UpgradeTaskReport *report = (UpgradeTaskReport *) arg;
+	int			ntups = PQntuples(res);
+	int			i_puboid = PQfnumber(res, "oid");
+	int			i_pubviaroot = PQfnumber(res, "pubviaroot");
+	int			i_puballtables = PQfnumber(res, "puballtables");
+	int			i_ftrelid = PQfnumber(res, "ftrelid");
+
+	if (ntups == 0)
+		return;
+
+	if (report->file == NULL &&
+		(report->file = fopen_priv(report->path, "w")) == NULL)
+		pg_fatal("could not open file \"%s\": %m", report->path);
+
+	fprintf(report->file, "In database: %s\n", dbinfo->db_name);
+
+	for (int rowno = 0; rowno < ntups; rowno++)
+		fprintf(report->file, "The publication with oid: %s, has foreign table with oid: %s as partition of a published table. "
+				"Publication is published with publish_via_partition_root = %s and it is published %s.\n",
+				PQgetvalue(res, rowno, i_puboid),
+				PQgetvalue(res, rowno, i_ftrelid),
+				strcmp(PQgetvalue(res, rowno, i_pubviaroot), "f") ? "true" : "false",
+				strcmp(PQgetvalue(res, rowno, i_puballtables), "f") ? "FOR ALL TABLES" : "FOR TABLE/FOR TABLES IN SCHEMA");
+}
+
+/*
+ * check_for_valid_publication
+ *
+ * Verify if the publication has any published foreign table which are
+ * partitions of a table.
+ *
+ * For PG 18 and below foreign tables can be indirectly published when it is
+ * partition of a partitioned table. Avoid upgrading the cluster if it has a
+ * publication with publish_via_partition_root = true and has foreign table
+ * which is partition of a published table.
+ */
+static void
+check_for_valid_publication(ClusterInfo *cluster)
+{
+	UpgradeTaskReport report;
+	UpgradeTask *task;
+	const char *query;
+
+	if (GET_MAJOR_VERSION(cluster->major_version) >= 1900)
+		return;
+
+	prep_status("Checking for invalid publications");
+
+	report.file = NULL;
+	snprintf(report.path, sizeof(report.path), "%s/%s",
+			 log_opts.basedir,
+			 "pubs_invalid.txt");
+
+	query = "SELECT p.oid, p.pubviaroot, p.puballtables, ft.ftrelid "
+		"from pg_catalog.pg_publication p, LATERAL "
+		"pg_get_publication_tables(p.pubname) gpt, LATERAL "
+		"pg_partition_tree(gpt.relid) gt JOIN pg_catalog.pg_foreign_table ft ON "
+		"ft.ftrelid = gt.relid WHERE p.pubviaroot = true;";
+
+	task = upgrade_task_create();
+	upgrade_task_add_step(task, query,
+						  process_pub_check,
+						  true, &report);
+	upgrade_task_run(task, cluster);
+	upgrade_task_free(task);
+
+	if (report.file)
+	{
+		fclose(report.file);
+		pg_log(PG_REPORT, "fatal");
+		pg_fatal("Your installation contains publications where one or more partitioned tables have foreign tables as partitions.\n"
+				 "A list of potentially-affected publications is in the file:\n"
+				 "    %s", report.path);
+	}
+	else
+		check_ok();
+}
diff --git a/src/include/partitioning/partdesc.h b/src/include/partitioning/partdesc.h
index 34533f7004c..5fbafdc06f9 100644
--- a/src/include/partitioning/partdesc.h
+++ b/src/include/partitioning/partdesc.h
@@ -71,5 +71,6 @@ extern PartitionDesc PartitionDirectoryLookup(PartitionDirectory, Relation);
 extern void DestroyPartitionDirectory(PartitionDirectory pdir);
 
 extern Oid	get_default_oid_from_partdesc(PartitionDesc partdesc);
+extern bool RelationHasForeignPartition(Relation rel);
 
 #endif							/* PARTDESC_H */
diff --git a/src/test/subscription/Makefile b/src/test/subscription/Makefile
index 50b65d8f6ea..11041f79667 100644
--- a/src/test/subscription/Makefile
+++ b/src/test/subscription/Makefile
@@ -14,6 +14,7 @@ top_builddir = ../../..
 include $(top_builddir)/src/Makefile.global
 
 EXTRA_INSTALL = contrib/hstore
+EXTRA_INSTALL += contrib/postgres_fdw
 
 export with_icu
 
diff --git a/src/test/subscription/t/013_partition.pl b/src/test/subscription/t/013_partition.pl
index 4f78dd48815..466473bd387 100644
--- a/src/test/subscription/t/013_partition.pl
+++ b/src/test/subscription/t/013_partition.pl
@@ -8,6 +8,9 @@ use PostgreSQL::Test::Cluster;
 use PostgreSQL::Test::Utils;
 use Test::More;
 
+my $stdout;
+my $stderr;
+
 # setup
 
 my $node_publisher = PostgreSQL::Test::Cluster->new('publisher');
@@ -890,4 +893,140 @@ $result = $node_subscriber2->safe_psql('postgres',
 	"SELECT a, b, c FROM tab5_1 ORDER BY 1");
 is($result, qq(4||1), 'updates of tab5 replicated correctly');
 
+# Test with partitioned table with foreign table as its partition.
+$node_publisher->safe_psql('postgres', "CREATE DATABASE db1");
+$node_publisher->safe_psql(
+	'db1',
+	q{CREATE EXTENSION IF NOT EXISTS postgres_fdw;
+	CREATE SERVER fdw FOREIGN DATA WRAPPER postgres_fdw;
+	CREATE USER MAPPING FOR CURRENT_USER SERVER fdw;});
+
+$node_publisher->safe_psql(
+	'db1',
+	q{CREATE SCHEMA sc1;
+	CREATE SCHEMA sc2;
+	CREATE TABLE sc1.tab6(id int) PARTITION BY RANGE(id);
+	CREATE TABLE sc1.tab6_1 PARTITION OF sc1.tab6 FOR VALUES FROM (0) TO (5);
+	CREATE TABLE sc1.tab6_2 PARTITION OF sc1.tab6 FOR VALUES FROM (5) TO (15) PARTITION BY RANGE(id);
+	CREATE FOREIGN TABLE sc1.tab6_2_1 PARTITION OF sc1.tab6_2 FOR VALUES FROM (10) TO (15) SERVER fdw;}
+);
+$node_publisher->safe_psql(
+	'db1',
+	q{CREATE PUBLICATION pub_foreign_1 FOR TABLE sc1.tab6 with (publish_via_partition_root);
+	CREATE PUBLICATION pub_foreign_2 FOR TABLES IN SCHEMA sc1 with (publish_via_partition_root);
+	CREATE PUBLICATION pub_foreign_3 FOR ALL TABLES with (publish_via_partition_root);
+	CREATE PUBLICATION pub_foreign_4 FOR TABLES IN SCHEMA sc2 with (publish_via_partition_root);
+	});
+
+$node_subscriber1->safe_psql('postgres', "CREATE DATABASE db1");
+$node_subscriber1->safe_psql(
+	'db1',
+	q{CREATE SCHEMA sc1;
+	CREATE SCHEMA sc2;
+	CREATE TABLE sc1.tab6(id int) PARTITION BY RANGE(id);
+	CREATE TABLE sc1.tab6_1 PARTITION OF sc1.tab6 FOR VALUES FROM (0) TO (5);
+	CREATE TABLE sc1.tab6_2 PARTITION OF sc1.tab6 FOR VALUES FROM (5) TO (15) PARTITION BY RANGE(id);
+	CREATE TABLE sc1.tab6_2_1 PARTITION OF sc1.tab6_2 FOR VALUES FROM (10) TO (15);}
+);
+
+$publisher_connstr = $node_publisher->connstr . ' dbname=db1';
+
+# Create subscription on publication which has partitioned table with foreign
+# table as its partition.
+($result, $stdout, $stderr) = $node_subscriber1->psql('db1',
+	"CREATE SUBSCRIPTION sub_foreign_1 CONNECTION '$publisher_connstr' PUBLICATION pub_foreign_1;"
+);
+like(
+	$stderr,
+	qr/ERROR:  subscription "sub_foreign_1" with copy_data = true cannot subscribe to a publication with publish_via_partition_root = true and publishes partitioned table with foreign table as partition/,
+	"Create subscription on a publication FOR TABLE on partitioned table with foreign table as its partition"
+);
+
+($result, $stdout, $stderr) = $node_subscriber1->psql('db1',
+	"CREATE SUBSCRIPTION sub_foreign_1 CONNECTION '$publisher_connstr' PUBLICATION pub_foreign_2;"
+);
+like(
+	$stderr,
+	qr/ERROR:  subscription "sub_foreign_1" with copy_data = true cannot subscribe to a publication with publish_via_partition_root = true and publishes partitioned table with foreign table as partition/,
+	"Create subscription on a publication FOR TABLES IN SCHEMA on partitioned table with foreign table as its partition"
+);
+($result, $stdout, $stderr) = $node_subscriber1->psql('db1',
+	"CREATE SUBSCRIPTION sub_foreign_1 CONNECTION '$publisher_connstr' PUBLICATION pub_foreign_3;"
+);
+like(
+	$stderr,
+	qr/ERROR:  subscription "sub_foreign_1" with copy_data = true cannot subscribe to a publication with publish_via_partition_root = true and publishes partitioned table with foreign table as partition/,
+	"Create subscription on a publication FOR ALL TABLES on partitioned table with foreign table as its partition"
+);
+
+# If publication is created with publish_via_partition_root = false, we can
+# create subscription on it. But replication of partitioned table with foreign
+# is skipped
+$node_publisher->safe_psql('db1',
+	"ALTER PUBLICATION pub_foreign_1 SET (publish_via_partition_root = false)"
+);
+($result, $stdout, $stderr) = $node_subscriber1->psql('db1',
+	"CREATE SUBSCRIPTION sub_foreign_1 CONNECTION '$publisher_connstr' PUBLICATION pub_foreign_1;"
+);
+$node_subscriber1->safe_psql('db1', 'DROP SUBSCRIPTION sub_foreign_1');
+
+# if subscription is created with copy_data=false, we can create subscription
+($result, $stdout, $stderr) = $node_subscriber1->psql('db1',
+	"CREATE SUBSCRIPTION sub_foreign_1 CONNECTION '$publisher_connstr' PUBLICATION pub_foreign_2 WITH (copy_data=false)"
+);
+$node_subscriber1->safe_psql('db1', 'DROP SUBSCRIPTION sub_foreign_1');
+
+# Alter subscription to include publisher which has partitioned table with
+# foreign table as its partition.
+$node_subscriber1->safe_psql('db1',
+	"CREATE SUBSCRIPTION sub_foreign_1 CONNECTION '$publisher_connstr' PUBLICATION pub_foreign_4;"
+);
+($result, $stdout, $stderr) = $node_subscriber1->psql('db1',
+	"ALTER SUBSCRIPTION sub_foreign_1 ADD PUBLICATION pub_foreign_2;");
+like(
+	$stderr,
+	qr/ERROR:  subscription "sub_foreign_1" with copy_data = true cannot subscribe to a publication with publish_via_partition_root = true and publishes partitioned table with foreign table as partition/,
+	"Add publication which has partitioned table with foreign table as its partition to subscription"
+);
+($result, $stdout, $stderr) = $node_subscriber1->psql('db1',
+	"ALTER SUBSCRIPTION sub_foreign_1 SET PUBLICATION pub_foreign_2;");
+like(
+	$stderr,
+	qr/ERROR:  subscription "sub_foreign_1" with copy_data = true cannot subscribe to a publication with publish_via_partition_root = true and publishes partitioned table with foreign table as partition/,
+	"set publication which has partitioned table with foreign table as its partition for subscription"
+);
+
+# Create foreign table as a partition of partitioned table already part of
+# publication.
+($result, $stdout, $stderr) = $node_subscriber1->psql('db1',
+	"CREATE FOREIGN TABLE sc1.tab6_2_1 PARTITION OF sc1.tab6_2 FOR VALUES FROM (10) TO (15) SERVER fdw;"
+);
+$node_publisher->safe_psql('db1', "DROP FOREIGN TABLE sc1.tab6_2_1");
+($result, $stdout, $stderr) = $node_publisher->psql('db1',
+	"CREATE FOREIGN TABLE sc1.tab6_2_1 PARTITION OF sc1.tab6_2 FOR VALUES FROM (10) TO (15) SERVER fdw"
+);
+like(
+	$stderr,
+	qr/ERROR:  cannot create foreign table "tab6_2_1" as a partition of "tab6_2"/,
+	"Create foreign table as a partition of partitioned table already part of publication"
+);
+
+# Attach foreign table as a partition of partitioned table already part of
+# publication.
+($result, $stdout, $stderr) = $node_subscriber1->psql('db1',
+	"CREATE FOREIGN TABLE sc1.tab6_2_1 (id int) SERVER fdw");
+($result, $stdout, $stderr) = $node_subscriber1->psql('db1',
+	"ALTER TABLE sc1.tab6_2 ATTACH PARTITION sc1.tab6_2_1 FOR VALUES FROM (10) TO (15)"
+);
+$node_publisher->safe_psql('db1',
+	"CREATE FOREIGN TABLE sc1.tab6_2_1 (id int) SERVER fdw");
+($result, $stdout, $stderr) = $node_publisher->psql('db1',
+	"ALTER TABLE sc1.tab6_2 ATTACH PARTITION sc1.tab6_2_1 FOR VALUES FROM (10) TO (15)"
+);
+like(
+	$stderr,
+	qr/ERROR:  cannot attach foreign table "tab6_2_1" to partition table "tab6_2"/,
+	"Attach foreign table as a partition of already published partitioned table"
+);
+
 done_testing();
-- 
2.34.1

