diff --git a/contrib/postgres_fdw/expected/postgres_fdw.out b/contrib/postgres_fdw/expected/postgres_fdw.out
index 9ad9035..310c715 100644
--- a/contrib/postgres_fdw/expected/postgres_fdw.out
+++ b/contrib/postgres_fdw/expected/postgres_fdw.out
@@ -7116,9 +7116,9 @@ select * from bar where f1 in (select f1 from foo) for update;
                                           QUERY PLAN                                          
 ----------------------------------------------------------------------------------------------
  LockRows
-   Output: bar.f1, bar.f2, bar.ctid, bar.*, bar.tableoid, foo.ctid, foo.*, foo.tableoid
+   Output: bar.f1, bar.f2, bar.ctid, foo.ctid, bar.*, bar.tableoid, foo.*, foo.tableoid
    ->  Hash Join
-         Output: bar.f1, bar.f2, bar.ctid, bar.*, bar.tableoid, foo.ctid, foo.*, foo.tableoid
+         Output: bar.f1, bar.f2, bar.ctid, foo.ctid, bar.*, bar.tableoid, foo.*, foo.tableoid
          Inner Unique: true
          Hash Cond: (bar.f1 = foo.f1)
          ->  Append
@@ -7128,15 +7128,15 @@ select * from bar where f1 in (select f1 from foo) for update;
                      Output: bar2.f1, bar2.f2, bar2.ctid, bar2.*, bar2.tableoid
                      Remote SQL: SELECT f1, f2, f3, ctid FROM public.loct2 FOR UPDATE
          ->  Hash
-               Output: foo.ctid, foo.*, foo.tableoid, foo.f1
+               Output: foo.ctid, foo.f1, foo.*, foo.tableoid
                ->  HashAggregate
-                     Output: foo.ctid, foo.*, foo.tableoid, foo.f1
+                     Output: foo.ctid, foo.f1, foo.*, foo.tableoid
                      Group Key: foo.f1
                      ->  Append
                            ->  Seq Scan on public.foo
-                                 Output: foo.ctid, foo.*, foo.tableoid, foo.f1
+                                 Output: foo.ctid, foo.f1, foo.*, foo.tableoid
                            ->  Foreign Scan on public.foo2
-                                 Output: foo2.ctid, foo2.*, foo2.tableoid, foo2.f1
+                                 Output: foo2.ctid, foo2.f1, foo2.*, foo2.tableoid
                                  Remote SQL: SELECT f1, f2, f3, ctid FROM public.loct1
 (23 rows)
 
@@ -7154,9 +7154,9 @@ select * from bar where f1 in (select f1 from foo) for share;
                                           QUERY PLAN                                          
 ----------------------------------------------------------------------------------------------
  LockRows
-   Output: bar.f1, bar.f2, bar.ctid, bar.*, bar.tableoid, foo.ctid, foo.*, foo.tableoid
+   Output: bar.f1, bar.f2, bar.ctid, foo.ctid, bar.*, bar.tableoid, foo.*, foo.tableoid
    ->  Hash Join
-         Output: bar.f1, bar.f2, bar.ctid, bar.*, bar.tableoid, foo.ctid, foo.*, foo.tableoid
+         Output: bar.f1, bar.f2, bar.ctid, foo.ctid, bar.*, bar.tableoid, foo.*, foo.tableoid
          Inner Unique: true
          Hash Cond: (bar.f1 = foo.f1)
          ->  Append
@@ -7166,15 +7166,15 @@ select * from bar where f1 in (select f1 from foo) for share;
                      Output: bar2.f1, bar2.f2, bar2.ctid, bar2.*, bar2.tableoid
                      Remote SQL: SELECT f1, f2, f3, ctid FROM public.loct2 FOR SHARE
          ->  Hash
-               Output: foo.ctid, foo.*, foo.tableoid, foo.f1
+               Output: foo.ctid, foo.f1, foo.*, foo.tableoid
                ->  HashAggregate
-                     Output: foo.ctid, foo.*, foo.tableoid, foo.f1
+                     Output: foo.ctid, foo.f1, foo.*, foo.tableoid
                      Group Key: foo.f1
                      ->  Append
                            ->  Seq Scan on public.foo
-                                 Output: foo.ctid, foo.*, foo.tableoid, foo.f1
+                                 Output: foo.ctid, foo.f1, foo.*, foo.tableoid
                            ->  Foreign Scan on public.foo2
-                                 Output: foo2.ctid, foo2.*, foo2.tableoid, foo2.f1
+                                 Output: foo2.ctid, foo2.f1, foo2.*, foo2.tableoid
                                  Remote SQL: SELECT f1, f2, f3, ctid FROM public.loct1
 (23 rows)
 
@@ -7203,15 +7203,15 @@ update bar set f2 = f2 + 100 where f1 in (select f1 from foo);
          ->  Seq Scan on public.bar
                Output: bar.f1, bar.f2, bar.ctid
          ->  Hash
-               Output: foo.ctid, foo.*, foo.tableoid, foo.f1
+               Output: foo.ctid, foo.f1, foo.*, foo.tableoid
                ->  HashAggregate
-                     Output: foo.ctid, foo.*, foo.tableoid, foo.f1
+                     Output: foo.ctid, foo.f1, foo.*, foo.tableoid
                      Group Key: foo.f1
                      ->  Append
                            ->  Seq Scan on public.foo
-                                 Output: foo.ctid, foo.*, foo.tableoid, foo.f1
+                                 Output: foo.ctid, foo.f1, foo.*, foo.tableoid
                            ->  Foreign Scan on public.foo2
-                                 Output: foo2.ctid, foo2.*, foo2.tableoid, foo2.f1
+                                 Output: foo2.ctid, foo2.f1, foo2.*, foo2.tableoid
                                  Remote SQL: SELECT f1, f2, f3, ctid FROM public.loct1
    ->  Hash Join
          Output: bar2.f1, (bar2.f2 + 100), bar2.f3, bar2.ctid, foo.ctid, foo.*, foo.tableoid
@@ -7221,15 +7221,15 @@ update bar set f2 = f2 + 100 where f1 in (select f1 from foo);
                Output: bar2.f1, bar2.f2, bar2.f3, bar2.ctid
                Remote SQL: SELECT f1, f2, f3, ctid FROM public.loct2 FOR UPDATE
          ->  Hash
-               Output: foo.ctid, foo.*, foo.tableoid, foo.f1
+               Output: foo.ctid, foo.f1, foo.*, foo.tableoid
                ->  HashAggregate
-                     Output: foo.ctid, foo.*, foo.tableoid, foo.f1
+                     Output: foo.ctid, foo.f1, foo.*, foo.tableoid
                      Group Key: foo.f1
                      ->  Append
                            ->  Seq Scan on public.foo
-                                 Output: foo.ctid, foo.*, foo.tableoid, foo.f1
+                                 Output: foo.ctid, foo.f1, foo.*, foo.tableoid
                            ->  Foreign Scan on public.foo2
-                                 Output: foo2.ctid, foo2.*, foo2.tableoid, foo2.f1
+                                 Output: foo2.ctid, foo2.f1, foo2.*, foo2.tableoid
                                  Remote SQL: SELECT f1, f2, f3, ctid FROM public.loct1
 (39 rows)
 
@@ -8435,7 +8435,7 @@ SELECT t1.a,t2.b,t2.c FROM fprt1 t1 LEFT JOIN (SELECT * FROM fprt2 WHERE a < 10)
  Foreign Scan
    Output: t1.a, ftprt2_p1.b, ftprt2_p1.c
    Relations: (public.ftprt1_p1 t1) LEFT JOIN (public.ftprt2_p1 fprt2)
-   Remote SQL: SELECT r5.a, r7.b, r7.c FROM (public.fprt1_p1 r5 LEFT JOIN public.fprt2_p1 r7 ON (((r5.a = r7.b)) AND ((r5.b = r7.a)) AND ((r7.a < 10)))) WHERE ((r5.a < 10)) ORDER BY r5.a ASC NULLS LAST, r7.b ASC NULLS LAST, r7.c ASC NULLS LAST
+   Remote SQL: SELECT r5.a, r6.b, r6.c FROM (public.fprt1_p1 r5 LEFT JOIN public.fprt2_p1 r6 ON (((r5.a = r6.b)) AND ((r5.b = r6.a)) AND ((r6.a < 10)))) WHERE ((r5.a < 10)) ORDER BY r5.a ASC NULLS LAST, r6.b ASC NULLS LAST, r6.c ASC NULLS LAST
 (4 rows)
 
 SELECT t1.a,t2.b,t2.c FROM fprt1 t1 LEFT JOIN (SELECT * FROM fprt2 WHERE a < 10) t2 ON (t1.a = t2.b and t1.b = t2.a) WHERE t1.a < 10 ORDER BY 1,2,3;
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index cfad8a3..b72db85 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -1654,9 +1654,17 @@ ExecCreatePartitionPruneState(PlanState *planstate,
 				memcpy(pprune->subplan_map, pinfo->subplan_map,
 					   sizeof(int) * pinfo->nparts);
 
-				/* Double-check that list of relations has not changed. */
-				Assert(memcmp(partdesc->oids, pinfo->relid_map,
-					   pinfo->nparts * sizeof(Oid)) == 0);
+				/*
+				 * Double-check that the list of unpruned relations has not
+				 * changed.  (Pruned partitions are not in relid_map[].)
+				 */
+#ifdef USE_ASSERT_CHECKING
+				for (int k = 0; k < pinfo->nparts; k++)
+				{
+					Assert(partdesc->oids[k] == pinfo->relid_map[k] ||
+						   pinfo->subplan_map[k] == -1);
+				}
+#endif
 			}
 			else
 			{
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 56a5084..09f5f0c 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -139,9 +139,6 @@ static void subquery_push_qual(Query *subquery,
 static void recurse_push_qual(Node *setOp, Query *topquery,
 				  RangeTblEntry *rte, Index rti, Node *qual);
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
-static bool apply_child_basequals(PlannerInfo *root, RelOptInfo *rel,
-					  RelOptInfo *childrel,
-					  RangeTblEntry *childRTE, AppendRelInfo *appinfo);
 
 
 /*
@@ -946,8 +943,6 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	double	   *parent_attrsizes;
 	int			nattrs;
 	ListCell   *l;
-	Relids		live_children = NULL;
-	bool		did_pruning = false;
 
 	/* Guard against stack overflow due to overly deep inheritance tree. */
 	check_stack_depth();
@@ -966,21 +961,6 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		rel->partitioned_child_rels = list_make1_int(rti);
 
 	/*
-	 * If the partitioned relation has any baserestrictinfo quals then we
-	 * attempt to use these quals to prune away partitions that cannot
-	 * possibly contain any tuples matching these quals.  In this case we'll
-	 * store the relids of all partitions which could possibly contain a
-	 * matching tuple, and skip anything else in the loop below.
-	 */
-	if (enable_partition_pruning &&
-		rte->relkind == RELKIND_PARTITIONED_TABLE &&
-		rel->baserestrictinfo != NIL)
-	{
-		live_children = prune_append_rel_partitions(rel);
-		did_pruning = true;
-	}
-
-	/*
 	 * If this is a partitioned baserel, set the consider_partitionwise_join
 	 * flag; currently, we only consider partitionwise joins with the baserel
 	 * if its targetlist doesn't contain a whole-row Var.
@@ -1034,30 +1014,17 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		childrel = find_base_rel(root, childRTindex);
 		Assert(childrel->reloptkind == RELOPT_OTHER_MEMBER_REL);
 
-		if (did_pruning && !bms_is_member(appinfo->child_relid, live_children))
-		{
-			/* This partition was pruned; skip it. */
-			set_dummy_rel_pathlist(childrel);
+		/* We may have already proven the child to be dummy. */
+		if (IS_DUMMY_REL(childrel))
 			continue;
-		}
 
 		/*
 		 * We have to copy the parent's targetlist and quals to the child,
-		 * with appropriate substitution of variables.  If any constant false
-		 * or NULL clauses turn up, we can disregard the child right away. If
-		 * not, we can apply constraint exclusion with just the
-		 * baserestrictinfo quals.
+		 * with appropriate substitution of variables.  However, the
+		 * baserestrictinfo quals were already copied/substituted when the
+		 * child RelOptInfo was built.  So we don't need any additional setup
+		 * before applying constraint exclusion.
 		 */
-		if (!apply_child_basequals(root, rel, childrel, childRTE, appinfo))
-		{
-			/*
-			 * Some restriction clause reduced to constant FALSE or NULL after
-			 * substitution, so this child need not be scanned.
-			 */
-			set_dummy_rel_pathlist(childrel);
-			continue;
-		}
-
 		if (relation_excluded_by_constraints(root, childrel, childRTE))
 		{
 			/*
@@ -1069,7 +1036,8 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		}
 
 		/*
-		 * CE failed, so finish copying/modifying targetlist and join quals.
+		 * Constraint exclusion failed, so copy the parent's join quals and
+		 * targetlist to the child, with appropriate variable substitutions.
 		 *
 		 * NB: the resulting childrel->reltarget->exprs may contain arbitrary
 		 * expressions, which otherwise would not occur in a rel's targetlist.
@@ -3594,133 +3562,6 @@ generate_partitionwise_join_paths(PlannerInfo *root, RelOptInfo *rel)
 	list_free(live_children);
 }
 
-/*
- * apply_child_basequals
- *		Populate childrel's quals based on rel's quals, translating them using
- *		appinfo.
- *
- * If any of the resulting clauses evaluate to false or NULL, we return false
- * and don't apply any quals.  Caller can mark the relation as a dummy rel in
- * this case, since it needn't be scanned.
- *
- * If any resulting clauses evaluate to true, they're unnecessary and we don't
- * apply then.
- */
-static bool
-apply_child_basequals(PlannerInfo *root, RelOptInfo *rel,
-					  RelOptInfo *childrel, RangeTblEntry *childRTE,
-					  AppendRelInfo *appinfo)
-{
-	List	   *childquals;
-	Index		cq_min_security;
-	ListCell   *lc;
-
-	/*
-	 * The child rel's targetlist might contain non-Var expressions, which
-	 * means that substitution into the quals could produce opportunities for
-	 * const-simplification, and perhaps even pseudoconstant quals. Therefore,
-	 * transform each RestrictInfo separately to see if it reduces to a
-	 * constant or pseudoconstant.  (We must process them separately to keep
-	 * track of the security level of each qual.)
-	 */
-	childquals = NIL;
-	cq_min_security = UINT_MAX;
-	foreach(lc, rel->baserestrictinfo)
-	{
-		RestrictInfo *rinfo = (RestrictInfo *) lfirst(lc);
-		Node	   *childqual;
-		ListCell   *lc2;
-
-		Assert(IsA(rinfo, RestrictInfo));
-		childqual = adjust_appendrel_attrs(root,
-										   (Node *) rinfo->clause,
-										   1, &appinfo);
-		childqual = eval_const_expressions(root, childqual);
-		/* check for flat-out constant */
-		if (childqual && IsA(childqual, Const))
-		{
-			if (((Const *) childqual)->constisnull ||
-				!DatumGetBool(((Const *) childqual)->constvalue))
-			{
-				/* Restriction reduces to constant FALSE or NULL */
-				return false;
-			}
-			/* Restriction reduces to constant TRUE, so drop it */
-			continue;
-		}
-		/* might have gotten an AND clause, if so flatten it */
-		foreach(lc2, make_ands_implicit((Expr *) childqual))
-		{
-			Node	   *onecq = (Node *) lfirst(lc2);
-			bool		pseudoconstant;
-
-			/* check for pseudoconstant (no Vars or volatile functions) */
-			pseudoconstant =
-				!contain_vars_of_level(onecq, 0) &&
-				!contain_volatile_functions(onecq);
-			if (pseudoconstant)
-			{
-				/* tell createplan.c to check for gating quals */
-				root->hasPseudoConstantQuals = true;
-			}
-			/* reconstitute RestrictInfo with appropriate properties */
-			childquals = lappend(childquals,
-								 make_restrictinfo((Expr *) onecq,
-												   rinfo->is_pushed_down,
-												   rinfo->outerjoin_delayed,
-												   pseudoconstant,
-												   rinfo->security_level,
-												   NULL, NULL, NULL));
-			/* track minimum security level among child quals */
-			cq_min_security = Min(cq_min_security, rinfo->security_level);
-		}
-	}
-
-	/*
-	 * In addition to the quals inherited from the parent, we might have
-	 * securityQuals associated with this particular child node. (Currently
-	 * this can only happen in appendrels originating from UNION ALL;
-	 * inheritance child tables don't have their own securityQuals, see
-	 * expand_inherited_rtentry().)	Pull any such securityQuals up into the
-	 * baserestrictinfo for the child.  This is similar to
-	 * process_security_barrier_quals() for the parent rel, except that we
-	 * can't make any general deductions from such quals, since they don't
-	 * hold for the whole appendrel.
-	 */
-	if (childRTE->securityQuals)
-	{
-		Index		security_level = 0;
-
-		foreach(lc, childRTE->securityQuals)
-		{
-			List	   *qualset = (List *) lfirst(lc);
-			ListCell   *lc2;
-
-			foreach(lc2, qualset)
-			{
-				Expr	   *qual = (Expr *) lfirst(lc2);
-
-				/* not likely that we'd see constants here, so no check */
-				childquals = lappend(childquals,
-									 make_restrictinfo(qual,
-													   true, false, false,
-													   security_level,
-													   NULL, NULL, NULL));
-				cq_min_security = Min(cq_min_security, security_level);
-			}
-			security_level++;
-		}
-		Assert(security_level <= root->qual_security_level);
-	}
-
-	/*
-	 * OK, we've got all the baserestrictinfo quals for this child.
-	 */
-	childrel->baserestrictinfo = childquals;
-	childrel->baserestrict_min_security = cq_min_security;
-
-	return true;
-}
 
 /*****************************************************************************
  *			DEBUG SUPPORT
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 9604a54..d17cc05 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -14,6 +14,7 @@
  */
 #include "postgres.h"
 
+#include "access/table.h"
 #include "miscadmin.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/appendinfo.h"
@@ -21,6 +22,8 @@
 #include "optimizer/joininfo.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
+#include "optimizer/tlist.h"
+#include "parser/parsetree.h"
 #include "partitioning/partbounds.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
@@ -51,6 +54,9 @@ static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 						Relids left_relids, Relids right_relids);
 static int match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel,
 							 bool strict_op);
+static RelOptInfo *build_dummy_partition_rel(PlannerInfo *root,
+						  RelOptInfo *parent, Relation parentrel,
+						  int partidx);
 
 
 /*
@@ -1346,6 +1352,8 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 					   RelOptInfo *joinrel, SpecialJoinInfo *parent_sjinfo,
 					   List *parent_restrictlist)
 {
+	Relation	baserel1 = NULL,
+				baserel2 = NULL;
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
 	int			nparts;
@@ -1396,6 +1404,19 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 
 	nparts = joinrel->nparts;
 
+	if (rel1_is_simple)
+	{
+		RangeTblEntry *rte = planner_rt_fetch(rel1->relid, root);
+
+		baserel1 = table_open(rte->relid, NoLock);
+	}
+	if (rel2_is_simple)
+	{
+		RangeTblEntry *rte = planner_rt_fetch(rel2->relid, root);
+
+		baserel2 = table_open(rte->relid, NoLock);
+	}
+
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
@@ -1412,6 +1433,13 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		if (rel1_is_simple && child_rel1 == NULL)
+			child_rel1 = build_dummy_partition_rel(root, rel1, baserel1,
+												   cnt_parts);
+		if (rel2_is_simple && child_rel2 == NULL)
+			child_rel2 = build_dummy_partition_rel(root, rel2, baserel2,
+												   cnt_parts);
+
 		/*
 		 * If a child table has consider_partitionwise_join=false, it means
 		 * that it's a dummy relation for which we skipped setting up tlist
@@ -1472,6 +1500,11 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
 	}
+
+	if (baserel1)
+		table_close(baserel1, NoLock);
+	if (baserel2)
+		table_close(baserel2, NoLock);
 }
 
 /*
@@ -1490,8 +1523,14 @@ update_child_rel_info(PlannerInfo *root,
 							   (Node *) rel->reltarget->exprs,
 							   1, &appinfo);
 
-	/* Make child entries in the EquivalenceClass as well */
-	if (rel->has_eclass_joins || has_useful_pathkeys(root, rel))
+	/*
+	 * Make child entries in the EquivalenceClass as well.  If the childrel
+	 * appears to be a dummy one (one built by build_dummy_partition_rel()),
+	 * no need to make any new entries, because anything that would need those
+	 * can instead use the parent's (rel).
+	 */
+	if (childrel->relid != rel->relid &&
+		(rel->has_eclass_joins || has_useful_pathkeys(root, rel)))
 		add_child_rel_equivalences(root, appinfo, rel, childrel);
 	childrel->has_eclass_joins = rel->has_eclass_joins;
 }
@@ -1702,3 +1741,53 @@ match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op)
 
 	return -1;
 }
+
+/*
+ * build_dummy_partition_rel
+ *		Build a RelOptInfo and AppendRelInfo for a pruned partition
+ *
+ * This does not result in opening the relation or a range table entry being
+ * created.  Also, the RelOptInfo thus created is not stored anywhere else
+ * beside the parent's part_rels array.
+ *
+ * The only reason this exists is because partition-wise join, in some cases,
+ * needs a RelOptInfo to represent an empty relation that's on the nullable
+ * side of an outer join, so that a Path representing the outer join can be
+ * created.
+ */
+static RelOptInfo *
+build_dummy_partition_rel(PlannerInfo *root, RelOptInfo *parent,
+						  Relation parentrel, int partidx)
+{
+	RelOptInfo *rel;
+
+	Assert(parent->part_rels[partidx] == NULL);
+
+	/* Create minimally valid-looking RelOptInfo with parent's relid. */
+	rel = makeNode(RelOptInfo);
+	rel->reloptkind = RELOPT_OTHER_MEMBER_REL;
+	rel->relid = parent->relid;
+	rel->relids = bms_copy(parent->relids);
+	if (parent->top_parent_relids)
+		rel->top_parent_relids = parent->top_parent_relids;
+	else
+		rel->top_parent_relids = bms_copy(parent->relids);
+	rel->reltarget = copy_pathtarget(parent->reltarget);
+	parent->part_rels[partidx] = rel;
+	mark_dummy_rel(rel);
+
+	/*
+	 * Now we'll need a (no-op) AppendRelInfo for parent, because we're
+	 * setting the dummy partition's relid to be same as the parent's.
+	 */
+	if (root->append_rel_array[parent->relid] == NULL)
+	{
+		AppendRelInfo *appinfo = make_append_rel_info(parentrel, parentrel,
+													  parent->relid,
+													  parent->relid);
+
+		root->append_rel_array[parent->relid] = appinfo;
+	}
+
+	return rel;
+}
diff --git a/src/backend/optimizer/plan/initsplan.c b/src/backend/optimizer/plan/initsplan.c
index 62dfac6..b3f264a 100644
--- a/src/backend/optimizer/plan/initsplan.c
+++ b/src/backend/optimizer/plan/initsplan.c
@@ -20,6 +20,7 @@
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
+#include "optimizer/inherit.h"
 #include "optimizer/joininfo.h"
 #include "optimizer/optimizer.h"
 #include "optimizer/pathnode.h"
@@ -161,9 +162,10 @@ add_other_rels_to_query(PlannerInfo *root)
 		if (rte->inh)
 		{
 			/* Only relation and subquery RTEs can have children. */
-			Assert(rte->rtekind == RTE_RELATION ||
-				   rte->rtekind == RTE_SUBQUERY);
-			add_appendrel_other_rels(root, rel, rti);
+			if (rte->rtekind == RTE_RELATION)
+				expand_inherited_rtentry(root, rel, rte, rti);
+			else
+				expand_appendrel_subquery(root, rel, rte, rti);
 		}
 	}
 }
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index ca7a0fb..c4d00b4 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -25,6 +25,7 @@
 #include "access/table.h"
 #include "access/xact.h"
 #include "catalog/pg_constraint.h"
+#include "catalog/pg_inherits.h"
 #include "catalog/pg_proc.h"
 #include "catalog/pg_type.h"
 #include "executor/executor.h"
@@ -679,12 +680,14 @@ subquery_planner(PlannerGlobal *glob, Query *parse,
 		flatten_simple_union_all(root);
 
 	/*
-	 * Detect whether any rangetable entries are RTE_JOIN kind; if not, we can
-	 * avoid the expense of doing flatten_join_alias_vars().  Likewise check
-	 * whether any are RTE_RESULT kind; if not, we can skip
-	 * remove_useless_result_rtes().  Also check for outer joins --- if none,
-	 * we can skip reduce_outer_joins().  And check for LATERAL RTEs, too.
-	 * This must be done after we have done pull_up_subqueries(), of course.
+	 * Check rangetable entries marked "inh" to see if they really need to be
+	 * treated as inheritance parents.  Also detect whether any rangetable
+	 * entries are RTE_JOIN kind; if not, we can avoid the expense of doing
+	 * flatten_join_alias_vars().  Also check for outer joins --- if none, we
+	 * can skip reduce_outer_joins().  Likewise check whether any RTEs are
+	 * RTE_RESULT kind; if not, we can skip remove_useless_result_rtes().  And
+	 * check for LATERAL RTEs, too.  This must be done after we have done
+	 * pull_up_subqueries(), of course.
 	 */
 	root->hasJoinRTEs = false;
 	root->hasLateralRTEs = false;
@@ -694,15 +697,36 @@ subquery_planner(PlannerGlobal *glob, Query *parse,
 	{
 		RangeTblEntry *rte = lfirst_node(RangeTblEntry, l);
 
-		if (rte->rtekind == RTE_JOIN)
+		switch (rte->rtekind)
 		{
-			root->hasJoinRTEs = true;
-			if (IS_OUTER_JOIN(rte->jointype))
-				hasOuterJoins = true;
-		}
-		else if (rte->rtekind == RTE_RESULT)
-		{
-			hasResultRTEs = true;
+			case RTE_RELATION:
+				if (rte->inh)
+				{
+					/*
+					 * Check to see if the relation actually has any children;
+					 * if not, clear the inh flag so we can treat it as a
+					 * plain base relation.
+					 *
+					 * Note: this could give a false-positive result, if the
+					 * rel once had children but no longer does.  We used to
+					 * be able to reset rte->inh later on when we discovered
+					 * that, but no more; we have to handle such cases as
+					 * full-fledged inheritance.
+					 */
+					rte->inh = has_subclass(rte->relid);
+				}
+				break;
+			case RTE_JOIN:
+				root->hasJoinRTEs = true;
+				if (IS_OUTER_JOIN(rte->jointype))
+					hasOuterJoins = true;
+				break;
+			case RTE_RESULT:
+				hasResultRTEs = true;
+				break;
+			default:
+				/* No work here for other RTE types */
+				break;
 		}
 		if (rte->lateral)
 			root->hasLateralRTEs = true;
@@ -710,23 +734,11 @@ subquery_planner(PlannerGlobal *glob, Query *parse,
 
 	/*
 	 * Preprocess RowMark information.  We need to do this after subquery
-	 * pullup (so that all non-inherited RTEs are present) and before
-	 * inheritance expansion (so that the info is available for
-	 * expand_inherited_tables to examine and modify).
+	 * pullup, so that all base relations are present.
 	 */
 	preprocess_rowmarks(root);
 
 	/*
-	 * Expand any rangetable entries that are inheritance sets into "append
-	 * relations".  This can add entries to the rangetable, but they must be
-	 * plain RTE_RELATION entries, so it's OK (and marginally more efficient)
-	 * to do it after checking for joins and other special RTEs.  We must do
-	 * this after pulling up subqueries, else we'd fail to handle inherited
-	 * tables in subqueries.
-	 */
-	expand_inherited_tables(root);
-
-	/*
 	 * Set hasHavingQual to remember if HAVING clause is present.  Needed
 	 * because preprocess_expression will reduce a constant-true condition to
 	 * an empty qual list ... but "HAVING TRUE" is not a semantic no-op.
@@ -1180,11 +1192,17 @@ inheritance_planner(PlannerInfo *root)
 {
 	Query	   *parse = root->parse;
 	int			top_parentRTindex = parse->resultRelation;
+	List	   *select_rtable;
+	List	   *select_appinfos;
+	List	   *child_appinfos;
+	List	   *old_child_rtis;
+	List	   *new_child_rtis;
 	Bitmapset  *subqueryRTindexes;
-	Bitmapset  *modifiableARIindexes;
+	Index		next_subquery_rti;
 	int			nominalRelation = -1;
 	Index		rootRelation = 0;
 	List	   *final_rtable = NIL;
+	List	   *final_rowmarks = NIL;
 	int			save_rel_array_size = 0;
 	RelOptInfo **save_rel_array = NULL;
 	AppendRelInfo **save_append_rel_array = NULL;
@@ -1196,14 +1214,15 @@ inheritance_planner(PlannerInfo *root)
 	List	   *rowMarks;
 	RelOptInfo *final_rel;
 	ListCell   *lc;
+	ListCell   *lc2;
 	Index		rti;
 	RangeTblEntry *parent_rte;
-	PlannerInfo *parent_root;
-	Query	   *parent_parse;
-	Bitmapset  *parent_relids = bms_make_singleton(top_parentRTindex);
-	PlannerInfo **parent_roots = NULL;
+	Bitmapset  *parent_relids;
+	Query	  **parent_parses;
 
-	Assert(parse->commandType != CMD_INSERT);
+	/* Should only get here for UPDATE or DELETE */
+	Assert(parse->commandType == CMD_UPDATE ||
+		   parse->commandType == CMD_DELETE);
 
 	/*
 	 * We generate a modified instance of the original Query for each target
@@ -1234,39 +1253,14 @@ inheritance_planner(PlannerInfo *root)
 	}
 
 	/*
-	 * Next, we want to identify which AppendRelInfo items contain references
-	 * to any of the aforesaid subquery RTEs.  These items will need to be
-	 * copied and modified to adjust their subquery references; whereas the
-	 * other ones need not be touched.  It's worth being tense over this
-	 * because we can usually avoid processing most of the AppendRelInfo
-	 * items, thereby saving O(N^2) space and time when the target is a large
-	 * inheritance tree.  We can identify AppendRelInfo items by their
-	 * child_relid, since that should be unique within the list.
-	 */
-	modifiableARIindexes = NULL;
-	if (subqueryRTindexes != NULL)
-	{
-		foreach(lc, root->append_rel_list)
-		{
-			AppendRelInfo *appinfo = lfirst_node(AppendRelInfo, lc);
-
-			if (bms_is_member(appinfo->parent_relid, subqueryRTindexes) ||
-				bms_is_member(appinfo->child_relid, subqueryRTindexes) ||
-				bms_overlap(pull_varnos((Node *) appinfo->translated_vars),
-							subqueryRTindexes))
-				modifiableARIindexes = bms_add_member(modifiableARIindexes,
-													  appinfo->child_relid);
-		}
-	}
-
-	/*
 	 * If the parent RTE is a partitioned table, we should use that as the
 	 * nominal target relation, because the RTEs added for partitioned tables
 	 * (including the root parent) as child members of the inheritance set do
 	 * not appear anywhere else in the plan, so the confusion explained below
 	 * for non-partitioning inheritance cases is not possible.
 	 */
-	parent_rte = rt_fetch(top_parentRTindex, root->parse->rtable);
+	parent_rte = rt_fetch(top_parentRTindex, parse->rtable);
+	Assert(parent_rte->inh);
 	if (parent_rte->relkind == RELKIND_PARTITIONED_TABLE)
 	{
 		nominalRelation = top_parentRTindex;
@@ -1274,48 +1268,218 @@ inheritance_planner(PlannerInfo *root)
 	}
 
 	/*
-	 * The PlannerInfo for each child is obtained by translating the relevant
-	 * members of the PlannerInfo for its immediate parent, which we find
-	 * using the parent_relid in its AppendRelInfo.  We save the PlannerInfo
-	 * for each parent in an array indexed by relid for fast retrieval. Since
-	 * the maximum number of parents is limited by the number of RTEs in the
-	 * query, we use that number to allocate the array. An extra entry is
-	 * needed since relids start from 1.
+	 * Before generating the real per-child-relation plans, do a cycle of
+	 * planning as though the query were a SELECT.  The objective here is to
+	 * find out which child relations need to be processed, using the same
+	 * expansion and pruning logic as for a SELECT.  We'll then pull out the
+	 * RangeTblEntry-s generated for the child rels, and make use of the
+	 * AppendRelInfo entries for them to guide the real planning.  (This is
+	 * rather inefficient; we could perhaps stop short of making a full Path
+	 * tree.  But this whole function is inefficient and slated for
+	 * destruction, so let's not contort query_planner for that.)
+	 */
+	{
+		PlannerInfo *subroot;
+
+		/*
+		 * Flat-copy the PlannerInfo to prevent modification of the original.
+		 */
+		subroot = makeNode(PlannerInfo);
+		memcpy(subroot, root, sizeof(PlannerInfo));
+
+		/*
+		 * Make a deep copy of the parsetree for this planning cycle to mess
+		 * around with, and change it to look like a SELECT.  (Hack alert: the
+		 * target RTE still has updatedCols set if this is an UPDATE, so that
+		 * expand_partitioned_rtentry will correctly update
+		 * subroot->partColsUpdated.)
+		 */
+		subroot->parse = copyObject(root->parse);
+
+		subroot->parse->commandType = CMD_SELECT;
+		subroot->parse->resultRelation = 0;
+
+		/*
+		 * Ensure the subroot has its own copy of the original
+		 * append_rel_list, since it'll be scribbled on.  (Note that at this
+		 * point, the list only contains AppendRelInfos for flattened UNION
+		 * ALL subqueries.)
+		 */
+		subroot->append_rel_list = copyObject(root->append_rel_list);
+
+		/*
+		 * Better make a private copy of the rowMarks, too.
+		 */
+		subroot->rowMarks = copyObject(root->rowMarks);
+
+		/* There shouldn't be any OJ info to translate, as yet */
+		Assert(subroot->join_info_list == NIL);
+		/* and we haven't created PlaceHolderInfos, either */
+		Assert(subroot->placeholder_list == NIL);
+
+		/* Generate Path(s) for accessing this result relation */
+		grouping_planner(subroot, true, 0.0 /* retrieve all tuples */ );
+
+		/* Extract the info we need. */
+		select_rtable = subroot->parse->rtable;
+		select_appinfos = subroot->append_rel_list;
+
+		/*
+		 * We need to propagate partColsUpdated back, too.  (The later
+		 * planning cycles will not set this because they won't run
+		 * expand_partitioned_rtentry for the UPDATE target.)
+		 */
+		root->partColsUpdated = subroot->partColsUpdated;
+	}
+
+	/*----------
+	 * Since only one rangetable can exist in the final plan, we need to make
+	 * sure that it contains all the RTEs needed for any child plan.  This is
+	 * complicated by the need to use separate subquery RTEs for each child.
+	 * We arrange the final rtable as follows:
+	 * 1. All original rtable entries (with their original RT indexes).
+	 * 2. All the relation RTEs generated for children of the target table.
+	 * 3. Subquery RTEs for children after the first.  We need N * (K - 1)
+	 *    RT slots for this, if there are N subqueries and K child tables.
+	 * 4. Additional RTEs generated during the child planning runs, such as
+	 *    children of inheritable RTEs other than the target table.
+	 * We assume that each child planning run will create an identical set
+	 * of type-4 RTEs.
+	 *
+	 * So the next thing to do is append the type-2 RTEs (the target table's
+	 * children) to the original rtable.  We look through select_appinfos
+	 * to find them.
+	 *
+	 * To identify which AppendRelInfos are relevant as we thumb through
+	 * select_appinfos, we need to look for both direct and indirect children
+	 * of top_parentRTindex, so we use a bitmap of known parent relids.
+	 * expand_inherited_rtentry() always processes a parent before any of that
+	 * parent's children, so we should see an intermediate parent before its
+	 * children.
+	 *----------
+	 */
+	child_appinfos = NIL;
+	old_child_rtis = NIL;
+	new_child_rtis = NIL;
+	parent_relids = bms_make_singleton(top_parentRTindex);
+	foreach(lc, select_appinfos)
+	{
+		AppendRelInfo *appinfo = lfirst_node(AppendRelInfo, lc);
+		RangeTblEntry *child_rte;
+
+		/* append_rel_list contains all append rels; ignore others */
+		if (!bms_is_member(appinfo->parent_relid, parent_relids))
+			continue;
+
+		/* remember relevant AppendRelInfos for use below */
+		child_appinfos = lappend(child_appinfos, appinfo);
+
+		/* extract RTE for this child rel */
+		child_rte = rt_fetch(appinfo->child_relid, select_rtable);
+
+		/* and append it to the original rtable */
+		parse->rtable = lappend(parse->rtable, child_rte);
+
+		/* remember child's index in the SELECT rtable */
+		old_child_rtis = lappend_int(old_child_rtis, appinfo->child_relid);
+
+		/* and its new index in the final rtable */
+		new_child_rtis = lappend_int(new_child_rtis, list_length(parse->rtable));
+
+		/* if child is itself partitioned, update parent_relids */
+		if (child_rte->inh)
+		{
+			Assert(child_rte->relkind == RELKIND_PARTITIONED_TABLE);
+			parent_relids = bms_add_member(parent_relids, appinfo->child_relid);
+		}
+	}
+
+	/*
+	 * It's possible that the RTIs we just assigned for the child rels in the
+	 * final rtable are different from where they were in the SELECT query.
+	 * Adjust the AppendRelInfos so that they will correctly map RT indexes to
+	 * the final indexes.  We can do this left-to-right since no child rel's
+	 * final RT index could be greater than what it had in the SELECT query.
 	 */
-	parent_roots = (PlannerInfo **) palloc0((list_length(parse->rtable) + 1) *
-											sizeof(PlannerInfo *));
-	parent_roots[top_parentRTindex] = root;
+	forboth(lc, old_child_rtis, lc2, new_child_rtis)
+	{
+		int			old_child_rti = lfirst_int(lc);
+		int			new_child_rti = lfirst_int(lc2);
+
+		if (old_child_rti == new_child_rti)
+			continue;			/* nothing to do */
+
+		Assert(old_child_rti > new_child_rti);
+
+		ChangeVarNodes((Node *) child_appinfos,
+					   old_child_rti, new_child_rti, 0);
+	}
+
+	/*
+	 * Now set up rangetable entries for subqueries for additional children
+	 * (the first child will just use the original ones).  These all have to
+	 * look more or less real, or EXPLAIN will get unhappy; so we just make
+	 * them all clones of the original subqueries.
+	 */
+	next_subquery_rti = list_length(parse->rtable) + 1;
+	if (subqueryRTindexes != NULL)
+	{
+		int			n_children = list_length(child_appinfos);
+
+		while (n_children-- > 1)
+		{
+			int			oldrti = -1;
+
+			while ((oldrti = bms_next_member(subqueryRTindexes, oldrti)) >= 0)
+			{
+				RangeTblEntry *subqrte;
+
+				subqrte = rt_fetch(oldrti, parse->rtable);
+				parse->rtable = lappend(parse->rtable, copyObject(subqrte));
+			}
+		}
+	}
+
+	/*
+	 * The query for each child is obtained by translating the query for its
+	 * immediate parent, since the AppendRelInfo data we have shows deltas
+	 * between parents and children.  We use the parent_parses array to
+	 * remember the appropriate query trees.  This is indexed by parent relid.
+	 * Since the maximum number of parents is limited by the number of RTEs in
+	 * the SELECT query, we use that number to allocate the array.  An extra
+	 * entry is needed since relids start from 1.
+	 */
+	parent_parses = (Query **) palloc0((list_length(select_rtable) + 1) *
+									   sizeof(Query *));
+	parent_parses[top_parentRTindex] = parse;
 
 	/*
 	 * And now we can get on with generating a plan for each child table.
 	 */
-	foreach(lc, root->append_rel_list)
+	foreach(lc, child_appinfos)
 	{
 		AppendRelInfo *appinfo = lfirst_node(AppendRelInfo, lc);
+		Index		this_subquery_rti = next_subquery_rti;
+		Query	   *parent_parse;
 		PlannerInfo *subroot;
 		RangeTblEntry *child_rte;
 		RelOptInfo *sub_final_rel;
 		Path	   *subpath;
 
-		/* append_rel_list contains all append rels; ignore others */
-		if (!bms_is_member(appinfo->parent_relid, parent_relids))
-			continue;
-
 		/*
 		 * expand_inherited_rtentry() always processes a parent before any of
-		 * that parent's children, so the parent_root for this relation should
-		 * already be available.
+		 * that parent's children, so the parent query for this relation
+		 * should already be available.
 		 */
-		parent_root = parent_roots[appinfo->parent_relid];
-		Assert(parent_root != NULL);
-		parent_parse = parent_root->parse;
+		parent_parse = parent_parses[appinfo->parent_relid];
+		Assert(parent_parse != NULL);
 
 		/*
 		 * We need a working copy of the PlannerInfo so that we can control
 		 * propagation of information back to the main copy.
 		 */
 		subroot = makeNode(PlannerInfo);
-		memcpy(subroot, parent_root, sizeof(PlannerInfo));
+		memcpy(subroot, root, sizeof(PlannerInfo));
 
 		/*
 		 * Generate modified query with this rel as target.  We first apply
@@ -1324,7 +1488,7 @@ inheritance_planner(PlannerInfo *root)
 		 * then fool around with subquery RTEs.
 		 */
 		subroot->parse = (Query *)
-			adjust_appendrel_attrs(parent_root,
+			adjust_appendrel_attrs(subroot,
 								   (Node *) parent_parse,
 								   1, &appinfo);
 
@@ -1360,9 +1524,7 @@ inheritance_planner(PlannerInfo *root)
 		if (child_rte->inh)
 		{
 			Assert(child_rte->relkind == RELKIND_PARTITIONED_TABLE);
-			parent_relids = bms_add_member(parent_relids, appinfo->child_relid);
-			parent_roots[appinfo->child_relid] = subroot;
-
+			parent_parses[appinfo->child_relid] = subroot->parse;
 			continue;
 		}
 
@@ -1383,108 +1545,38 @@ inheritance_planner(PlannerInfo *root)
 		 * is used elsewhere in the plan, so using the original parent RTE
 		 * would give rise to confusing use of multiple aliases in EXPLAIN
 		 * output for what the user will think is the "same" table.  OTOH,
-		 * it's not a problem in the partitioned inheritance case, because the
-		 * duplicate child RTE added for the parent does not appear anywhere
-		 * else in the plan tree.
+		 * it's not a problem in the partitioned inheritance case, because
+		 * there is no duplicate RTE for the parent.
 		 */
 		if (nominalRelation < 0)
 			nominalRelation = appinfo->child_relid;
 
 		/*
-		 * The rowMarks list might contain references to subquery RTEs, so
-		 * make a copy that we can apply ChangeVarNodes to.  (Fortunately, the
-		 * executor doesn't need to see the modified copies --- we can just
-		 * pass it the original rowMarks list.)
+		 * As above, each child plan run needs its own append_rel_list and
+		 * rowmarks, which should start out as pristine copies of the
+		 * originals.  There can't be any references to UPDATE/DELETE target
+		 * rels in them; but there could be subquery references, which we'll
+		 * fix up in a moment.
 		 */
-		subroot->rowMarks = copyObject(parent_root->rowMarks);
+		subroot->append_rel_list = copyObject(root->append_rel_list);
+		subroot->rowMarks = copyObject(root->rowMarks);
 
 		/*
-		 * The append_rel_list likewise might contain references to subquery
-		 * RTEs (if any subqueries were flattenable UNION ALLs).  So prepare
-		 * to apply ChangeVarNodes to that, too.  As explained above, we only
-		 * want to copy items that actually contain such references; the rest
-		 * can just get linked into the subroot's append_rel_list.
-		 *
-		 * If we know there are no such references, we can just use the outer
-		 * append_rel_list unmodified.
-		 */
-		if (modifiableARIindexes != NULL)
-		{
-			ListCell   *lc2;
-
-			subroot->append_rel_list = NIL;
-			foreach(lc2, parent_root->append_rel_list)
-			{
-				AppendRelInfo *appinfo2 = lfirst_node(AppendRelInfo, lc2);
-
-				if (bms_is_member(appinfo2->child_relid, modifiableARIindexes))
-					appinfo2 = copyObject(appinfo2);
-
-				subroot->append_rel_list = lappend(subroot->append_rel_list,
-												   appinfo2);
-			}
-		}
-
-		/*
-		 * Add placeholders to the child Query's rangetable list to fill the
-		 * RT indexes already reserved for subqueries in previous children.
-		 * These won't be referenced, so there's no need to make them very
-		 * valid-looking.
-		 */
-		while (list_length(subroot->parse->rtable) < list_length(final_rtable))
-			subroot->parse->rtable = lappend(subroot->parse->rtable,
-											 makeNode(RangeTblEntry));
-
-		/*
-		 * If this isn't the first child Query, generate duplicates of all
-		 * subquery RTEs, and adjust Var numbering to reference the
-		 * duplicates. To simplify the loop logic, we scan the original rtable
-		 * not the copy just made by adjust_appendrel_attrs; that should be OK
-		 * since subquery RTEs couldn't contain any references to the target
-		 * rel.
+		 * If this isn't the first child Query, adjust Vars and jointree
+		 * entries to reference the appropriate set of subquery RTEs.
 		 */
 		if (final_rtable != NIL && subqueryRTindexes != NULL)
 		{
-			ListCell   *lr;
+			int			oldrti = -1;
 
-			rti = 1;
-			foreach(lr, parent_parse->rtable)
+			while ((oldrti = bms_next_member(subqueryRTindexes, oldrti)) >= 0)
 			{
-				RangeTblEntry *rte = lfirst_node(RangeTblEntry, lr);
-
-				if (bms_is_member(rti, subqueryRTindexes))
-				{
-					Index		newrti;
-
-					/*
-					 * The RTE can't contain any references to its own RT
-					 * index, except in its securityQuals, so we can save a
-					 * few cycles by applying ChangeVarNodes to the rest of
-					 * the rangetable before we append the RTE to it.
-					 */
-					newrti = list_length(subroot->parse->rtable) + 1;
-					ChangeVarNodes((Node *) subroot->parse, rti, newrti, 0);
-					ChangeVarNodes((Node *) subroot->rowMarks, rti, newrti, 0);
-					/* Skip processing unchanging parts of append_rel_list */
-					if (modifiableARIindexes != NULL)
-					{
-						ListCell   *lc2;
-
-						foreach(lc2, subroot->append_rel_list)
-						{
-							AppendRelInfo *appinfo2 = lfirst_node(AppendRelInfo, lc2);
+				Index		newrti = next_subquery_rti++;
 
-							if (bms_is_member(appinfo2->child_relid,
-											  modifiableARIindexes))
-								ChangeVarNodes((Node *) appinfo2, rti, newrti, 0);
-						}
-					}
-					rte = copyObject(rte);
-					ChangeVarNodes((Node *) rte->securityQuals, rti, newrti, 0);
-					subroot->parse->rtable = lappend(subroot->parse->rtable,
-													 rte);
-				}
-				rti++;
+				ChangeVarNodes((Node *) subroot->parse, oldrti, newrti, 0);
+				ChangeVarNodes((Node *) subroot->append_rel_list,
+							   oldrti, newrti, 0);
+				ChangeVarNodes((Node *) subroot->rowMarks, oldrti, newrti, 0);
 			}
 		}
 
@@ -1514,22 +1606,43 @@ inheritance_planner(PlannerInfo *root)
 
 		/*
 		 * If this is the first non-excluded child, its post-planning rtable
-		 * becomes the initial contents of final_rtable; otherwise, append
-		 * just its modified subquery RTEs to final_rtable.
+		 * becomes the initial contents of final_rtable; otherwise, copy its
+		 * modified subquery RTEs into final_rtable, to ensure we have sane
+		 * copies of those.  Also save the first non-excluded child's version
+		 * of the rowmarks list; we assume all children will end up with
+		 * equivalent versions of that.
 		 */
 		if (final_rtable == NIL)
+		{
 			final_rtable = subroot->parse->rtable;
+			final_rowmarks = subroot->rowMarks;
+		}
 		else
-			final_rtable = list_concat(final_rtable,
-									   list_copy_tail(subroot->parse->rtable,
-													  list_length(final_rtable)));
+		{
+			Assert(list_length(final_rtable) ==
+				   list_length(subroot->parse->rtable));
+			if (subqueryRTindexes != NULL)
+			{
+				int			oldrti = -1;
+
+				while ((oldrti = bms_next_member(subqueryRTindexes, oldrti)) >= 0)
+				{
+					Index		newrti = this_subquery_rti++;
+					RangeTblEntry *subqrte;
+					ListCell   *newrticell;
+
+					subqrte = rt_fetch(newrti, subroot->parse->rtable);
+					newrticell = list_nth_cell(final_rtable, newrti - 1);
+					lfirst(newrticell) = subqrte;
+				}
+			}
+		}
 
 		/*
 		 * We need to collect all the RelOptInfos from all child plans into
 		 * the main PlannerInfo, since setrefs.c will need them.  We use the
-		 * last child's simple_rel_array (previous ones are too short), so we
-		 * have to propagate forward the RelOptInfos that were already built
-		 * in previous children.
+		 * last child's simple_rel_array, so we have to propagate forward the
+		 * RelOptInfos that were already built in previous children.
 		 */
 		Assert(subroot->simple_rel_array_size >= save_rel_array_size);
 		for (rti = 1; rti < save_rel_array_size; rti++)
@@ -1543,7 +1656,11 @@ inheritance_planner(PlannerInfo *root)
 		save_rel_array = subroot->simple_rel_array;
 		save_append_rel_array = subroot->append_rel_array;
 
-		/* Make sure any initplans from this rel get into the outer list */
+		/*
+		 * Make sure any initplans from this rel get into the outer list. Note
+		 * we're effectively assuming all children generate the same
+		 * init_plans.
+		 */
 		root->init_plans = subroot->init_plans;
 
 		/* Build list of sub-paths */
@@ -1626,6 +1743,9 @@ inheritance_planner(PlannerInfo *root)
 
 			root->simple_rte_array[rti++] = rte;
 		}
+
+		/* Put back adjusted rowmarks, too */
+		root->rowMarks = final_rowmarks;
 	}
 
 	/*
@@ -6128,8 +6248,9 @@ plan_create_index_workers(Oid tableOid, Oid indexOid)
 	 * Build a minimal RTE.
 	 *
 	 * Set the target's table to be an inheritance parent.  This is a kludge
-	 * that prevents problems within get_relation_info(), which does not
-	 * expect that any IndexOptInfo is currently undergoing REINDEX.
+	 * to prevent get_relation_info() from fetching index information, which
+	 * is needed because it does not expect that any IndexOptInfo is currently
+	 * undergoing REINDEX.
 	 */
 	rte = makeNode(RangeTblEntry);
 	rte->rtekind = RTE_RELATION;
@@ -6993,6 +7114,10 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
 			List	   *child_scanjoin_targets = NIL;
 			ListCell   *lc;
 
+			/* Skip processing pruned partitions. */
+			if (child_rel == NULL)
+				continue;
+
 			/* Translate scan/join targets for this child. */
 			appinfos = find_appinfos_by_relids(root, child_rel->relids,
 											   &nappinfos);
@@ -7093,6 +7218,10 @@ create_partitionwise_grouping_paths(PlannerInfo *root,
 		RelOptInfo *child_grouped_rel;
 		RelOptInfo *child_partially_grouped_rel;
 
+		/* Skip processing pruned partitions. */
+		if (child_input_rel == NULL)
+			continue;
+
 		/* Input child rel must have a path */
 		Assert(child_input_rel->pathlist != NIL);
 
diff --git a/src/backend/optimizer/prep/preptlist.c b/src/backend/optimizer/prep/preptlist.c
index 5392d1a..66e6ad9 100644
--- a/src/backend/optimizer/prep/preptlist.c
+++ b/src/backend/optimizer/prep/preptlist.c
@@ -121,7 +121,9 @@ preprocess_targetlist(PlannerInfo *root)
 	/*
 	 * Add necessary junk columns for rowmarked rels.  These values are needed
 	 * for locking of rels selected FOR UPDATE/SHARE, and to do EvalPlanQual
-	 * rechecking.  See comments for PlanRowMark in plannodes.h.
+	 * rechecking.  See comments for PlanRowMark in plannodes.h.  If you
+	 * change this stanza, see also expand_inherited_rtentry(), which has to
+	 * be able to add on junk columns equivalent to these.
 	 */
 	foreach(lc, root->rowMarks)
 	{
diff --git a/src/backend/optimizer/util/inherit.c b/src/backend/optimizer/util/inherit.c
index 1d1e506..31c1bec 100644
--- a/src/backend/optimizer/util/inherit.c
+++ b/src/backend/optimizer/util/inherit.c
@@ -18,110 +18,72 @@
 #include "access/table.h"
 #include "catalog/partition.h"
 #include "catalog/pg_inherits.h"
+#include "catalog/pg_type.h"
 #include "miscadmin.h"
+#include "nodes/makefuncs.h"
 #include "optimizer/appendinfo.h"
 #include "optimizer/inherit.h"
+#include "optimizer/optimizer.h"
+#include "optimizer/pathnode.h"
+#include "optimizer/planmain.h"
 #include "optimizer/planner.h"
 #include "optimizer/prep.h"
+#include "optimizer/restrictinfo.h"
+#include "parser/parsetree.h"
 #include "partitioning/partdesc.h"
+#include "partitioning/partprune.h"
 #include "utils/rel.h"
 
 
-static void expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte,
-						 Index rti);
-static void expand_partitioned_rtentry(PlannerInfo *root,
+static void expand_partitioned_rtentry(PlannerInfo *root, RelOptInfo *relinfo,
 						   RangeTblEntry *parentrte,
 						   Index parentRTindex, Relation parentrel,
-						   PlanRowMark *top_parentrc, LOCKMODE lockmode,
-						   List **appinfos);
+						   PlanRowMark *top_parentrc, LOCKMODE lockmode);
 static void expand_single_inheritance_child(PlannerInfo *root,
 								RangeTblEntry *parentrte,
 								Index parentRTindex, Relation parentrel,
 								PlanRowMark *top_parentrc, Relation childrel,
-								List **appinfos, RangeTblEntry **childrte_p,
+								RangeTblEntry **childrte_p,
 								Index *childRTindex_p);
 static Bitmapset *translate_col_privs(const Bitmapset *parent_privs,
 					List *translated_vars);
 
 
 /*
- * expand_inherited_tables
- *		Expand each rangetable entry that represents an inheritance set
- *		into an "append relation".  At the conclusion of this process,
- *		the "inh" flag is set in all and only those RTEs that are append
- *		relation parents.
- */
-void
-expand_inherited_tables(PlannerInfo *root)
-{
-	Index		nrtes;
-	Index		rti;
-	ListCell   *rl;
-
-	/*
-	 * expand_inherited_rtentry may add RTEs to parse->rtable. The function is
-	 * expected to recursively handle any RTEs that it creates with inh=true.
-	 * So just scan as far as the original end of the rtable list.
-	 */
-	nrtes = list_length(root->parse->rtable);
-	rl = list_head(root->parse->rtable);
-	for (rti = 1; rti <= nrtes; rti++)
-	{
-		RangeTblEntry *rte = (RangeTblEntry *) lfirst(rl);
-
-		expand_inherited_rtentry(root, rte, rti);
-		rl = lnext(rl);
-	}
-}
-
-/*
  * expand_inherited_rtentry
- *		Check whether a rangetable entry represents an inheritance set.
- *		If so, add entries for all the child tables to the query's
- *		rangetable, and build AppendRelInfo nodes for all the child tables
- *		and add them to root->append_rel_list.  If not, clear the entry's
- *		"inh" flag to prevent later code from looking for AppendRelInfos.
+ *		The given rangetable entry represents an inheritance set.
+ *		Add entries for all the child tables to the query's rangetable,
+ *		and build additional planner data structures for them, including
+ *		RelOptInfos, AppendRelInfos, and possibly PlanRowMarks.
  *
- * Note that the original RTE is considered to represent the whole
- * inheritance set.  The first of the generated RTEs is an RTE for the same
- * table, but with inh = false, to represent the parent table in its role
- * as a simple member of the inheritance set.
- *
- * A childless table is never considered to be an inheritance set. For
- * regular inheritance, a parent RTE must always have at least two associated
- * AppendRelInfos: one corresponding to the parent table as a simple member of
- * the inheritance set and one or more corresponding to the actual children.
- * (But a partitioned table might have only one associated AppendRelInfo,
- * since it's not itself scanned and hence doesn't need a second RTE to
- * represent itself as a member of the set.)
+ * Note that the original RTE is considered to represent the whole inheritance
+ * set.  In the case of traditional inheritance, the first of the generated
+ * RTEs is an RTE for the same table, but with inh = false, to represent the
+ * parent table in its role as a simple member of the inheritance set.  For
+ * partitioning, we don't need a second RTE because the partitioned table
+ * itself has no data and need not be scanned.
  */
-static void
-expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
+void
+expand_inherited_rtentry(PlannerInfo *root, RelOptInfo *rel,
+						 RangeTblEntry *rte, Index rti)
 {
 	Oid			parentOID;
-	PlanRowMark *oldrc;
 	Relation	oldrelation;
 	LOCKMODE	lockmode;
-	List	   *inhOIDs;
-	ListCell   *l;
+	PlanRowMark *oldrc;
+	bool		old_isParent = false;
+	int			old_allMarkTypes = 0;
+
+	/* Should only come here for plain relations with inh bit set */
+	Assert(rte->inh);
+	Assert(rte->rtekind == RTE_RELATION);
 
-	/* Does RT entry allow inheritance? */
-	if (!rte->inh)
-		return;
-	/* Ignore any already-expanded UNION ALL nodes */
-	if (rte->rtekind != RTE_RELATION)
-	{
-		Assert(rte->rtekind == RTE_SUBQUERY);
-		return;
-	}
-	/* Fast path for common case of childless table */
 	parentOID = rte->relid;
-	if (!has_subclass(parentOID))
-	{
-		/* Clear flag before returning */
-		rte->inh = false;
-		return;
-	}
+
+	/*
+	 * We used to check has_subclass() here, but there's no longer any need
+	 * to, because subquery_planner already did.
+	 */
 
 	/*
 	 * The rewriter should already have obtained an appropriate lock on each
@@ -141,7 +103,12 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 	 */
 	oldrc = get_plan_rowmark(root->rowMarks, rti);
 	if (oldrc)
+	{
+		old_isParent = oldrc->isParent;
 		oldrc->isParent = true;
+		/* Save initial value of allMarkTypes before children add to it */
+		old_allMarkTypes = oldrc->allMarkTypes;
+	}
 
 	/* Scan the inheritance set and expand it */
 	if (oldrelation->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
@@ -151,17 +118,12 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 		 */
 		Assert(rte->relkind == RELKIND_PARTITIONED_TABLE);
 
-		if (root->glob->partition_directory == NULL)
-			root->glob->partition_directory =
-				CreatePartitionDirectory(CurrentMemoryContext);
-
 		/*
-		 * If this table has partitions, recursively expand and lock them.
-		 * While at it, also extract the partition key columns of all the
-		 * partitioned tables.
+		 * Recursively expand and lock the partitions.  While at it, also
+		 * extract the partition key columns of all the partitioned tables.
 		 */
-		expand_partitioned_rtentry(root, rte, rti, oldrelation, oldrc,
-								   lockmode, &root->append_rel_list);
+		expand_partitioned_rtentry(root, rel, rte, rti,
+								   oldrelation, oldrc, lockmode);
 	}
 	else
 	{
@@ -170,25 +132,25 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 		 * that partitioned tables are not allowed to have inheritance
 		 * children, so it's not possible for both cases to apply.)
 		 */
-		List	   *appinfos = NIL;
-		RangeTblEntry *childrte;
-		Index		childRTindex;
+		List	   *inhOIDs;
+		ListCell   *l;
 
 		/* Scan for all members of inheritance set, acquire needed locks */
 		inhOIDs = find_all_inheritors(parentOID, lockmode, NULL);
 
 		/*
-		 * Check that there's at least one descendant, else treat as no-child
-		 * case.  This could happen despite above has_subclass() check, if the
-		 * table once had a child but no longer does.
+		 * We used to special-case the situation where the table no longer has
+		 * any children, by clearing rte->inh and exiting.  That no longer
+		 * works, because this function doesn't get run until after decisions
+		 * have been made that depend on rte->inh.  We have to treat such
+		 * situations as normal inheritance.  The table itself should always
+		 * have been found, though.
 		 */
-		if (list_length(inhOIDs) < 2)
-		{
-			/* Clear flag before returning */
-			rte->inh = false;
-			heap_close(oldrelation, NoLock);
-			return;
-		}
+		Assert(inhOIDs != NIL);
+		Assert(linitial_oid(inhOIDs) == parentOID);
+
+		/* Expand simple_rel_array and friends to hold child objects. */
+		expand_planner_arrays(root, list_length(inhOIDs));
 
 		/*
 		 * Expand inheritance children in the order the OIDs were returned by
@@ -198,6 +160,8 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 		{
 			Oid			childOID = lfirst_oid(l);
 			Relation	newrelation;
+			RangeTblEntry *childrte;
+			Index		childRTindex;
 
 			/* Open rel if needed; we already have required locks */
 			if (childOID != parentOID)
@@ -217,29 +181,78 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 				continue;
 			}
 
-			expand_single_inheritance_child(root, rte, rti, oldrelation, oldrc,
-											newrelation,
-											&appinfos, &childrte,
-											&childRTindex);
+			/* Create RTE and AppendRelInfo, plus PlanRowMark if needed. */
+			expand_single_inheritance_child(root, rte, rti, oldrelation,
+											oldrc, newrelation,
+											&childrte, &childRTindex);
+
+			/* Create the otherrel RelOptInfo too. */
+			(void) build_simple_rel(root, childRTindex, rel);
 
 			/* Close child relations, but keep locks */
 			if (childOID != parentOID)
 				table_close(newrelation, NoLock);
 		}
+	}
+
+	/*
+	 * Some children might require different mark types, which would've been
+	 * reported into oldrc.  If so, add relevant entries to the top-level
+	 * targetlist and update parent rel's reltarget.  This should match what
+	 * preprocess_targetlist() would have added if the mark types had been
+	 * requested originally.
+	 */
+	if (oldrc)
+	{
+		int			new_allMarkTypes = oldrc->allMarkTypes;
+		Var		   *var;
+		TargetEntry *tle;
+		char		resname[32];
+		List	   *newvars = NIL;
+
+		/* The old PlanRowMark should already have necessitated adding TID */
+		Assert(old_allMarkTypes & ~(1 << ROW_MARK_COPY));
+
+		/* Add whole-row junk Var if needed, unless we had it already */
+		if ((new_allMarkTypes & (1 << ROW_MARK_COPY)) &&
+			!(old_allMarkTypes & (1 << ROW_MARK_COPY)))
+		{
+			var = makeWholeRowVar(planner_rt_fetch(oldrc->rti, root),
+								  oldrc->rti,
+								  0,
+								  false);
+			snprintf(resname, sizeof(resname), "wholerow%u", oldrc->rowmarkId);
+			tle = makeTargetEntry((Expr *) var,
+								  list_length(root->processed_tlist) + 1,
+								  pstrdup(resname),
+								  true);
+			root->processed_tlist = lappend(root->processed_tlist, tle);
+			newvars = lappend(newvars, var);
+		}
+
+		/* Add tableoid junk Var, unless we had it already */
+		if (!old_isParent)
+		{
+			var = makeVar(oldrc->rti,
+						  TableOidAttributeNumber,
+						  OIDOID,
+						  -1,
+						  InvalidOid,
+						  0);
+			snprintf(resname, sizeof(resname), "tableoid%u", oldrc->rowmarkId);
+			tle = makeTargetEntry((Expr *) var,
+								  list_length(root->processed_tlist) + 1,
+								  pstrdup(resname),
+								  true);
+			root->processed_tlist = lappend(root->processed_tlist, tle);
+			newvars = lappend(newvars, var);
+		}
 
 		/*
-		 * If all the children were temp tables, pretend it's a
-		 * non-inheritance situation; we don't need Append node in that case.
-		 * The duplicate RTE we added for the parent table is harmless, so we
-		 * don't bother to get rid of it; ditto for the useless PlanRowMark
-		 * node.
+		 * Add the newly added Vars to parent's reltarget.  We needn't worry
+		 * about the childrens' reltargets, they'll be made later.
 		 */
-		if (list_length(appinfos) < 2)
-			rte->inh = false;
-		else
-			root->append_rel_list = list_concat(root->append_rel_list,
-												appinfos);
-
+		add_vars_to_targetlist(root, newvars, bms_make_singleton(0), false);
 	}
 
 	table_close(oldrelation, NoLock);
@@ -250,25 +263,36 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
  *		Recursively expand an RTE for a partitioned table.
  */
 static void
-expand_partitioned_rtentry(PlannerInfo *root, RangeTblEntry *parentrte,
+expand_partitioned_rtentry(PlannerInfo *root, RelOptInfo *relinfo,
+						   RangeTblEntry *parentrte,
 						   Index parentRTindex, Relation parentrel,
-						   PlanRowMark *top_parentrc, LOCKMODE lockmode,
-						   List **appinfos)
+						   PlanRowMark *top_parentrc, LOCKMODE lockmode)
 {
-	int			i;
-	RangeTblEntry *childrte;
-	Index		childRTindex;
 	PartitionDesc partdesc;
+	Bitmapset  *live_parts;
+	int			num_live_parts;
+	int			i;
+
+	check_stack_depth();
+
+	Assert(parentrte->inh);
 
 	partdesc = PartitionDirectoryLookup(root->glob->partition_directory,
 										parentrel);
 
-	check_stack_depth();
-
 	/* A partitioned table should always have a partition descriptor. */
 	Assert(partdesc);
 
-	Assert(parentrte->inh);
+	/*
+	 * If the partitioned table has no partitions, treat this as the
+	 * non-inheritance case.
+	 */
+	if (partdesc->nparts == 0)
+	{
+		/* XXX wrong? */
+		parentrte->inh = false;
+		return;
+	}
 
 	/*
 	 * Note down whether any partition key cols are being updated. Though it's
@@ -282,24 +306,40 @@ expand_partitioned_rtentry(PlannerInfo *root, RangeTblEntry *parentrte,
 			has_partition_attrs(parentrel, parentrte->updatedCols, NULL);
 
 	/*
-	 * If the partitioned table has no partitions, treat this as the
-	 * non-inheritance case.
+	 * Perform partition pruning using restriction clauses assigned to parent
+	 * relation.  live_parts will contain PartitionDesc indexes of partitions
+	 * that survive pruning.  Below, we will initialize child objects for the
+	 * surviving partitions.
 	 */
-	if (partdesc->nparts == 0)
-	{
-		parentrte->inh = false;
-		return;
-	}
+	live_parts = prune_append_rel_partitions(relinfo);
+
+	/* Expand simple_rel_array and friends to hold child objects. */
+	num_live_parts = bms_num_members(live_parts);
+	if (num_live_parts > 0)
+		expand_planner_arrays(root, num_live_parts);
 
 	/*
-	 * Create a child RTE for each partition.  Note that unlike traditional
-	 * inheritance, we don't need a child RTE for the partitioned table
-	 * itself, because it's not going to be scanned.
+	 * We also store partition RelOptInfo pointers in the parent relation.
+	 * Since we're palloc0'ing, slots corresponding to pruned partitions will
+	 * contain NULL.
 	 */
-	for (i = 0; i < partdesc->nparts; i++)
+	Assert(relinfo->part_rels == NULL);
+	relinfo->part_rels = (RelOptInfo **)
+		palloc0(relinfo->nparts * sizeof(RelOptInfo *));
+
+	/*
+	 * Create a child RTE for each live partition.  Note that unlike
+	 * traditional inheritance, we don't need a child RTE for the partitioned
+	 * table itself, because it's not going to be scanned.
+	 */
+	i = -1;
+	while ((i = bms_next_member(live_parts, i)) >= 0)
 	{
 		Oid			childOID = partdesc->oids[i];
 		Relation	childrel;
+		RangeTblEntry *childrte;
+		Index		childRTindex;
+		RelOptInfo *childrelinfo;
 
 		/* Open rel, acquiring required locks */
 		childrel = table_open(childOID, lockmode);
@@ -312,15 +352,20 @@ expand_partitioned_rtentry(PlannerInfo *root, RangeTblEntry *parentrte,
 		if (RELATION_IS_OTHER_TEMP(childrel))
 			elog(ERROR, "temporary relation from another session found as partition");
 
+		/* Create RTE and AppendRelInfo, plus PlanRowMark if needed. */
 		expand_single_inheritance_child(root, parentrte, parentRTindex,
 										parentrel, top_parentrc, childrel,
-										appinfos, &childrte, &childRTindex);
+										&childrte, &childRTindex);
+
+		/* Create the otherrel RelOptInfo too. */
+		childrelinfo = build_simple_rel(root, childRTindex, relinfo);
+		relinfo->part_rels[i] = childrelinfo;
 
 		/* If this child is itself partitioned, recurse */
 		if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
-			expand_partitioned_rtentry(root, childrte, childRTindex,
-									   childrel, top_parentrc, lockmode,
-									   appinfos);
+			expand_partitioned_rtentry(root, childrelinfo,
+									   childrte, childRTindex,
+									   childrel, top_parentrc, lockmode);
 
 		/* Close child relation, but keep locks */
 		table_close(childrel, NoLock);
@@ -351,7 +396,7 @@ static void
 expand_single_inheritance_child(PlannerInfo *root, RangeTblEntry *parentrte,
 								Index parentRTindex, Relation parentrel,
 								PlanRowMark *top_parentrc, Relation childrel,
-								List **appinfos, RangeTblEntry **childrte_p,
+								RangeTblEntry **childrte_p,
 								Index *childRTindex_p)
 {
 	Query	   *parse = root->parse;
@@ -363,8 +408,8 @@ expand_single_inheritance_child(PlannerInfo *root, RangeTblEntry *parentrte,
 
 	/*
 	 * Build an RTE for the child, and attach to query's rangetable list. We
-	 * copy most fields of the parent's RTE, but replace relation OID and
-	 * relkind, and set inh = false.  Also, set requiredPerms to zero since
+	 * copy most fields of the parent's RTE, but replace relation OID,
+	 * relkind, and inh for the child.  Also, set requiredPerms to zero since
 	 * all required permissions checks are done on the original RTE. Likewise,
 	 * set the child's securityQuals to empty, because we only want to apply
 	 * the parent's RLS conditions regardless of what RLS properties
@@ -396,7 +441,7 @@ expand_single_inheritance_child(PlannerInfo *root, RangeTblEntry *parentrte,
 	 */
 	appinfo = make_append_rel_info(parentrel, childrel,
 								   parentRTindex, childRTindex);
-	*appinfos = lappend(*appinfos, appinfo);
+	root->append_rel_list = lappend(root->append_rel_list, appinfo);
 
 	/*
 	 * Translate the column permissions bitmaps to the child's attnums (we
@@ -418,6 +463,16 @@ expand_single_inheritance_child(PlannerInfo *root, RangeTblEntry *parentrte,
 	}
 
 	/*
+	 * Store the RTE and appinfo in the respective PlannerInfo arrays, which
+	 * the caller must already have allocated space for.
+	 */
+	Assert(childRTindex < root->simple_rel_array_size);
+	Assert(root->simple_rte_array[childRTindex] == NULL);
+	root->simple_rte_array[childRTindex] = childrte;
+	Assert(root->append_rel_array[childRTindex] == NULL);
+	root->append_rel_array[childRTindex] = appinfo;
+
+	/*
 	 * Build a PlanRowMark if parent is marked FOR UPDATE/SHARE.
 	 */
 	if (top_parentrc)
@@ -437,7 +492,7 @@ expand_single_inheritance_child(PlannerInfo *root, RangeTblEntry *parentrte,
 		/*
 		 * We mark RowMarks for partitioned child tables as parent RowMarks so
 		 * that the executor ignores them (except their existence means that
-		 * the child tables be locked using appropriate mode).
+		 * the child tables will be locked using the appropriate mode).
 		 */
 		childrc->isParent = (childrte->relkind == RELKIND_PARTITIONED_TABLE);
 
@@ -499,3 +554,129 @@ translate_col_privs(const Bitmapset *parent_privs,
 
 	return child_privs;
 }
+
+
+/*
+ * apply_child_basequals
+ *		Populate childrel's base restriction quals from parent rel's quals,
+ *		translating them using appinfo.
+ *
+ * If any of the resulting clauses evaluate to constant false or NULL, we
+ * return false and don't apply any quals.  Caller should mark the relation as
+ * a dummy rel in this case, since it doesn't need to be scanned.
+ */
+bool
+apply_child_basequals(PlannerInfo *root, RelOptInfo *parentrel,
+					  RelOptInfo *childrel, RangeTblEntry *childRTE,
+					  AppendRelInfo *appinfo)
+{
+	List	   *childquals;
+	Index		cq_min_security;
+	ListCell   *lc;
+
+	/*
+	 * The child rel's targetlist might contain non-Var expressions, which
+	 * means that substitution into the quals could produce opportunities for
+	 * const-simplification, and perhaps even pseudoconstant quals. Therefore,
+	 * transform each RestrictInfo separately to see if it reduces to a
+	 * constant or pseudoconstant.  (We must process them separately to keep
+	 * track of the security level of each qual.)
+	 */
+	childquals = NIL;
+	cq_min_security = UINT_MAX;
+	foreach(lc, parentrel->baserestrictinfo)
+	{
+		RestrictInfo *rinfo = (RestrictInfo *) lfirst(lc);
+		Node	   *childqual;
+		ListCell   *lc2;
+
+		Assert(IsA(rinfo, RestrictInfo));
+		childqual = adjust_appendrel_attrs(root,
+										   (Node *) rinfo->clause,
+										   1, &appinfo);
+		childqual = eval_const_expressions(root, childqual);
+		/* check for flat-out constant */
+		if (childqual && IsA(childqual, Const))
+		{
+			if (((Const *) childqual)->constisnull ||
+				!DatumGetBool(((Const *) childqual)->constvalue))
+			{
+				/* Restriction reduces to constant FALSE or NULL */
+				return false;
+			}
+			/* Restriction reduces to constant TRUE, so drop it */
+			continue;
+		}
+		/* might have gotten an AND clause, if so flatten it */
+		foreach(lc2, make_ands_implicit((Expr *) childqual))
+		{
+			Node	   *onecq = (Node *) lfirst(lc2);
+			bool		pseudoconstant;
+
+			/* check for pseudoconstant (no Vars or volatile functions) */
+			pseudoconstant =
+				!contain_vars_of_level(onecq, 0) &&
+				!contain_volatile_functions(onecq);
+			if (pseudoconstant)
+			{
+				/* tell createplan.c to check for gating quals */
+				root->hasPseudoConstantQuals = true;
+			}
+			/* reconstitute RestrictInfo with appropriate properties */
+			childquals = lappend(childquals,
+								 make_restrictinfo((Expr *) onecq,
+												   rinfo->is_pushed_down,
+												   rinfo->outerjoin_delayed,
+												   pseudoconstant,
+												   rinfo->security_level,
+												   NULL, NULL, NULL));
+			/* track minimum security level among child quals */
+			cq_min_security = Min(cq_min_security, rinfo->security_level);
+		}
+	}
+
+	/*
+	 * In addition to the quals inherited from the parent, we might have
+	 * securityQuals associated with this particular child node.  (Currently
+	 * this can only happen in appendrels originating from UNION ALL;
+	 * inheritance child tables don't have their own securityQuals, see
+	 * expand_inherited_rtentry().)  Pull any such securityQuals up into the
+	 * baserestrictinfo for the child.  This is similar to
+	 * process_security_barrier_quals() for the parent rel, except that we
+	 * can't make any general deductions from such quals, since they don't
+	 * hold for the whole appendrel.
+	 */
+	if (childRTE->securityQuals)
+	{
+		Index		security_level = 0;
+
+		foreach(lc, childRTE->securityQuals)
+		{
+			List	   *qualset = (List *) lfirst(lc);
+			ListCell   *lc2;
+
+			foreach(lc2, qualset)
+			{
+				Expr	   *qual = (Expr *) lfirst(lc2);
+
+				/* not likely that we'd see constants here, so no check */
+				childquals = lappend(childquals,
+									 make_restrictinfo(qual,
+													   true, false, false,
+													   security_level,
+													   NULL, NULL, NULL));
+				cq_min_security = Min(cq_min_security, security_level);
+			}
+			security_level++;
+		}
+		Assert(security_level <= root->qual_security_level);
+	}
+
+	/*
+	 * OK, we've got all the baserestrictinfo quals for this child.
+	 */
+	childrel->baserestrictinfo = childquals;
+	childrel->baserestrict_min_security = cq_min_security;
+
+	return true;
+}
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 702c4f8..89b281f 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -2082,7 +2082,10 @@ set_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
 {
 	PartitionDesc partdesc;
 
-	Assert(relation->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
+	/* Create the PartitionDirectory infrastructure if we didn't already */
+	if (root->glob->partition_directory == NULL)
+		root->glob->partition_directory =
+			CreatePartitionDirectory(CurrentMemoryContext);
 
 	partdesc = PartitionDirectoryLookup(root->glob->partition_directory,
 										relation);
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 0d40b8d..f0f1811 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -20,11 +20,11 @@
 #include "optimizer/appendinfo.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
+#include "optimizer/inherit.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
-#include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
 #include "optimizer/tlist.h"
 #include "partitioning/partbounds.h"
@@ -132,6 +132,49 @@ setup_append_rel_array(PlannerInfo *root)
 }
 
 /*
+ * expand_planner_arrays
+ *		Expand the PlannerInfo's per-RTE arrays by add_size members
+ *		and initialize the newly added entries to NULLs
+ */
+void
+expand_planner_arrays(PlannerInfo *root, int add_size)
+{
+	int			new_size;
+
+	Assert(add_size > 0);
+
+	new_size = root->simple_rel_array_size + add_size;
+
+	root->simple_rte_array = (RangeTblEntry **)
+		repalloc(root->simple_rte_array,
+				 sizeof(RangeTblEntry *) * new_size);
+	MemSet(root->simple_rte_array + root->simple_rel_array_size,
+		   0, sizeof(RangeTblEntry *) * add_size);
+
+	root->simple_rel_array = (RelOptInfo **)
+		repalloc(root->simple_rel_array,
+				 sizeof(RelOptInfo *) * new_size);
+	MemSet(root->simple_rel_array + root->simple_rel_array_size,
+		   0, sizeof(RelOptInfo *) * add_size);
+
+	if (root->append_rel_array)
+	{
+		root->append_rel_array = (AppendRelInfo **)
+			repalloc(root->append_rel_array,
+					 sizeof(AppendRelInfo *) * new_size);
+		MemSet(root->append_rel_array + root->simple_rel_array_size,
+			   0, sizeof(AppendRelInfo *) * add_size);
+	}
+	else
+	{
+		root->append_rel_array = (AppendRelInfo **)
+			palloc0(sizeof(AppendRelInfo *) * new_size);
+	}
+
+	root->simple_rel_array_size = new_size;
+}
+
+/*
  * build_simple_rel
  *	  Construct a new RelOptInfo for a base relation or 'other' relation.
  */
@@ -281,49 +324,60 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 			break;
 	}
 
-	/* Save the finished struct in the query's simple_rel_array */
-	root->simple_rel_array[relid] = rel;
-
 	/*
 	 * This is a convenient spot at which to note whether rels participating
 	 * in the query have any securityQuals attached.  If so, increase
 	 * root->qual_security_level to ensure it's larger than the maximum
-	 * security level needed for securityQuals.
+	 * security level needed for securityQuals.  (Must do this before we call
+	 * apply_child_basequals, else we'll hit an Assert therein.)
 	 */
 	if (rte->securityQuals)
 		root->qual_security_level = Max(root->qual_security_level,
 										list_length(rte->securityQuals));
 
+	/*
+	 * Copy the parent's quals to the child, with appropriate substitution of
+	 * variables.  If any constant false or NULL clauses turn up, we can mark
+	 * the child as dummy right away.  (We must do this immediately so that
+	 * pruning works correctly when recursing in expand_partitioned_rtentry.)
+	 */
+	if (parent)
+	{
+		AppendRelInfo *appinfo = root->append_rel_array[relid];
+
+		Assert(appinfo != NULL);
+		if (!apply_child_basequals(root, parent, rel, rte, appinfo))
+		{
+			/*
+			 * Some restriction clause reduced to constant FALSE or NULL after
+			 * substitution, so this child need not be scanned.
+			 */
+			mark_dummy_rel(rel);
+		}
+	}
+
+	/* Save the finished struct in the query's simple_rel_array */
+	root->simple_rel_array[relid] = rel;
+
 	return rel;
 }
 
 /*
- * add_appendrel_other_rels
+ * expand_appendrel_subquery
  *		Add "other rel" RelOptInfos for the children of an appendrel baserel
  *
- * "rel" is a relation that (still) has the rte->inh flag set, meaning it
- * has appendrel children listed in root->append_rel_list.  We need to build
+ * "rel" is a subquery relation that has the rte->inh flag set, meaning it
+ * is a UNION ALL subquery that's been flattened into an appendrel, with
+ * child subqueries listed in root->append_rel_list.  We need to build
  * a RelOptInfo for each child relation so that we can plan scans on them.
- * (The parent relation might be a partitioned table, a table with
- * traditional inheritance children, or a flattened UNION ALL subquery.)
  */
 void
-add_appendrel_other_rels(PlannerInfo *root, RelOptInfo *rel, Index rti)
+expand_appendrel_subquery(PlannerInfo *root, RelOptInfo *rel,
+						  RangeTblEntry *rte, Index rti)
 {
-	int			cnt_parts = 0;
 	ListCell   *l;
 
-	/*
-	 * If rel is a partitioned table, then we also need to build a part_rels
-	 * array so that the child RelOptInfos can be conveniently accessed from
-	 * the parent.
-	 */
-	if (rel->part_scheme != NULL)
-	{
-		Assert(rel->nparts > 0);
-		rel->part_rels = (RelOptInfo **)
-			palloc0(sizeof(RelOptInfo *) * rel->nparts);
-	}
+	Assert(rte->rtekind == RTE_SUBQUERY);
 
 	foreach(l, root->append_rel_list)
 	{
@@ -341,33 +395,18 @@ add_appendrel_other_rels(PlannerInfo *root, RelOptInfo *rel, Index rti)
 		childrte = root->simple_rte_array[childRTindex];
 		Assert(childrte != NULL);
 
-		/* build child RelOptInfo, and add to main query data structures */
+		/* Build the child RelOptInfo. */
 		childrel = build_simple_rel(root, childRTindex, rel);
 
-		/*
-		 * If rel is a partitioned table, fill in the part_rels array.  The
-		 * order in which child tables appear in append_rel_list is the same
-		 * as the order in which they appear in the parent's PartitionDesc, so
-		 * assigning partitions like this works.
-		 */
-		if (rel->part_scheme != NULL)
-		{
-			Assert(cnt_parts < rel->nparts);
-			rel->part_rels[cnt_parts++] = childrel;
-		}
-
-		/* Child may itself be an inherited relation. */
+		/* Child may itself be an inherited rel, either table or subquery. */
 		if (childrte->inh)
 		{
-			/* Only relation and subquery RTEs can have children. */
-			Assert(childrte->rtekind == RTE_RELATION ||
-				   childrte->rtekind == RTE_SUBQUERY);
-			add_appendrel_other_rels(root, childrel, childRTindex);
+			if (childrte->rtekind == RTE_RELATION)
+				expand_inherited_rtentry(root, childrel, childrte, childRTindex);
+			else
+				expand_appendrel_subquery(root, childrel, childrte, childRTindex);
 		}
 	}
-
-	/* We should have filled all of the part_rels array if it's partitioned */
-	Assert(cnt_parts == rel->nparts);
 }
 
 /*
diff --git a/src/backend/partitioning/partprune.c b/src/backend/partitioning/partprune.c
index af3f911..aecea82 100644
--- a/src/backend/partitioning/partprune.c
+++ b/src/backend/partitioning/partprune.c
@@ -45,6 +45,7 @@
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/appendinfo.h"
+#include "optimizer/cost.h"
 #include "optimizer/optimizer.h"
 #include "optimizer/pathnode.h"
 #include "parser/parsetree.h"
@@ -474,18 +475,24 @@ make_partitionedrel_pruneinfo(PlannerInfo *root, RelOptInfo *parentrel,
 		 * is, not pruned already).
 		 */
 		subplan_map = (int *) palloc(nparts * sizeof(int));
+		memset(subplan_map, -1, nparts * sizeof(int));
 		subpart_map = (int *) palloc(nparts * sizeof(int));
-		relid_map = (Oid *) palloc(nparts * sizeof(Oid));
+		memset(subpart_map, -1, nparts * sizeof(Oid));
+		relid_map = (Oid *) palloc0(nparts * sizeof(Oid));
 		present_parts = NULL;
 
 		for (i = 0; i < nparts; i++)
 		{
 			RelOptInfo *partrel = subpart->part_rels[i];
-			int			subplanidx = relid_subplan_map[partrel->relid] - 1;
-			int			subpartidx = relid_subpart_map[partrel->relid] - 1;
+			int			subplanidx;
+			int			subpartidx;
 
-			subplan_map[i] = subplanidx;
-			subpart_map[i] = subpartidx;
+			/* Skip processing pruned partitions. */
+			if (partrel == NULL)
+				continue;
+
+			subplan_map[i] = subplanidx = relid_subplan_map[partrel->relid] - 1;
+			subpart_map[i] = subpartidx = relid_subpart_map[partrel->relid] - 1;
 			relid_map[i] = planner_rt_fetch(partrel->relid, root)->relid;
 			if (subplanidx >= 0)
 			{
@@ -567,23 +574,20 @@ gen_partprune_steps(RelOptInfo *rel, List *clauses, bool *contradictory)
 
 /*
  * prune_append_rel_partitions
- *		Returns RT indexes of the minimum set of child partitions which must
- *		be scanned to satisfy rel's baserestrictinfo quals.
+ *		Returns indexes into rel->part_rels of the minimum set of child
+ *		partitions which must be scanned to satisfy rel's baserestrictinfo
+ *		quals.
  *
  * Callers must ensure that 'rel' is a partitioned table.
  */
-Relids
+Bitmapset *
 prune_append_rel_partitions(RelOptInfo *rel)
 {
-	Relids		result;
 	List	   *clauses = rel->baserestrictinfo;
 	List	   *pruning_steps;
 	bool		contradictory;
 	PartitionPruneContext context;
-	Bitmapset  *partindexes;
-	int			i;
 
-	Assert(clauses != NIL);
 	Assert(rel->part_scheme != NULL);
 
 	/* If there are no partitions, return the empty set */
@@ -591,6 +595,13 @@ prune_append_rel_partitions(RelOptInfo *rel)
 		return NULL;
 
 	/*
+	 * If pruning is disabled or if there are no clauses to prune with, return
+	 * all partitions.
+	 */
+	if (!enable_partition_pruning || clauses == NIL)
+		return bms_add_range(NULL, 0, rel->nparts - 1);
+
+	/*
 	 * Process clauses.  If the clauses are found to be contradictory, we can
 	 * return the empty set.
 	 */
@@ -617,15 +628,7 @@ prune_append_rel_partitions(RelOptInfo *rel)
 	context.evalexecparams = false;
 
 	/* Actual pruning happens here. */
-	partindexes = get_matching_partitions(&context, pruning_steps);
-
-	/* Add selected partitions' RT indexes to result. */
-	i = -1;
-	result = NULL;
-	while ((i = bms_next_member(partindexes, i)) >= 0)
-		result = bms_add_member(result, rel->part_rels[i]->relid);
-
-	return result;
+	return get_matching_partitions(&context, pruning_steps);
 }
 
 /*
diff --git a/src/include/optimizer/inherit.h b/src/include/optimizer/inherit.h
index d2418f1..02a23e5 100644
--- a/src/include/optimizer/inherit.h
+++ b/src/include/optimizer/inherit.h
@@ -17,6 +17,11 @@
 #include "nodes/pathnodes.h"
 
 
-extern void expand_inherited_tables(PlannerInfo *root);
+extern void expand_inherited_rtentry(PlannerInfo *root, RelOptInfo *rel,
+						 RangeTblEntry *rte, Index rti);
+
+extern bool apply_child_basequals(PlannerInfo *root, RelOptInfo *parentrel,
+					  RelOptInfo *childrel, RangeTblEntry *childRTE,
+					  AppendRelInfo *appinfo);
 
 #endif							/* INHERIT_H */
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 9e79e1c..21d0e67 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -277,10 +277,11 @@ extern Path *reparameterize_path_by_child(PlannerInfo *root, Path *path,
  */
 extern void setup_simple_rel_arrays(PlannerInfo *root);
 extern void setup_append_rel_array(PlannerInfo *root);
+extern void expand_planner_arrays(PlannerInfo *root, int add_size);
 extern RelOptInfo *build_simple_rel(PlannerInfo *root, int relid,
 				 RelOptInfo *parent);
-extern void add_appendrel_other_rels(PlannerInfo *root, RelOptInfo *rel,
-						 Index rti);
+extern void expand_appendrel_subquery(PlannerInfo *root, RelOptInfo *rel,
+						  RangeTblEntry *rte, Index rti);
 extern RelOptInfo *find_base_rel(PlannerInfo *root, int relid);
 extern RelOptInfo *find_join_rel(PlannerInfo *root, Relids relids);
 extern RelOptInfo *build_join_rel(PlannerInfo *root,
diff --git a/src/test/regress/expected/partition_aggregate.out b/src/test/regress/expected/partition_aggregate.out
index 6bc1068..1450cef 100644
--- a/src/test/regress/expected/partition_aggregate.out
+++ b/src/test/regress/expected/partition_aggregate.out
@@ -144,7 +144,7 @@ SELECT c, sum(a) FROM pagg_tab WHERE 1 = 2 GROUP BY c;
            QUERY PLAN           
 --------------------------------
  HashAggregate
-   Group Key: pagg_tab.c
+   Group Key: c
    ->  Result
          One-Time Filter: false
 (4 rows)
@@ -159,7 +159,7 @@ SELECT c, sum(a) FROM pagg_tab WHERE c = 'x' GROUP BY c;
            QUERY PLAN           
 --------------------------------
  GroupAggregate
-   Group Key: pagg_tab.c
+   Group Key: c
    ->  Result
          One-Time Filter: false
 (4 rows)
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index 50ca03b..7806ba1 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -2568,6 +2568,60 @@ table ab;
  1 | 3
 (1 row)
 
+-- Test UPDATE where source relation has run-time pruning enabled
+truncate ab;
+insert into ab values (1, 1), (1, 2), (1, 3), (2, 1);
+explain (analyze, costs off, summary off, timing off)
+update ab_a1 set b = 3 from ab_a2 where ab_a2.b = (select 1);
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Update on ab_a1 (actual rows=0 loops=1)
+   Update on ab_a1_b1
+   Update on ab_a1_b2
+   Update on ab_a1_b3
+   InitPlan 1 (returns $0)
+     ->  Result (actual rows=1 loops=1)
+   ->  Nested Loop (actual rows=1 loops=1)
+         ->  Seq Scan on ab_a1_b1 (actual rows=1 loops=1)
+         ->  Materialize (actual rows=1 loops=1)
+               ->  Append (actual rows=1 loops=1)
+                     ->  Seq Scan on ab_a2_b1 (actual rows=1 loops=1)
+                           Filter: (b = $0)
+                     ->  Seq Scan on ab_a2_b2 (never executed)
+                           Filter: (b = $0)
+                     ->  Seq Scan on ab_a2_b3 (never executed)
+                           Filter: (b = $0)
+   ->  Nested Loop (actual rows=1 loops=1)
+         ->  Seq Scan on ab_a1_b2 (actual rows=1 loops=1)
+         ->  Materialize (actual rows=1 loops=1)
+               ->  Append (actual rows=1 loops=1)
+                     ->  Seq Scan on ab_a2_b1 (actual rows=1 loops=1)
+                           Filter: (b = $0)
+                     ->  Seq Scan on ab_a2_b2 (never executed)
+                           Filter: (b = $0)
+                     ->  Seq Scan on ab_a2_b3 (never executed)
+                           Filter: (b = $0)
+   ->  Nested Loop (actual rows=1 loops=1)
+         ->  Seq Scan on ab_a1_b3 (actual rows=1 loops=1)
+         ->  Materialize (actual rows=1 loops=1)
+               ->  Append (actual rows=1 loops=1)
+                     ->  Seq Scan on ab_a2_b1 (actual rows=1 loops=1)
+                           Filter: (b = $0)
+                     ->  Seq Scan on ab_a2_b2 (never executed)
+                           Filter: (b = $0)
+                     ->  Seq Scan on ab_a2_b3 (never executed)
+                           Filter: (b = $0)
+(36 rows)
+
+select tableoid::regclass, * from ab;
+ tableoid | a | b 
+----------+---+---
+ ab_a1_b3 | 1 | 3
+ ab_a1_b3 | 1 | 3
+ ab_a1_b3 | 1 | 3
+ ab_a2_b1 | 2 | 1
+(4 rows)
+
 drop table ab, lprt_a;
 -- Join
 create table tbl1(col1 int);
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index a5514c7..2e4d2b4 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -588,6 +588,13 @@ explain (analyze, costs off, summary off, timing off)
 update ab_a1 set b = 3 from ab where ab.a = 1 and ab.a = ab_a1.a;
 table ab;
 
+-- Test UPDATE where source relation has run-time pruning enabled
+truncate ab;
+insert into ab values (1, 1), (1, 2), (1, 3), (2, 1);
+explain (analyze, costs off, summary off, timing off)
+update ab_a1 set b = 3 from ab_a2 where ab_a2.b = (select 1);
+select tableoid::regclass, * from ab;
+
 drop table ab, lprt_a;
 
 -- Join
