diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index c83991c93c..f9a812480d 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -1349,7 +1349,11 @@ adjust_partition_tlist(List *tlist, TupleConversionMap *map)
  * difference between these that we need to concern ourselves with is the
  * time when the values of the Params are known.  External Param values are
  * known at any time of execution, including executor startup, but exec Param
- * values are only known when the executor is running.
+ * values are only known when the executor is running.  We also support
+ * pruning using any stable expression which does not contain any Vars.
+ * Immutable expressions would have been evaluated to a Const during planning,
+ * so plan-time pruning would have taken care of any pruning.  It's not
+ * possible for pruning to take place using volatile expressions.
  *
  * For external Params we may be able to prune away unneeded partitions
  * during executor startup.  This has the added benefit of not having to
@@ -1418,6 +1422,7 @@ ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo)
 	prunestate->num_partprunedata = list_length(partitionpruneinfo);
 	prunestate->extparams = NULL;
 	prunestate->execparams = NULL;
+	prunestate->hasparamlessexprs = false;
 
 	/*
 	 * Create a sub memory context which we'll use when making calls to the
@@ -1513,6 +1518,7 @@ ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo)
 		pprune->pruning_steps = pinfo->pruning_steps;
 		pprune->extparams = bms_copy(pinfo->extparams);
 		pprune->allparams = bms_union(pinfo->extparams, pinfo->execparams);
+		pprune->hasparamlessexprs = pinfo->hasparamlessexprs;
 
 		/*
 		 * Accumulate the paramids which match the partitioned keys of all
@@ -1524,6 +1530,8 @@ ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo)
 		prunestate->execparams = bms_add_members(prunestate->execparams,
 												 pinfo->execparams);
 
+		prunestate->hasparamlessexprs |= pinfo->hasparamlessexprs;
+
 		relation_close(rel, NoLock);
 
 		i++;
@@ -1566,10 +1574,11 @@ ExecFindInitialMatchingSubPlans(PartitionPruneState *prunestate, int nsubnodes)
 	Bitmapset  *result = NULL;
 
 	/*
-	 * Ensure there's actually external params, or we've not been called
-	 * already.
+	 * If there are no parameter-less Exprs then ensure there's actually
+	 * external params, or we've not been called already.
 	 */
-	Assert(!bms_is_empty(prunestate->extparams));
+	Assert(prunestate->hasparamlessexprs ||
+		   !bms_is_empty(prunestate->extparams));
 
 	pprune = prunestate->partprunedata;
 
@@ -1739,14 +1748,14 @@ find_subplans_for_params_recurse(PartitionPruneState *prunestate,
 
 	/*
 	 * We only need to determine the matching partitions if there are any
-	 * params matching the partition key at this level.  If there are no
-	 * matching params, then we can simply return all subnodes which belong to
-	 * this parent partition.  The planner should have already determined
-	 * these to be the minimum possible set.  We must still recursively visit
-	 * any subpartitioned tables as we may find their partition keys match
-	 * some Params at their level.
+	 * params matching the partition key at this level, or if there are any
+	 * parameter-less expressions matching the partition key.  However, if
+	 * it's just parameter-less expressions then we only prune during
+	 * ExecFindInitialMatchingSubPlans, there's no point in doing this from
+	 * ExecFindMatchingSubPlans too, once is enough.
 	 */
-	if (!bms_is_empty(pruneparams))
+	if ((!allparams && pprune->hasparamlessexprs) ||
+		!bms_is_empty(pruneparams))
 	{
 		context->safeparams = pruneparams;
 		partset = get_matching_partitions(context,
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 6bc3e470bf..275254aa65 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -139,10 +139,11 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 												  node->part_prune_infos);
 
 		/*
-		 * When there are external params matching the partition key we may be
-		 * able to prune away Append subplans now.
+		 * When there are parameter-less exprs or any external params matching
+		 * the partition key we may be able to prune away Append subplans now.
 		 */
-		if (!bms_is_empty(prunestate->extparams))
+		if (prunestate->hasparamlessexprs ||
+			!bms_is_empty(prunestate->extparams))
 		{
 			/* Determine which subplans match the external params */
 			validsubplans = ExecFindInitialMatchingSubPlans(prunestate,
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 7c045a7afe..78b737b2b9 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -2174,6 +2174,7 @@ _copyPartitionPruneInfo(const PartitionPruneInfo *from)
 	COPY_SCALAR_FIELD(reloid);
 	COPY_NODE_FIELD(pruning_steps);
 	COPY_BITMAPSET_FIELD(present_parts);
+	COPY_SCALAR_FIELD(hasparamlessexprs);
 	COPY_SCALAR_FIELD(nparts);
 	COPY_POINTER_FIELD(subnode_map, from->nparts * sizeof(int));
 	COPY_POINTER_FIELD(subpart_map, from->nparts * sizeof(int));
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 610f9edaf5..846055d68d 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -1741,6 +1741,7 @@ _outPartitionPruneInfo(StringInfo str, const PartitionPruneInfo *node)
 	WRITE_OID_FIELD(reloid);
 	WRITE_NODE_FIELD(pruning_steps);
 	WRITE_BITMAPSET_FIELD(present_parts);
+	WRITE_BOOL_FIELD(hasparamlessexprs);
 	WRITE_INT_FIELD(nparts);
 
 	appendStringInfoString(str, " :subnode_map");
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 2826cec2f8..67f30f431e 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1362,6 +1362,7 @@ _readPartitionPruneInfo(void)
 	READ_OID_FIELD(reloid);
 	READ_NODE_FIELD(pruning_steps);
 	READ_BITMAPSET_FIELD(present_parts);
+	READ_BOOL_FIELD(hasparamlessexprs);
 	READ_INT_FIELD(nparts);
 	READ_INT_ARRAY(subnode_map, local_node->nparts);
 	READ_INT_ARRAY(subpart_map, local_node->nparts);
diff --git a/src/backend/partitioning/partprune.c b/src/backend/partitioning/partprune.c
index 58ec2a684d..cb2969403d 100644
--- a/src/backend/partitioning/partprune.c
+++ b/src/backend/partitioning/partprune.c
@@ -53,6 +53,7 @@
 #include "optimizer/planner.h"
 #include "optimizer/predtest.h"
 #include "optimizer/prep.h"
+#include "optimizer/var.h"
 #include "partitioning/partprune.h"
 #include "partitioning/partbounds.h"
 #include "rewrite/rewriteManip.h"
@@ -115,6 +116,24 @@ typedef struct PruneStepResult
 	bool		scan_null;		/* Scan the partition for NULL values? */
 } PruneStepResult;
 
+/*
+ * expression_tree_walker context struct for gathering paramids of params
+ * matching the partition key.
+ */
+typedef struct PullParamContext
+{
+	Bitmapset *extparams;
+	Bitmapset *execparams;
+} PullParamContext;
+
+/*
+ * expression_tree_walker context struct for checking if an Expr contains any
+ * Params not listed in 'safeparams'.
+ */
+typedef struct SafeParamContext
+{
+	Bitmapset *safeparams;
+} SafeParamContext;
 
 static List *gen_partprune_steps(RelOptInfo *rel, List *clauses,
 					bool *contradictory);
@@ -162,6 +181,8 @@ static PruneStepResult *get_matching_list_bounds(PartitionPruneContext *context,
 static PruneStepResult *get_matching_range_bounds(PartitionPruneContext *context,
 						  StrategyNumber opstrategy, Datum *values, int nvalues,
 						  FmgrInfo *partsupfunc, Bitmapset *nullkeys);
+static void pull_params(Expr *expr, PartitionPruneInfo *pinfo);
+static bool pull_params_walker(Node *node, PullParamContext *context);
 static bool pull_partkey_params(PartitionPruneInfo *pinfo, List *steps);
 static PruneStepResult *perform_pruning_base_step(PartitionPruneContext *context,
 						  PartitionPruneStepOp *opstep);
@@ -172,6 +193,9 @@ static bool match_boolean_partition_clause(Oid partopfamily, Expr *clause,
 							   Expr *partkey, Expr **outconst);
 static bool partkey_datum_from_expr(PartitionPruneContext *context,
 						Expr *expr, int stateidx, Datum *value);
+static bool contains_only_safeparams_walker(Node *node,
+								SafeParamContext *context);
+static bool contains_only_safeparams(Expr *expr, Bitmapset *safeparams);
 
 /*
  * make_partition_pruneinfo
@@ -197,7 +221,7 @@ make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
 	int		   *relid_subnode_map;
 	int		   *relid_subpart_map;
 	int			i;
-	bool		gotparam = false;
+	bool		gotnonconst = false;
 
 	/*
 	 * Allocate two arrays to store the 1-based indexes of the 'subpaths' and
@@ -326,11 +350,12 @@ make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
 		pinfo->subpart_map = subpart_map;
 
 		/*
-		 * Extract Params matching partition key and record if we got any.
-		 * We'll not bother enabling run-time pruning if no params matched the
-		 * partition key at any level of partitioning.
+		 * Extract Params matching partition key and record if any steps
+		 * compare a non-Const value to the partition key.  If everything
+		 * is Const then we've no need to perform run-time pruning as the
+		 * planner will have already selected the minimum set of partitions.
 		 */
-		gotparam |= pull_partkey_params(pinfo, pruning_steps);
+		gotnonconst |= pull_partkey_params(pinfo, pruning_steps);
 
 		pinfolist = lappend(pinfolist, pinfo);
 	}
@@ -338,14 +363,11 @@ make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
 	pfree(relid_subnode_map);
 	pfree(relid_subpart_map);
 
-	if (gotparam)
+	/* Enable pruning if we got any non-Consts */
+	if (gotnonconst)
 		return pinfolist;
 
-	/*
-	 * If no Params were found to match the partition key on any of the
-	 * partitioned relations then there's no point doing any run-time
-	 * partition pruning.
-	 */
+	/* Run-time pruning would be useless */
 	return NIL;
 }
 
@@ -1478,6 +1500,11 @@ match_clause_to_partition_key(RelOptInfo *rel,
 		if (contain_volatile_functions((Node *) expr))
 			return PARTCLAUSE_UNSUPPORTED;
 
+		/* We can't prune using an expression with Vars */
+		/* XXX this only checks for vars at level 0, We need to disable any Var */
+		if (contain_var_clause((Node *) expr))
+			return PARTCLAUSE_UNSUPPORTED;
+
 		/*
 		 * Determine the input types of the operator we're considering.
 		 *
@@ -2682,16 +2709,74 @@ get_matching_range_bounds(PartitionPruneContext *context,
 	return result;
 }
 
+/*
+ * pull_params
+ *		Determine all external and exec params in 'expr' and add the found
+ *		paramids to the appropriate 'pinfo' field.  pinfo's hasparamlessexprs
+ *		field is set if any non-Const expression is found which have no
+ *		parameters.
+ */
+static void
+pull_params(Expr *expr, PartitionPruneInfo *pinfo)
+{
+	PullParamContext context;
+
+	context.extparams = NULL;
+	context.execparams = NULL;
+
+	pull_params_walker((Node *) expr, &context);
+
+	pinfo->extparams = bms_add_members(pinfo->extparams, context.extparams);
+	pinfo->execparams = bms_add_members(pinfo->execparams,
+										context.execparams);
+
+	/* Mark that an Expr has been seen which contains no Params */
+	if (!context.extparams && !context.execparams)
+		pinfo->hasparamlessexprs = true;
+}
+
+static bool
+pull_params_walker(Node *node, PullParamContext *context)
+{
+	if (node == NULL)
+		return false;
+	if (IsA(node, Param))
+	{
+		Param	   *param = (Param *) node;
+
+		switch (param->paramkind)
+		{
+			case PARAM_EXTERN:
+				context->extparams = bms_add_member(context->extparams,
+													param->paramid);
+				break;
+			case PARAM_EXEC:
+				context->execparams = bms_add_member(context->execparams,
+													 param->paramid);
+				break;
+
+			default:
+				elog(ERROR, "unrecognized paramkind: %d",
+					 (int) param->paramkind);
+				break;
+		}
+	}
+	return expression_tree_walker(node, pull_params_walker,
+								  (void *) context);
+}
+
 /*
  * pull_partkey_params
  *		Loop through each pruning step and record each external and exec
  *		Params being compared to the partition keys.
+ *
+ * Returns true if any non-const value is being compared to the partition key.
  */
 static bool
 pull_partkey_params(PartitionPruneInfo *pinfo, List *steps)
 {
 	ListCell   *lc;
-	bool		gotone = false;
+	bool		gotnonconst = false;
 
 	foreach(lc, steps)
 	{
@@ -2705,32 +2790,15 @@ pull_partkey_params(PartitionPruneInfo *pinfo, List *steps)
 		{
 			Expr	   *expr = lfirst(lc2);
 
-			if (IsA(expr, Param))
-			{
-				Param	   *param = (Param *) expr;
-
-				switch (param->paramkind)
-				{
-					case PARAM_EXTERN:
-						pinfo->extparams = bms_add_member(pinfo->extparams,
-														  param->paramid);
-						break;
-					case PARAM_EXEC:
-						pinfo->execparams = bms_add_member(pinfo->execparams,
-														   param->paramid);
-						break;
+			if (IsA(expr, Const))
+				continue;
 
-					default:
-						elog(ERROR, "unrecognized paramkind: %d",
-							 (int) param->paramkind);
-						break;
-				}
-				gotone = true;
-			}
+			gotnonconst = true;
+			pull_params(expr, pinfo);
 		}
 	}
 
-	return gotone;
+	return gotnonconst;
 }
 
 /*
@@ -3031,38 +3099,62 @@ static bool
 partkey_datum_from_expr(PartitionPruneContext *context,
 						Expr *expr, int stateidx, Datum *value)
 {
-	switch (nodeTag(expr))
+	if (IsA(expr, Const))
+	{
+		*value = ((Const *) expr)->constvalue;
+		return true;
+	}
+	else
 	{
-		case T_Const:
-			*value = ((Const *) expr)->constvalue;
+		/*
+		 * When being called during planning constant folding the Param's
+		 * value.
+		 */
+		if (context->planstate &&
+			contains_only_safeparams(expr, context->safeparams))
+		{
+			ExprState  *exprstate;
+			ExprContext *ectx;
+			bool		isNull;
+
+			/* Exprs with volatile functions shouldn't make it here */
+			Assert(!contain_volatile_functions((Node *) expr));
+
+			exprstate = context->exprstates[stateidx];
+			ectx = context->planstate->ps_ExprContext;
+			*value = ExecEvalExprSwitchContext(exprstate, ectx, &isNull);
+			if (isNull)
+				return false;
+
 			return true;
+		}
+	}
 
-		case T_Param:
+	return false;
+}
 
-			/*
-			 * When being called from the executor we may be able to evaluate
-			 * the Param's value.
-			 */
-			if (context->planstate &&
-				bms_is_member(((Param *) expr)->paramid, context->safeparams))
-			{
-				ExprState  *exprstate;
-				ExprContext *ectx;
-				bool		isNull;
+static bool
+contains_only_safeparams(Expr *expr, Bitmapset *safeparams)
+{
+	SafeParamContext context;
 
-				exprstate = context->exprstates[stateidx];
-				ectx = context->planstate->ps_ExprContext;
-				*value = ExecEvalExprSwitchContext(exprstate, ectx, &isNull);
-				if (isNull)
-					return false;
+	context.safeparams = safeparams;
 
-				return true;
-			}
-			break;
+	return contains_only_safeparams_walker((Node *) expr, &context);
+}
 
-		default:
-			break;
+static bool
+contains_only_safeparams_walker(Node *node, SafeParamContext *context)
+{
+	if (node == NULL)
+		return false;
+	if (IsA(node, Param))
+	{
+		Param *param = (Param *) node;
+		return bms_is_member(param->paramid, context->safeparams);
 	}
+	(void) expression_tree_walker(node, contains_only_safeparams_walker,
+								  (void *) context);
 
-	return false;
+	return true;
 }
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index fc6e9574e3..a48cf72d8c 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -136,6 +136,8 @@ typedef struct PartitionTupleRouting
  * extparams					Contains paramids of external params found
  *								matching partition keys in 'pruning_steps'.
  * allparams					As 'extparams' but also including exec params.
+ * hasparamlessexprs			Some pruning steps contain Exprs without any
+ *								Params.
  *-----------------------
  */
 typedef struct PartitionPruningData
@@ -147,6 +149,7 @@ typedef struct PartitionPruningData
 	List	   *pruning_steps;
 	Bitmapset  *extparams;
 	Bitmapset  *allparams;
+	bool		hasparamlessexprs;
 } PartitionPruningData;
 
 /*-----------------------
@@ -163,6 +166,7 @@ typedef struct PartitionPruningData
  *						partitioned relation. First element contains the
  *						details for the target partitioned table.
  * num_partprunedata	Number of items in 'partprunedata' array.
+ * hasparamlessexprs	Some pruning steps contain Exprs without any Params.
  * prune_context		A memory context which can be used to call the query
  *						planner's partition prune functions.
  * extparams			All PARAM_EXTERN paramids which were found to match a
@@ -177,6 +181,7 @@ typedef struct PartitionPruneState
 {
 	PartitionPruningData *partprunedata;
 	int			num_partprunedata;
+	bool		hasparamlessexprs;
 	MemoryContext prune_context;
 	Bitmapset  *extparams;
 	Bitmapset  *execparams;
diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h
index f90aa7b2a1..aae102126f 100644
--- a/src/include/nodes/primnodes.h
+++ b/src/include/nodes/primnodes.h
@@ -1597,6 +1597,8 @@ typedef struct PartitionPruneInfo
 	List	   *pruning_steps;	/* List of PartitionPruneStep */
 	Bitmapset  *present_parts;	/* Indexes of all partitions which subnodes
 								 * are present for. */
+	bool		hasparamlessexprs;	/* True if Exprs exist which don't contain
+									 * any Params */
 	int			nparts;			/* The length of the following two arrays */
 	int		   *subnode_map;	/* subnode index by partition id, or -1 */
 	int		   *subpart_map;	/* subpart index by partition id, or -1 */
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index cf331e79c1..64b4e933d6 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -1835,6 +1835,54 @@ fetch backward all from cur;
 (2 rows)
 
 commit;
+begin;
+-- Test run-time pruning using stable functions
+create function list_part_fn(int) returns int as $$ begin return $1; end;$$ language plpgsql stable;
+-- Ensure pruning works using a stable function containing no Vars
+explain (analyze, costs off, summary off, timing off) select * from list_part where a = list_part_fn(1);
+                      QUERY PLAN                      
+------------------------------------------------------
+ Append (actual rows=1 loops=1)
+   Subplans Removed: 3
+   ->  Seq Scan on list_part1 (actual rows=1 loops=1)
+         Filter: (a = list_part_fn(1))
+(4 rows)
+
+-- Ensure pruning does not take place when the function contains a Var parameter
+explain (analyze, costs off, summary off, timing off) select * from list_part where a = list_part_fn(a);
+                      QUERY PLAN                      
+------------------------------------------------------
+ Append (actual rows=4 loops=1)
+   ->  Seq Scan on list_part1 (actual rows=1 loops=1)
+         Filter: (a = list_part_fn(a))
+   ->  Seq Scan on list_part2 (actual rows=1 loops=1)
+         Filter: (a = list_part_fn(a))
+   ->  Seq Scan on list_part3 (actual rows=1 loops=1)
+         Filter: (a = list_part_fn(a))
+   ->  Seq Scan on list_part4 (actual rows=1 loops=1)
+         Filter: (a = list_part_fn(a))
+(9 rows)
+
+-- Ensure pruning does not take place when the expression contains a Var.
+explain (analyze, costs off, summary off, timing off) select * from list_part where a = list_part_fn(1) + a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on list_part1 (actual rows=0 loops=1)
+         Filter: (a = (list_part_fn(1) + a))
+         Rows Removed by Filter: 1
+   ->  Seq Scan on list_part2 (actual rows=0 loops=1)
+         Filter: (a = (list_part_fn(1) + a))
+         Rows Removed by Filter: 1
+   ->  Seq Scan on list_part3 (actual rows=0 loops=1)
+         Filter: (a = (list_part_fn(1) + a))
+         Rows Removed by Filter: 1
+   ->  Seq Scan on list_part4 (actual rows=0 loops=1)
+         Filter: (a = (list_part_fn(1) + a))
+         Rows Removed by Filter: 1
+(13 rows)
+
+rollback;
 drop table list_part;
 -- Parallel append
 -- Suppress the number of loops each parallel node runs for.  This is because
@@ -2079,6 +2127,40 @@ select explain_parallel_append('select avg(ab.a) from ab inner join lprt_a a on
                                  Index Cond: (a = a.a)
 (27 rows)
 
+-- Ensure the same partitions are pruned when we make the nested loop
+-- parameter an Expr rather than a plain Param.
+select explain_parallel_append('select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a + 0 where a.a in(0, 0, 1)');
+                                      explain_parallel_append                                      
+---------------------------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=2 loops=1)
+         Workers Planned: 1
+         Workers Launched: 1
+         ->  Partial Aggregate (actual rows=1 loops=2)
+               ->  Nested Loop (actual rows=0 loops=2)
+                     ->  Parallel Seq Scan on lprt_a a (actual rows=51 loops=N)
+                           Filter: (a = ANY ('{0,0,1}'::integer[]))
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = (a.a + 0))
+                           ->  Index Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = (a.a + 0))
+                           ->  Index Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = (a.a + 0))
+                           ->  Index Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = (a.a + 0))
+                           ->  Index Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = (a.a + 0))
+                           ->  Index Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = (a.a + 0))
+                           ->  Index Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = (a.a + 0))
+                           ->  Index Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = (a.a + 0))
+                           ->  Index Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = (a.a + 0))
+(27 rows)
+
 insert into lprt_a values(3),(3);
 select explain_parallel_append('select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in(1, 0, 3)');
                                       explain_parallel_append                                      
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index 1464f4dcd9..b6681fa44c 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -396,6 +396,22 @@ fetch backward all from cur;
 
 commit;
 
+begin;
+
+-- Test run-time pruning using stable functions
+create function list_part_fn(int) returns int as $$ begin return $1; end;$$ language plpgsql stable;
+
+-- Ensure pruning works using a stable function containing no Vars
+explain (analyze, costs off, summary off, timing off) select * from list_part where a = list_part_fn(1);
+
+-- Ensure pruning does not take place when the function contains a Var parameter
+explain (analyze, costs off, summary off, timing off) select * from list_part where a = list_part_fn(a);
+
+-- Ensure pruning does not take place when the expression contains a Var.
+explain (analyze, costs off, summary off, timing off) select * from list_part where a = list_part_fn(1) + a;
+
+rollback;
+
 drop table list_part;
 
 -- Parallel append
@@ -486,6 +502,10 @@ set enable_mergejoin = 0;
 
 select explain_parallel_append('select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in(0, 0, 1)');
 
+-- Ensure the same partitions are pruned when we make the nested loop
+-- parameter an Expr rather than a plain Param.
+select explain_parallel_append('select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a + 0 where a.a in(0, 0, 1)');
+
 insert into lprt_a values(3),(3);
 
 select explain_parallel_append('select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in(1, 0, 3)');
